kafka-jira mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (KAFKA-5996) JsonConverter generates "Mismatching schema" DataException
Date Wed, 14 Feb 2018 16:47:00 GMT

    [ https://issues.apache.org/jira/browse/KAFKA-5996?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16364393#comment-16364393
] 

ASF GitHub Bot commented on KAFKA-5996:
---------------------------------------

hachikuji closed pull request #4523: [KAFKA-5996] JsonConverter generates Mismatching schema
DataException
URL: https://github.com/apache/kafka/pull/4523
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java
index 32ded4498ea..c1322b1ecdb 100644
--- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java
+++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java
@@ -673,7 +673,7 @@ else if (value instanceof ByteBuffer)
                 }
                 case STRUCT: {
                     Struct struct = (Struct) value;
-                    if (struct.schema() != schema)
+                    if (!struct.schema().equals(schema))
                         throw new DataException("Mismatching schema.");
                     ObjectNode obj = JsonNodeFactory.instance.objectNode();
                     for (Field field : schema.fields()) {
diff --git a/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java
b/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java
index 0a710446856..7686fdb7ab5 100644
--- a/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java
+++ b/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java
@@ -563,6 +563,20 @@ public void structToJson() {
                 converted.get(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME));
     }
 
+    @Test
+    public void structSchemaIdentical() {
+        Schema schema = SchemaBuilder.struct().field("field1", Schema.BOOLEAN_SCHEMA)
+                                              .field("field2", Schema.STRING_SCHEMA)
+                                              .field("field3", Schema.STRING_SCHEMA)
+                                              .field("field4", Schema.BOOLEAN_SCHEMA).build();
+        Schema inputSchema = SchemaBuilder.struct().field("field1", Schema.BOOLEAN_SCHEMA)
+                                                   .field("field2", Schema.STRING_SCHEMA)
+                                                   .field("field3", Schema.STRING_SCHEMA)
+                                                   .field("field4", Schema.BOOLEAN_SCHEMA).build();
+        Struct input = new Struct(inputSchema).put("field1", true).put("field2", "string2").put("field3",
"string3").put("field4", false);
+        assertStructSchemaEqual(schema, input);
+    }
+
 
     @Test
     public void decimalToJson() throws IOException {
@@ -735,7 +749,6 @@ public void testJsonSchemaCacheSizeFromConfigFile() throws URISyntaxException,
I
 
         JsonConverter rc = new JsonConverter();
         rc.configure(workerProps, false);
-
     }
 
 
@@ -791,4 +804,9 @@ private void validateEnvelopeNullSchema(JsonNode env) {
         assertTrue(env.get(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME).isNull());
         assertTrue(env.has(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME));
     }
+    
+    private void assertStructSchemaEqual(Schema schema, Struct struct) {
+        converter.fromConnectData(TOPIC, schema, struct);
+        assertEquals(schema, struct.schema());
+    }
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> JsonConverter generates "Mismatching schema" DataException
> ----------------------------------------------------------
>
>                 Key: KAFKA-5996
>                 URL: https://issues.apache.org/jira/browse/KAFKA-5996
>             Project: Kafka
>          Issue Type: Bug
>          Components: KafkaConnect
>    Affects Versions: 0.10.0.0, 0.10.0.1, 0.10.1.0, 0.10.1.1, 0.10.2.0, 0.10.2.1, 0.11.0.0
>            Reporter: Yuqi Li
>            Priority: Major
>
> Run connect-standalone.sh with original connect-standalone.properties and custom jdbc
source connect configuration.
> part of the jdbc source connect configuration:
> connector.class=io.confluent.connect.jdbc.JdbcSourceConnector
> query=Select * from ......
> poll.interval.ms=45000
> transforms=MakeMap, InsertSourceId, InsertSourceEvent
> transforms.MakeMap.type=org.apache.kafka.connect.transforms.HoistField$Value
> transforms.MakeMap.field=emailRequest
> transforms.InsertSourceId.type=org.apache.kafka.connect.transforms.InsertField$Value
> transforms.InsertSourceId.static.field=id
> transforms.InsertSourceId.static.value=1
> transforms.InsertSourceEvent.type=org.apache.kafka.connect.transforms.InsertField$Value
> transforms.InsertSourceEvent.static.field=event
> transforms.InsertSourceEvent.static.value=REMINDER_EMAIL
> When the source connect starts, the first iteration(executing the query) runs ok, but
45 seconds later, it throws following exception:
> org.apache.kafka.connect.errors.DataException: Mismatching schema.
>         at org.apache.kafka.connect.json.JsonConverter.convertToJson(JsonConverter.java:647)
>         at org.apache.kafka.connect.json.JsonConverter.convertToJson(JsonConverter.java:650)
>         at org.apache.kafka.connect.json.JsonConverter.convertToJsonWithEnvelope(JsonConverter.java:537)
>         at org.apache.kafka.connect.json.JsonConverter.fromConnectData(JsonConverter.java:290)
>         at org.apache.kafka.connect.runtime.WorkerSourceTask.sendRecords(WorkerSourceTask.java:198)
>         at org.apache.kafka.connect.runtime.WorkerSourceTask.execute(WorkerSourceTask.java:168)
>         at org.apache.kafka.connect.runtime.WorkerTask.doRun(WorkerTask.java:146)
>         at org.apache.kafka.connect.runtime.WorkerTask.run(WorkerTask.java:190)
>         at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
>         at java.util.concurrent.FutureTask.run(Unknown Source)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>         at java.lang.Thread.run(Unknown Source)
> [2017-09-29 13:59:18,361] ERROR Task is being killed and will not recover until manually
restarted (org.apache.kafka.connect.runtime.WorkerTask:149)
> The SoureTask is being killed due to the exception, and the source connector is dead.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Mime
View raw message