[ https://issues.apache.org/jira/browse/KAFKA-5891?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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 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&password=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)