-
Notifications
You must be signed in to change notification settings - Fork 14.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KAFKA-10675: Add schema name to ConnectSchema.validateValue() error message #9541
Conversation
b6912b9
to
12ddffd
Compare
@C0urante, @gharris1727, could you take a look at this please? |
private static String buildSchemaInfo(Schema schema) { | ||
StringBuilder schemaInfo = new StringBuilder("schema"); | ||
if (schema.name() != null) { | ||
schemaInfo.append(" \"").append(schema.name()).append("\""); | ||
} | ||
schemaInfo.append(" with type ").append(schema.type()); | ||
return schemaInfo.toString(); | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Given that this is only used in one place, and the string that it creates is itself added to a formatted string, I think it'd be a bit more readable if we removed this method and inlined all the logic where it's being invoked right now.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for reviewing! Fixes were pushed
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM, one comment about readability but you can take it or leave it.
I'm also wondering if we might consider dropping the " for field ..." part from the exception message if field
is null (which occurs with the two-argument variant of ConnectSchema::validateValue
), but that can be left as follow-up work if you'd prefer.
9cfcb66
to
c8a361c
Compare
@kkonstantine, could you take a look at this please? |
@rhauch, could you take a look at this please? |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks, @Iskuskov. LGTM.
We're technically still frozen on the 2.6
and 2.8
branches, but both have RCs that passed their PMC votes and should be unblocked soon. I'll hopefully be able to merge and backport soon.
@@ -266,7 +267,7 @@ public static void validateValue(String name, Schema schema, Object value) { | |||
private static List<Class> expectedClassesFor(Schema schema) { | |||
List<Class> expectedClasses = LOGICAL_TYPE_CLASSES.get(schema.name()); | |||
if (expectedClasses == null) | |||
expectedClasses = SCHEMA_TYPE_CLASSES.get(schema.type()); | |||
expectedClasses = SCHEMA_TYPE_CLASSES.getOrDefault(schema.type(), Collections.emptyList()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Strictly speaking, this shouldn't be necessary as SCHEMA_TYPE_CLASSES
should have a Schema
instance for all Schema.Type
literals. And with SchemaBuilder
a connector or converter cannot create a schema instance with a null Schema.Type
.
However, it is possible to construct a ConnectSchema
instance with a null Type
reference (like what FakeSchema
essentially does in the existing test), which of course without this change would result in this method returning a null list.
So +1 for this line change since it simplifies the error handling in the calling code.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thank you! Highly appreciate your comment
@rhauch seems that this fix was approved but we were blocked on release candidates of previous releases. To include to 3.0 (as well as previous releases) and avoid entering a new code freeze phase we'd need to merge asap. Please take another look if possible. @Iskuskov in the meantime a merge conflict has been introduced. Would you mind rebasing on top of the latest changes on |
@kkonstantine thank you! Rebased |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We recently enabled warnings on the use of raw types. That's why the build failed after the rebase.
Adding a suggestion and merging to the PR to allow the tests to run.
connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java
Outdated
Show resolved
Hide resolved
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Test failures are not relevant.
LGTM
…essage (#9541) The following error message `org.apache.kafka.connect.errors.DataException: Invalid Java object for schema type INT64: class java.lang.Long for field: "moderate_time"` can be confusing because java.lang.Long is acceptable type for schema INT64. In fact, in this case `org.apache.kafka.connect.data.Timestamp` is used but this info is not logged. Reviewers: Randall Hauch <[email protected]>, Chris Egerton <[email protected]>, Konstantine Karantasis <[email protected]>
…essage (#9541) The following error message `org.apache.kafka.connect.errors.DataException: Invalid Java object for schema type INT64: class java.lang.Long for field: "moderate_time"` can be confusing because java.lang.Long is acceptable type for schema INT64. In fact, in this case `org.apache.kafka.connect.data.Timestamp` is used but this info is not logged. Reviewers: Randall Hauch <[email protected]>, Chris Egerton <[email protected]>, Konstantine Karantasis <[email protected]>
…essage (#9541) The following error message `org.apache.kafka.connect.errors.DataException: Invalid Java object for schema type INT64: class java.lang.Long for field: "moderate_time"` can be confusing because java.lang.Long is acceptable type for schema INT64. In fact, in this case `org.apache.kafka.connect.data.Timestamp` is used but this info is not logged. Reviewers: Randall Hauch <[email protected]>, Chris Egerton <[email protected]>, Konstantine Karantasis <[email protected]>
…essage (#9541) The following error message `org.apache.kafka.connect.errors.DataException: Invalid Java object for schema type INT64: class java.lang.Long for field: "moderate_time"` can be confusing because java.lang.Long is acceptable type for schema INT64. In fact, in this case `org.apache.kafka.connect.data.Timestamp` is used but this info is not logged. Reviewers: Randall Hauch <[email protected]>, Chris Egerton <[email protected]>, Konstantine Karantasis <[email protected]>
…essage (apache#9541) The following error message `org.apache.kafka.connect.errors.DataException: Invalid Java object for schema type INT64: class java.lang.Long for field: "moderate_time"` can be confusing because java.lang.Long is acceptable type for schema INT64. In fact, in this case `org.apache.kafka.connect.data.Timestamp` is used but this info is not logged. Reviewers: Randall Hauch <[email protected]>, Chris Egerton <[email protected]>, Konstantine Karantasis <[email protected]>
https://issues.apache.org/jira/browse/KAFKA-10675
The following error message
can be confusing because
java.lang.Long
is acceptable type for schemaINT64
.In fact, in this case
org.apache.kafka.connect.data.Timestamp
is used but this info is not logged.Committer Checklist (excluded from commit message)