diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/json_sr_-_struct_must_be_converted_to_String/7.4.0_1667521016436/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/json_sr_-_struct_must_be_converted_to_String/7.4.0_1667521016436/plan.json new file mode 100644 index 000000000000..56cfc6945f97 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/json_sr_-_struct_must_be_converted_to_String/7.4.0_1667521016436/plan.json @@ -0,0 +1,230 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (FOO STRUCT) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', PARTITIONS=1, VALUE_FORMAT='JSON_SR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`FOO` STRUCT<`F0` INTEGER, `F1` STRING>", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON_SR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (FOO STRING) WITH (KAFKA_TOPIC='test_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='JSON_SR');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`FOO` STRING", + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON_SR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`FOO` STRING", + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON_SR", + "properties" : { } + } + }, + "windowInfo" : null, + "orReplace" : false, + "isSource" : false + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON_SR", + "properties" : { } + } + }, + "timestampColumn" : null, + "sourceSchema" : "`FOO` STRING", + "pseudoColumnVersion" : 1 + }, + "keyColumnNames" : [ ], + "selectedKeys" : null, + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON_SR", + "properties" : { } + } + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0", + "runtimeId" : null + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "metric.reporters" : "", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.connect.basic.auth.credentials.reload" : "false", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.stream.enabled" : "true", + "ksql.query.push.v2.interpreter.enabled" : "true", + "ksql.queryanonymizer.logs_enabled" : "true", + "ksql.variable.substitution.enable" : "true", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.query.push.v2.alos.enabled" : "true", + "ksql.query.push.v2.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.query.pull.range.scan.enabled" : "true", + "ksql.transient.query.cleanup.service.initial.delay.seconds" : "600", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.lambdas.enabled" : "true", + "ksql.source.table.materialization.enabled" : "true", + "ksql.query.pull.max.hourly.bandwidth.megabytes" : "2147483647", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.websocket.connection.max.timeout.ms" : "3600000", + "ksql.persistence.wrap.single.values" : null, + "ksql.query.transient.max.bytes.buffering.total" : "-1", + "ksql.connect.basic.auth.credentials.source" : "NONE", + "ksql.schema.registry.url" : "schema_registry.url:0", + "ksql.properties.overrides.denylist" : "", + "ksql.service.id" : "some.ksql.service.id", + "ksql.query.push.v2.max.catchup.consumers" : "5", + "ksql.assert.topic.default.timeout.ms" : "1000", + "ksql.query.push.v2.enabled" : "false", + "ksql.transient.query.cleanup.service.enable" : "true", + "ksql.query.push.v2.metrics.enabled" : "true", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.query.pull.table.scan.enabled" : "true", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.endpoint.migrate.query" : "true", + "ksql.query.push.v2.registry.installed" : "false", + "ksql.streams.num.stream.threads" : "4", + "ksql.metrics.tags.custom" : "", + "ksql.query.push.v2.catchup.consumer.msg.window" : "50", + "ksql.runtime.feature.shared.enabled" : "false", + "ksql.udf.collect.metrics" : "false", + "ksql.new.query.planner.enabled" : "false", + "ksql.connect.request.headers.plugin" : null, + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.headers.columns.enabled" : "true", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.connect.request.timeout.ms" : "5000", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.query.persistent.max.bytes.buffering.total" : "-1", + "ksql.query.error.max.queue.size" : "10", + "ksql.query.cleanup.shutdown.timeout.ms" : "30000", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.queryanonymizer.cluster_namespace" : null, + "ksql.create.or.replace.enabled" : "true", + "ksql.shared.runtimes.count" : "2", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.transient.query.cleanup.service.period.seconds" : "600", + "ksql.suppress.enabled" : "true", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.connect.basic.auth.credentials.file" : "", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.query.pull.max.concurrent.requests" : "2147483647", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.query.push.v2.new.latest.delay.ms" : "5000", + "ksql.query.push.v2.latest.reset.age.ms" : "30000", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.query.pull.interpreter.enabled" : "true", + "ksql.json_sr.converter.deserializer.enabled" : "true", + "ksql.assert.schema.default.timeout.ms" : "1000", + "ksql.query.pull.limit.clause.enabled" : "true", + "ksql.query.pull.router.thread.pool.size" : "50", + "ksql.query.push.v2.continuation.tokens.enabled" : "false", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.nested.error.set.null" : "true", + "ksql.query.pull.thread.pool.size" : "50", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/json_sr_-_struct_must_be_converted_to_String/7.4.0_1667521016436/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/json_sr_-_struct_must_be_converted_to_String/7.4.0_1667521016436/spec.json new file mode 100644 index 000000000000..c857a39164b3 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/json_sr_-_struct_must_be_converted_to_String/7.4.0_1667521016436/spec.json @@ -0,0 +1,190 @@ +{ + "version" : "7.4.0", + "timestamp" : 1667521016436, + "path" : "query-validation-tests/json_sr.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`FOO` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON_SR" + } + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`FOO` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON_SR" + } + } + }, + "testCase" : { + "name" : "struct must be converted to String", + "inputs" : [ { + "topic" : "test_topic", + "key" : null, + "value" : { + "FOO" : { + "F0" : 1, + "F1" : "bar" + } + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : null, + "value" : { + "FOO" : "{F0=1,F1=bar}" + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "valueSchema" : { + "type" : "object", + "properties" : { + "FOO" : { + "connect.index" : 0, + "oneOf" : [ { + "type" : "null" + }, { + "type" : "object", + "properties" : { + "F0" : { + "connect.index" : 0, + "oneOf" : [ { + "type" : "null" + }, { + "type" : "integer", + "connect.type" : "int32" + } ] + }, + "F1" : { + "connect.index" : 1, + "oneOf" : [ { + "type" : "null" + }, { + "type" : "string" + } ] + } + } + } ] + } + } + }, + "valueFormat" : "JSON", + "replicas" : 1, + "numPartitions" : 1 + } ], + "statements" : [ "CREATE STREAM TEST (FOO STRUCT ) WITH (kafka_topic='test_topic', value_format='JSON_SR', partitions=1);", "CREATE STREAM INPUT (FOO STRING) WITH (kafka_topic='test_topic', value_format='JSON_SR');", "CREATE STREAM OUTPUT as select * from INPUT;" ], + "post" : { + "sources" : [ { + "name" : "INPUT", + "type" : "STREAM", + "schema" : "`FOO` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "JSON_SR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`FOO` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "JSON_SR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + }, { + "name" : "TEST", + "type" : "STREAM", + "schema" : "`FOO` STRUCT<`F0` INTEGER, `F1` STRING>", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : "JSON_SR", + "keyFeatures" : [ ], + "valueFeatures" : [ ], + "isSource" : false + } ], + "topics" : { + "topics" : [ { + "name" : "test_topic", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON_SR" + }, + "partitions" : 1, + "valueSchema" : { + "type" : "object", + "properties" : { + "FOO" : { + "connect.index" : 0, + "oneOf" : [ { + "type" : "null" + }, { + "type" : "object", + "properties" : { + "F0" : { + "connect.index" : 0, + "oneOf" : [ { + "type" : "null" + }, { + "type" : "integer", + "connect.type" : "int32" + } ] + }, + "F1" : { + "connect.index" : 1, + "oneOf" : [ { + "type" : "null" + }, { + "type" : "string" + } ] + } + } + } ] + } + } + } + }, { + "name" : "OUTPUT", + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON_SR" + }, + "partitions" : 4, + "valueSchema" : { + "type" : "object", + "properties" : { + "FOO" : { + "connect.index" : 0, + "oneOf" : [ { + "type" : "null" + }, { + "type" : "string" + } ] + } + } + } + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/json_sr_-_struct_must_be_converted_to_String/7.4.0_1667521016436/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/json_sr_-_struct_must_be_converted_to_String/7.4.0_1667521016436/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/json_sr_-_struct_must_be_converted_to_String/7.4.0_1667521016436/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/json_sr.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/json_sr.json index 9754b9c5b72a..6bd2a1498b2e 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/json_sr.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/json_sr.json @@ -869,15 +869,15 @@ { "name": "struct must be converted to String", "statements": [ - "CREATE STREAM TEST (FOO STRUCT ) WITH (kafka_topic='test_topic', value_format='JSON_SR', partitions=1);", - "CREATE STREAM INPUT (FOO VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON_SR');", + "CREATE STREAM TEST (FOO STRUCT ) WITH (kafka_topic='test_topic', value_format='JSON_SR', partitions=1);", + "CREATE STREAM INPUT (FOO STRING) WITH (kafka_topic='test_topic', value_format='JSON_SR');", "CREATE STREAM OUTPUT as select * from INPUT;" ], "inputs": [ - {"topic": "test_topic", "value": {"FOO": {"F0": 1}}} + {"topic": "test_topic", "value": {"FOO": {"F0": 1, "F1": "bar"}}} ], "outputs": [ - {"topic": "OUTPUT", "value": {"FOO": "Struct{F0=1}"}} + {"topic": "OUTPUT", "value": {"FOO": "{F0=1,F1=bar}"}} ] } ] diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/connect/ConnectDataTranslator.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/connect/ConnectDataTranslator.java index b4b42fe76371..f2e108300e59 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/connect/ConnectDataTranslator.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/connect/ConnectDataTranslator.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import org.apache.kafka.common.protocol.types.Field.Str; import org.apache.kafka.connect.data.Date; import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.Schema; @@ -233,6 +234,9 @@ private Object toKsqlValue( return toKsqlStruct(schema, connectSchema, (Struct) convertedValue, pathStr); case STRING: // use String.valueOf to convert various int types, Struct and Boolean to string + if (convertedValue instanceof Struct) { + return structToString((Struct) convertedValue); + } return String.valueOf(convertedValue); case BYTES: if (convertedValue instanceof byte[]) { @@ -245,6 +249,27 @@ private Object toKsqlValue( } } + private String structToString(Struct input) { + StringBuilder sb = new StringBuilder("{"); + boolean first = true; + List fields = input.schema().fields(); + + for (Field field : fields) { + Object value = input.get(field); + if (value != null) { + if (first) { + first = false; + } else { + sb.append(","); + } + + sb.append(field.name()).append("=").append(value); + } + } + + return sb.append("}").toString(); + } + private List toKsqlArray( final Schema valueSchema, final Schema connectValueSchema, diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/avro/KsqlAvroDeserializerTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/avro/KsqlAvroDeserializerTest.java index 0c7f8a29fb7b..78a8201272e3 100644 --- a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/avro/KsqlAvroDeserializerTest.java +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/avro/KsqlAvroDeserializerTest.java @@ -1788,7 +1788,7 @@ private static List buildEntriesForMapWithOptionalKey( } private static String toStructString (Map map) { - StringBuilder sb = new StringBuilder("Struct{"); + StringBuilder sb = new StringBuilder("{"); boolean first = true; for (Map.Entry entry : map.entrySet()) {