[jira] [Commented] (KAFKA-5891) Cast transformation fails if record schema contains timestamp field
[ https://issues.apache.org/jira/browse/KAFKA-5891?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16587583#comment-16587583 ] ASF GitHub Bot commented on KAFKA-5891: --- hachikuji closed pull request #5537: KAFKA-5891: Adapts #4633 with schema tests URL: https://github.com/apache/kafka/pull/5537 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/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java index 22b19722c47..a593c7b3934 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java @@ -174,7 +174,6 @@ private Schema getOrBuildSchema(Schema valueSchema) { } else { builder.field(field.name(), field.schema()); } - } } diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java index decd043b1db..06fbe311c16 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java @@ -337,6 +337,19 @@ public void castFieldsWithSchema() { assertEquals(42, ((Struct) transformed.value()).get("string")); assertEquals(new Date(0), ((Struct) transformed.value()).get("timestamp")); assertNull(((Struct) transformed.value()).get("optional")); + +Schema transformedSchema = ((Struct) transformed.value()).schema(); +assertEquals(Schema.INT16_SCHEMA.type(), transformedSchema.field("int8").schema().type()); +assertEquals(Schema.OPTIONAL_INT32_SCHEMA.type(), transformedSchema.field("int16").schema().type()); +assertEquals(Schema.INT64_SCHEMA.type(), transformedSchema.field("int32").schema().type()); +assertEquals(Schema.BOOLEAN_SCHEMA.type(), transformedSchema.field("int64").schema().type()); +assertEquals(Schema.FLOAT64_SCHEMA.type(), transformedSchema.field("float32").schema().type()); +assertEquals(Schema.BOOLEAN_SCHEMA.type(), transformedSchema.field("float64").schema().type()); +assertEquals(Schema.INT8_SCHEMA.type(), transformedSchema.field("boolean").schema().type()); +assertEquals(Schema.INT32_SCHEMA.type(), transformedSchema.field("string").schema().type()); +assertEquals(Schema.OPTIONAL_INT32_SCHEMA.type(), transformedSchema.field("optional").schema().type()); +// The following fields are not changed +assertEquals(Timestamp.SCHEMA.type(), transformedSchema.field("timestamp").schema().type()); } @SuppressWarnings("unchecked") 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: us...@infra.apache.org > Cast transformation fails if record schema contains timestamp field > --- > > Key: KAFKA-5891 > URL: https://issues.apache.org/jira/browse/KAFKA-5891 > Project: Kafka > Issue Type: Bug > Components: KafkaConnect >Affects Versions: 0.11.0.0 >Reporter: Artem Plotnikov >Priority: Major > Fix For: 1.0.3, 1.1.2, 2.0.1 > > > I have the following simple type cast transformation: > {code} > name=postgresql-source-simple > connector.class=io.confluent.connect.jdbc.JdbcSourceConnector > tasks.max=1 > connection.url=jdbc:postgresql://localhost:5432/testdb?user=postgres=mysecretpassword > query=SELECT 1::INT as a, '2017-09-14 10:23:54'::TIMESTAMP as b > transforms=Cast > transforms.Cast.type=org.apache.kafka.connect.transforms.Cast$Value > transforms.Cast.spec=a:boolean > mode=bulk > topic.prefix=clients > {code} > Which fails with the following exception in runtime: > {code} > [2017-09-14 16:51:01,885] ERROR Task postgresql-source-simple-0 threw an > uncaught and unrecoverable exception > (org.apache.kafka.connect.runtime.WorkerTask:148) > org.apache.kafka.connect.errors.DataException: Invalid Java object for schema > type INT64: class java.sql.Timestamp for field: "null" > at > org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:239) > at >
[jira] [Commented] (KAFKA-5891) Cast transformation fails if record schema contains timestamp field
[ https://issues.apache.org/jira/browse/KAFKA-5891?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16586665#comment-16586665 ] ASF GitHub Bot commented on KAFKA-5891: --- hachikuji closed pull request #4633: KAFKA-5891: Proper handling of LogicalTypes in Cast URL: https://github.com/apache/kafka/pull/4633 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/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java index d94f8f648b3..22b19722c47 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/Cast.java @@ -164,13 +164,17 @@ private Schema getOrBuildSchema(Schema valueSchema) { } else { builder = SchemaUtil.copySchemaBasics(valueSchema, SchemaBuilder.struct()); for (Field field : valueSchema.fields()) { -SchemaBuilder fieldBuilder = -convertFieldType(casts.containsKey(field.name()) ? casts.get(field.name()) : field.schema().type()); -if (field.schema().isOptional()) -fieldBuilder.optional(); -if (field.schema().defaultValue() != null) - fieldBuilder.defaultValue(castValueToType(field.schema().defaultValue(), fieldBuilder.type())); -builder.field(field.name(), fieldBuilder.build()); +if (casts.containsKey(field.name())) { +SchemaBuilder fieldBuilder = convertFieldType(casts.get(field.name())); +if (field.schema().isOptional()) +fieldBuilder.optional(); +if (field.schema().defaultValue() != null) + fieldBuilder.defaultValue(castValueToType(field.schema().defaultValue(), fieldBuilder.type())); +builder.field(field.name(), fieldBuilder.build()); +} else { +builder.field(field.name(), field.schema()); +} + } } diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java index b190189b35d..decd043b1db 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/CastTest.java @@ -21,12 +21,14 @@ import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.data.Timestamp; import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.source.SourceRecord; import org.junit.After; import org.junit.Test; import java.util.Collections; +import java.util.Date; import java.util.HashMap; import java.util.Map; @@ -304,6 +306,7 @@ public void castFieldsWithSchema() { builder.field("boolean", Schema.BOOLEAN_SCHEMA); builder.field("string", Schema.STRING_SCHEMA); builder.field("optional", Schema.OPTIONAL_FLOAT32_SCHEMA); +builder.field("timestamp", Timestamp.SCHEMA); Schema supportedTypesSchema = builder.build(); Struct recordValue = new Struct(supportedTypesSchema); @@ -315,6 +318,7 @@ public void castFieldsWithSchema() { recordValue.put("float64", -64.); recordValue.put("boolean", true); recordValue.put("string", "42"); +recordValue.put("timestamp", new Date(0)); // optional field intentionally omitted SourceRecord transformed = xformValue.apply(new SourceRecord(null, null, "topic", 0, @@ -331,6 +335,7 @@ public void castFieldsWithSchema() { assertEquals(true, ((Struct) transformed.value()).schema().field("float64").schema().defaultValue()); assertEquals((byte) 1, ((Struct) transformed.value()).get("boolean")); assertEquals(42, ((Struct) transformed.value()).get("string")); +assertEquals(new Date(0), ((Struct) transformed.value()).get("timestamp")); assertNull(((Struct) transformed.value()).get("optional")); } 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: us...@infra.apache.org > Cast transformation fails if record schema contains
[jira] [Commented] (KAFKA-5891) Cast transformation fails if record schema contains timestamp field
[ https://issues.apache.org/jira/browse/KAFKA-5891?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16586573#comment-16586573 ] ASF GitHub Bot commented on KAFKA-5891: --- rayokota opened a new pull request #5537: KAFKA-5891: Adapts #4633 with schema tests URL: https://github.com/apache/kafka/pull/5537 This adapts PR #4633 from @maver1ck with suggested unit test from @rhauch ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementation - [ ] Verify test coverage and CI build status - [ ] Verify documentation (including upgrade notes) 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: us...@infra.apache.org > Cast transformation fails if record schema contains timestamp field > --- > > Key: KAFKA-5891 > URL: https://issues.apache.org/jira/browse/KAFKA-5891 > Project: Kafka > Issue Type: Bug > Components: KafkaConnect >Affects Versions: 0.11.0.0 >Reporter: Artem Plotnikov >Priority: Major > > I have the following simple type cast transformation: > {code} > name=postgresql-source-simple > connector.class=io.confluent.connect.jdbc.JdbcSourceConnector > tasks.max=1 > connection.url=jdbc:postgresql://localhost:5432/testdb?user=postgres=mysecretpassword > query=SELECT 1::INT as a, '2017-09-14 10:23:54'::TIMESTAMP as b > transforms=Cast > transforms.Cast.type=org.apache.kafka.connect.transforms.Cast$Value > transforms.Cast.spec=a:boolean > mode=bulk > topic.prefix=clients > {code} > Which fails with the following exception in runtime: > {code} > [2017-09-14 16:51:01,885] ERROR Task postgresql-source-simple-0 threw an > uncaught and unrecoverable exception > (org.apache.kafka.connect.runtime.WorkerTask:148) > org.apache.kafka.connect.errors.DataException: Invalid Java object for schema > type INT64: class java.sql.Timestamp for field: "null" > at > org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:239) > at > org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:209) > at org.apache.kafka.connect.data.Struct.put(Struct.java:214) > at > org.apache.kafka.connect.transforms.Cast.applyWithSchema(Cast.java:152) > at org.apache.kafka.connect.transforms.Cast.apply(Cast.java:108) > at > org.apache.kafka.connect.runtime.TransformationChain.apply(TransformationChain.java:38) > at > org.apache.kafka.connect.runtime.WorkerSourceTask.sendRecords(WorkerSourceTask.java:190) > 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(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) > {code} > If I remove the transforms.* part of the connector it will work correctly. > Actually, it doesn't really matter which types I use in the transformation > for field 'a', just the existence of a timestamp field brings the exception. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (KAFKA-5891) Cast transformation fails if record schema contains timestamp field
[ https://issues.apache.org/jira/browse/KAFKA-5891?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16406667#comment-16406667 ] Randall Hauch commented on KAFKA-5891: -- KAFKA-6290 is another issue related to the CAST transformation, but deals with actually being able to CAST fields that use logical types. This issue, OTOH, is simply that the record's Struct can only contain fields that have a castable (i.e., currently primitive) type. > Cast transformation fails if record schema contains timestamp field > --- > > Key: KAFKA-5891 > URL: https://issues.apache.org/jira/browse/KAFKA-5891 > Project: Kafka > Issue Type: Bug > Components: KafkaConnect >Affects Versions: 0.11.0.0 >Reporter: Artem Plotnikov >Priority: Major > > I have the following simple type cast transformation: > {code} > name=postgresql-source-simple > connector.class=io.confluent.connect.jdbc.JdbcSourceConnector > tasks.max=1 > connection.url=jdbc:postgresql://localhost:5432/testdb?user=postgres=mysecretpassword > query=SELECT 1::INT as a, '2017-09-14 10:23:54'::TIMESTAMP as b > transforms=Cast > transforms.Cast.type=org.apache.kafka.connect.transforms.Cast$Value > transforms.Cast.spec=a:boolean > mode=bulk > topic.prefix=clients > {code} > Which fails with the following exception in runtime: > {code} > [2017-09-14 16:51:01,885] ERROR Task postgresql-source-simple-0 threw an > uncaught and unrecoverable exception > (org.apache.kafka.connect.runtime.WorkerTask:148) > org.apache.kafka.connect.errors.DataException: Invalid Java object for schema > type INT64: class java.sql.Timestamp for field: "null" > at > org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:239) > at > org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:209) > at org.apache.kafka.connect.data.Struct.put(Struct.java:214) > at > org.apache.kafka.connect.transforms.Cast.applyWithSchema(Cast.java:152) > at org.apache.kafka.connect.transforms.Cast.apply(Cast.java:108) > at > org.apache.kafka.connect.runtime.TransformationChain.apply(TransformationChain.java:38) > at > org.apache.kafka.connect.runtime.WorkerSourceTask.sendRecords(WorkerSourceTask.java:190) > 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(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) > {code} > If I remove the transforms.* part of the connector it will work correctly. > Actually, it doesn't really matter which types I use in the transformation > for field 'a', just the existence of a timestamp field brings the exception. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (KAFKA-5891) Cast transformation fails if record schema contains timestamp field
[ https://issues.apache.org/jira/browse/KAFKA-5891?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16383227#comment-16383227 ] Maciej BryĆski commented on KAFKA-5891: --- [~broartem] Could you test my patch ? > Cast transformation fails if record schema contains timestamp field > --- > > Key: KAFKA-5891 > URL: https://issues.apache.org/jira/browse/KAFKA-5891 > Project: Kafka > Issue Type: Bug > Components: KafkaConnect >Affects Versions: 0.11.0.0 >Reporter: Artem Plotnikov >Priority: Major > > I have the following simple type cast transformation: > {code} > name=postgresql-source-simple > connector.class=io.confluent.connect.jdbc.JdbcSourceConnector > tasks.max=1 > connection.url=jdbc:postgresql://localhost:5432/testdb?user=postgres=mysecretpassword > query=SELECT 1::INT as a, '2017-09-14 10:23:54'::TIMESTAMP as b > transforms=Cast > transforms.Cast.type=org.apache.kafka.connect.transforms.Cast$Value > transforms.Cast.spec=a:boolean > mode=bulk > topic.prefix=clients > {code} > Which fails with the following exception in runtime: > {code} > [2017-09-14 16:51:01,885] ERROR Task postgresql-source-simple-0 threw an > uncaught and unrecoverable exception > (org.apache.kafka.connect.runtime.WorkerTask:148) > org.apache.kafka.connect.errors.DataException: Invalid Java object for schema > type INT64: class java.sql.Timestamp for field: "null" > at > org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:239) > at > org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:209) > at org.apache.kafka.connect.data.Struct.put(Struct.java:214) > at > org.apache.kafka.connect.transforms.Cast.applyWithSchema(Cast.java:152) > at org.apache.kafka.connect.transforms.Cast.apply(Cast.java:108) > at > org.apache.kafka.connect.runtime.TransformationChain.apply(TransformationChain.java:38) > at > org.apache.kafka.connect.runtime.WorkerSourceTask.sendRecords(WorkerSourceTask.java:190) > 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(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) > {code} > If I remove the transforms.* part of the connector it will work correctly. > Actually, it doesn't really matter which types I use in the transformation > for field 'a', just the existence of a timestamp field brings the exception. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (KAFKA-5891) Cast transformation fails if record schema contains timestamp field
[ https://issues.apache.org/jira/browse/KAFKA-5891?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16382629#comment-16382629 ] ASF GitHub Bot commented on KAFKA-5891: --- maver1ck opened a new pull request #4633: [KAFKA-5891] Proper handling of LogicalTypes in Cast URL: https://github.com/apache/kafka/pull/4633 Currently logical types are dropped during Cast Transformation. This patch fixes this behaviour. ### Committer Checklist (excluded from commit message) - [X] Verify design and implementation - [X] Verify test coverage and CI build status - [X] Verify documentation (including upgrade notes) 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: us...@infra.apache.org > Cast transformation fails if record schema contains timestamp field > --- > > Key: KAFKA-5891 > URL: https://issues.apache.org/jira/browse/KAFKA-5891 > Project: Kafka > Issue Type: Bug > Components: KafkaConnect >Affects Versions: 0.11.0.0 >Reporter: Artem Plotnikov >Priority: Major > > I have the following simple type cast transformation: > {code} > name=postgresql-source-simple > connector.class=io.confluent.connect.jdbc.JdbcSourceConnector > tasks.max=1 > connection.url=jdbc:postgresql://localhost:5432/testdb?user=postgres=mysecretpassword > query=SELECT 1::INT as a, '2017-09-14 10:23:54'::TIMESTAMP as b > transforms=Cast > transforms.Cast.type=org.apache.kafka.connect.transforms.Cast$Value > transforms.Cast.spec=a:boolean > mode=bulk > topic.prefix=clients > {code} > Which fails with the following exception in runtime: > {code} > [2017-09-14 16:51:01,885] ERROR Task postgresql-source-simple-0 threw an > uncaught and unrecoverable exception > (org.apache.kafka.connect.runtime.WorkerTask:148) > org.apache.kafka.connect.errors.DataException: Invalid Java object for schema > type INT64: class java.sql.Timestamp for field: "null" > at > org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:239) > at > org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:209) > at org.apache.kafka.connect.data.Struct.put(Struct.java:214) > at > org.apache.kafka.connect.transforms.Cast.applyWithSchema(Cast.java:152) > at org.apache.kafka.connect.transforms.Cast.apply(Cast.java:108) > at > org.apache.kafka.connect.runtime.TransformationChain.apply(TransformationChain.java:38) > at > org.apache.kafka.connect.runtime.WorkerSourceTask.sendRecords(WorkerSourceTask.java:190) > 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(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) > {code} > If I remove the transforms.* part of the connector it will work correctly. > Actually, it doesn't really matter which types I use in the transformation > for field 'a', just the existence of a timestamp field brings the exception. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (KAFKA-5891) Cast transformation fails if record schema contains timestamp field
[ https://issues.apache.org/jira/browse/KAFKA-5891?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16254822#comment-16254822 ] Sudhir Pradhan commented on KAFKA-5891: --- I am facing same issue when consuming from KAFKA to HDFS with CAST TRANSFORMS. Any pointer please. My Connector : * {code:java} { "name": "hdfs-sink-avro-cast-test-stndln", "config": { "key.converter": "io.confluent.connect.avro.AvroConverter", "key.converter.schema.registry.url": "http://localhost:8081;, "value.converter": "io.confluent.connect.avro.AvroConverter", "value.converter.schema.registry.url": "http://localhost:8081;, "key.converter.schemas.enable": "true", "value.converter.schemas.enable": "true", "internal.key.converter": "org.apache.kafka.connect.json.JsonConverter", "internal.value.converter": "org.apache.kafka.connect.json.JsonConverter", "internal.key.converter.schemas.enable": "false", "internal.value.converter.schemas.enable": "false", "offset.storage.file.filename": "/tmp/connect.offsets.avroHdfsConsumer.casttest.stndln", "offset.flush.interval.ms": "500", "parse.key": "true", "connector.class": "io.confluent.connect.hdfs.HdfsSinkConnector", "hadoop.home": "/usr/lib/hadoop", "hdfs.url": "hdfs://ip-127-34-56-789.us-east-1.compute.interna:8020", "topics": "avro_raw_KFK_SRP_USER_TEST_V,avro_raw_KFK_SRP_PG_HITS_TEST_V", "tasks.max": "1", "topics.dir": "/home/hadoop/kafka/data/streams/in/raw/casttest1", "logs.dir": "/home/hadoop/kafka/wal/streams/in/raw/casttest1", "hive.integration": "true", "hive.metastore.uris": "thrift://ip-127-34-56-789.us-east-1.compute.internal:9083", "schema.compatibility": "BACKWARD", "flush.size": "1", "rotate.interval.ms": "1000", "mode": "timestamp", "transforms": "Cast", "transforms.Cast.type": "org.apache.kafka.connect.transforms.Cast$Value", "transforms.Cast.spec": "residuals:float64,comp:float64" } } {code} *** Exception : {code:java} [2017-11-16 01:14:39,719] ERROR Task hdfs-sink-avro-cast-test-stndln-0 threw an uncaught and unrecoverable exception (org.apache.kafka.connect.runtime.WorkerTask:148) org.apache.kafka.connect.errors.DataException: Invalid Java object for schema type INT64: class java.util.Date for field: "null" at org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:239) at org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:209) at org.apache.kafka.connect.data.Struct.put(Struct.java:214) at org.apache.kafka.connect.transforms.Cast.applyWithSchema(Cast.java:152) at org.apache.kafka.connect.transforms.Cast.apply(Cast.java:108) at org.apache.kafka.connect.runtime.TransformationChain.apply(TransformationChain.java:38) at org.apache.kafka.connect.runtime.WorkerSinkTask.convertMessages(WorkerSinkTask.java:414) at org.apache.kafka.connect.runtime.WorkerSinkTask.poll(WorkerSinkTask.java:250) at org.apache.kafka.connect.runtime.WorkerSinkTask.iteration(WorkerSinkTask.java:180) at org.apache.kafka.connect.runtime.WorkerSinkTask.execute(WorkerSinkTask.java:148) 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(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) [2017-11-16 01:14:39,719] ERROR Task is being killed and will not recover until manually restarted (org.apache.kafka.connect.runtime.WorkerTask:149) [2017-11-16 01:14:39,719] INFO Shutting down Hive executor service. (io.confluent.connect.hdfs.DataWriter:309) {code} > Cast transformation fails if record schema contains timestamp field > --- > > Key: KAFKA-5891 > URL: https://issues.apache.org/jira/browse/KAFKA-5891 > Project: Kafka > Issue Type: Bug > Components: KafkaConnect >Affects Versions: 0.11.0.0 >Reporter: Artem Plotnikov > > I have the following simple type cast transformation: > {code} > name=postgresql-source-simple > connector.class=io.confluent.connect.jdbc.JdbcSourceConnector > tasks.max=1 > connection.url=jdbc:postgresql://localhost:5432/testdb?user=postgres=mysecretpassword > query=SELECT 1::INT as a, '2017-09-14 10:23:54'::TIMESTAMP as b > transforms=Cast > transforms.Cast.type=org.apache.kafka.connect.transforms.Cast$Value > transforms.Cast.spec=a:boolean > mode=bulk > topic.prefix=clients > {code} > Which fails with the following
[jira] [Commented] (KAFKA-5891) Cast transformation fails if record schema contains timestamp field
[ https://issues.apache.org/jira/browse/KAFKA-5891?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16167493#comment-16167493 ] Artem Plotnikov commented on KAFKA-5891: Seems like Kafka Connect's Cast transformation loses schema information (basically, schema name) while doing type casting. I was able to reproduce this problem with the following test in org.apache.kafka.connect.transforms.CastTest for current trunk repository branch: {code} @SuppressWarnings("unchecked") @Test public void castWholeRecordValueWithSchemaBooleanAndTimestampField() { final Cast xform = new Cast.Value<>(); xform.configure(Collections.singletonMap(Cast.SPEC_CONFIG, "int64:boolean")); SchemaBuilder builder = SchemaBuilder.struct(); builder.field("int64", Schema.INT64_SCHEMA); builder.field("timestamp", Timestamp.SCHEMA); Schema supportedTypesSchema = builder.build(); Struct recordValue = new Struct(supportedTypesSchema); recordValue.put("int64", (long) 64); recordValue.put("timestamp", new java.sql.Timestamp(0L)); SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0, supportedTypesSchema, recordValue)); assertEquals(true, ((Struct) transformed.value()).get("int64")); assertEquals(new java.sql.Timestamp(0L), ((Struct) transformed.value()).get("timestamp")); } {code} The problem is that Timestamp.SCHEMA has schema.type = 'INT64' and schema.name = "org.apache.kafka.connect.data.Timestamp", but org.apache.kafka.connect.transforms.Cast#getOrBuildSchema method copies schema.type only: {code} SchemaBuilder fieldBuilder = convertFieldType(casts.containsKey(field.name()) ? casts.get(field.name()) : field.schema().type()); {code} > Cast transformation fails if record schema contains timestamp field > --- > > Key: KAFKA-5891 > URL: https://issues.apache.org/jira/browse/KAFKA-5891 > Project: Kafka > Issue Type: Bug > Components: KafkaConnect >Affects Versions: 0.11.0.0 >Reporter: Artem Plotnikov > > I have the following simple type cast transformation: > {code} > name=postgresql-source-simple > connector.class=io.confluent.connect.jdbc.JdbcSourceConnector > tasks.max=1 > connection.url=jdbc:postgresql://localhost:5432/testdb?user=postgres=mysecretpassword > query=SELECT 1::INT as a, '2017-09-14 10:23:54'::TIMESTAMP as b > transforms=Cast > transforms.Cast.type=org.apache.kafka.connect.transforms.Cast$Value > transforms.Cast.spec=a:boolean > mode=bulk > topic.prefix=clients > {code} > Which fails with the following exception in runtime: > {code} > [2017-09-14 16:51:01,885] ERROR Task postgresql-source-simple-0 threw an > uncaught and unrecoverable exception > (org.apache.kafka.connect.runtime.WorkerTask:148) > org.apache.kafka.connect.errors.DataException: Invalid Java object for schema > type INT64: class java.sql.Timestamp for field: "null" > at > org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:239) > at > org.apache.kafka.connect.data.ConnectSchema.validateValue(ConnectSchema.java:209) > at org.apache.kafka.connect.data.Struct.put(Struct.java:214) > at > org.apache.kafka.connect.transforms.Cast.applyWithSchema(Cast.java:152) > at org.apache.kafka.connect.transforms.Cast.apply(Cast.java:108) > at > org.apache.kafka.connect.runtime.TransformationChain.apply(TransformationChain.java:38) > at > org.apache.kafka.connect.runtime.WorkerSourceTask.sendRecords(WorkerSourceTask.java:190) > 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(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) > {code} > If I remove the transforms.* part of the connector it will work correctly. > Actually, it doesn't really matter which types I use in the transformation > for field 'a', just the existence of a timestamp field brings the exception. -- This message was sent by Atlassian JIRA (v6.4.14#64029)