Skip to content
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

feat: expose AVRO and JSON_SR as key formats #6694

Merged
merged 6 commits into from
Dec 2, 2020
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@
import static io.confluent.ksql.api.client.util.ClientTestUtil.subscribeAndWait;
import static io.confluent.ksql.test.util.AssertEventually.assertThatEventually;
import static io.confluent.ksql.util.KsqlConfig.KSQL_DEFAULT_KEY_FORMAT_CONFIG;
import static io.confluent.ksql.util.KsqlConfig.KSQL_KEY_FORMAT_ENABLED;
import static io.confluent.ksql.util.KsqlConfig.KSQL_STREAMS_PREFIX;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.contains;
Expand Down Expand Up @@ -179,7 +178,6 @@ public class ClientIntegrationTest {
private static final TestKsqlRestApp REST_APP = TestKsqlRestApp
.builder(TEST_HARNESS::kafkaBootstrapServers)
.withProperty(KSQL_STREAMS_PREFIX + StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1)
.withProperty(KSQL_KEY_FORMAT_ENABLED, true)
.withProperty(KSQL_DEFAULT_KEY_FORMAT_CONFIG, "JSON")
.build();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1470,7 +1470,7 @@ public void shouldThrowOnUnsupportedKeyFormatForCreateSource() {
// Given:
givenTopicsExist("foo");
final PreparedStatement<?> prepared =
prepare(parse("CREATE STREAM FOO (a int) WITH (kafka_topic='foo', value_format='json', key_format='avro');").get(0));
prepare(parse("CREATE STREAM FOO (a int) WITH (kafka_topic='foo', value_format='json', key_format='protobuf');").get(0));

// When:
final Exception e = assertThrows(
Expand All @@ -1482,7 +1482,7 @@ public void shouldThrowOnUnsupportedKeyFormatForCreateSource() {
);

// Then:
assertThat(e.getMessage(), containsString("The key format 'AVRO' is not currently supported."));
assertThat(e.getMessage(), containsString("The key format 'PROTOBUF' is not currently supported."));
}

@Test
Expand All @@ -1493,14 +1493,14 @@ public void shouldThrowOnUnsupportedKeyFormatForCSAS() {
() -> KsqlEngineTestUtil.execute(
serviceContext,
ksqlEngine,
"CREATE STREAM FOO WITH (KEY_FORMAT='AVRO') AS SELECT * FROM ORDERS;",
"CREATE STREAM FOO WITH (KEY_FORMAT='PROTOBUF') AS SELECT * FROM ORDERS;",
KSQL_CONFIG,
Collections.emptyMap()
)
);

// Then:
assertThat(e, rawMessage(containsString("The key format 'AVRO' is not currently supported.")));
assertThat(e, rawMessage(containsString("The key format 'PROTOBUF' is not currently supported.")));
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -124,10 +124,6 @@ public class EndToEndIntegrationTest {
.withAdditionalConfig(
KsqlConfig.SCHEMA_REGISTRY_URL_PROPERTY,
"http://foo:8080")
.withAdditionalConfig(
KsqlConfig.KSQL_KEY_FORMAT_ENABLED,
true
)
.build();

@Rule
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,147 @@
{
"plan" : [ {
"@type" : "ksqlPlanV1",
"statementText" : "CREATE STREAM INPUT (ROWKEY BIGINT KEY, FOO INTEGER) WITH (FORMAT='AVRO', KAFKA_TOPIC='input_topic', KEY_SCHEMA_ID=1);",
"ddlCommand" : {
"@type" : "createStreamV1",
"sourceName" : "INPUT",
"schema" : "`ROWKEY` BIGINT KEY, `FOO` INTEGER",
"topicName" : "input_topic",
"formats" : {
"keyFormat" : {
"format" : "AVRO"
},
"valueFormat" : {
"format" : "AVRO"
},
"keyFeatures" : [ "UNWRAP_SINGLES" ]
},
"orReplace" : false
}
}, {
"@type" : "ksqlPlanV1",
"statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES",
"ddlCommand" : {
"@type" : "createStreamV1",
"sourceName" : "OUTPUT",
"schema" : "`ROWKEY` BIGINT KEY, `FOO` INTEGER",
"topicName" : "OUTPUT",
"formats" : {
"keyFormat" : {
"format" : "AVRO"
},
"valueFormat" : {
"format" : "AVRO"
},
"keyFeatures" : [ "UNWRAP_SINGLES" ]
},
"orReplace" : 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" : "input_topic",
"formats" : {
"keyFormat" : {
"format" : "AVRO"
},
"valueFormat" : {
"format" : "AVRO"
},
"keyFeatures" : [ "UNWRAP_SINGLES" ]
},
"sourceSchema" : "`ROWKEY` BIGINT KEY, `FOO` INTEGER"
},
"keyColumnNames" : [ "ROWKEY" ],
"selectExpressions" : [ "FOO AS FOO" ]
},
"formats" : {
"keyFormat" : {
"format" : "AVRO"
},
"valueFormat" : {
"format" : "AVRO"
},
"keyFeatures" : [ "UNWRAP_SINGLES" ]
},
"topicName" : "OUTPUT"
},
"queryId" : "CSAS_OUTPUT_0"
}
} ],
"configs" : {
"ksql.extension.dir" : "ext",
"ksql.streams.cache.max.bytes.buffering" : "0",
"ksql.security.extension.class" : null,
"metric.reporters" : "",
"ksql.transient.prefix" : "transient_",
"ksql.query.status.running.threshold.seconds" : "300",
"ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler",
"ksql.output.topic.name.prefix" : "",
"ksql.query.pull.enable.standby.reads" : "false",
"ksql.persistence.default.format.key" : "KAFKA",
"ksql.query.error.max.queue.size" : "10",
"ksql.variable.substitution.enable" : "true",
"ksql.internal.topic.min.insync.replicas" : "1",
"ksql.streams.shutdown.timeout.ms" : "300000",
"ksql.internal.topic.replicas" : "1",
"ksql.insert.into.values.enabled" : "true",
"ksql.key.format.enabled" : "false",
"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" : "false",
"ksql.create.or.replace.enabled" : "true",
"ksql.metrics.extension" : null,
"ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
"ksql.cast.strings.preserve.nulls" : "true",
"ksql.authorization.cache.max.entries" : "10000",
"ksql.pull.queries.enable" : "true",
"ksql.suppress.enabled" : "false",
"ksql.sink.window.change.log.additional.retention" : "1000000",
"ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses",
"ksql.query.persistent.active.limit" : "2147483647",
"ksql.persistence.wrap.single.values" : null,
"ksql.authorization.cache.expiry.time.secs" : "30",
"ksql.query.retry.backoff.initial.ms" : "15000",
"ksql.schema.registry.url" : "",
"ksql.properties.overrides.denylist" : "",
"ksql.streams.auto.offset.reset" : "earliest",
"ksql.connect.url" : "http://localhost:8083",
"ksql.service.id" : "some.ksql.service.id",
"ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler",
"ksql.streams.commit.interval.ms" : "2000",
"ksql.streams.auto.commit.interval.ms" : "0",
"ksql.streams.topology.optimization" : "all",
"ksql.query.retry.backoff.max.ms" : "900000",
"ksql.streams.num.stream.threads" : "4",
"ksql.timestamp.throw.on.invalid" : "false",
"ksql.metrics.tags.custom" : "",
"ksql.persistence.default.format.value" : null,
"ksql.udfs.enabled" : "true",
"ksql.udf.enable.security.manager" : "true",
"ksql.connect.worker.config" : "",
"ksql.udf.collect.metrics" : "false",
"ksql.query.pull.thread.pool.size" : "100",
"ksql.persistent.prefix" : "query_",
"ksql.metastore.backup.location" : "",
"ksql.error.classifier.regex" : "",
"ksql.suppress.buffer.size.bytes" : "-1"
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,149 @@
{
"version" : "6.2.0",
"timestamp" : 1606796938952,
"path" : "query-validation-tests/avro.json",
"schemas" : {
"CSAS_OUTPUT_0.KsqlTopic.Source" : {
"schema" : "`ROWKEY` BIGINT KEY, `FOO` INTEGER",
"keyFormat" : {
"format" : "AVRO",
"features" : [ "UNWRAP_SINGLES" ]
},
"valueFormat" : {
"format" : "AVRO"
}
},
"CSAS_OUTPUT_0.OUTPUT" : {
"schema" : "`ROWKEY` BIGINT KEY, `FOO` INTEGER",
"keyFormat" : {
"format" : "AVRO",
"features" : [ "UNWRAP_SINGLES" ]
},
"valueFormat" : {
"format" : "AVRO"
}
}
},
"testCase" : {
"name" : "BIGINT - key - inference",
"inputs" : [ {
"topic" : "input_topic",
"key" : 998877665544332211,
"value" : {
"FOO" : 10
}
}, {
"topic" : "input_topic",
"key" : null,
"value" : {
"FOO" : 10
}
} ],
"outputs" : [ {
"topic" : "OUTPUT",
"key" : 998877665544332211,
"value" : {
"FOO" : 10
}
}, {
"topic" : "OUTPUT",
"key" : null,
"value" : {
"FOO" : 10
}
} ],
"topics" : [ {
"name" : "OUTPUT",
"replicas" : 1,
"numPartitions" : 4
}, {
"name" : "input_topic",
"keySchema" : "long",
"valueSchema" : {
"type" : "record",
"name" : "KsqlDataSourceSchema",
"namespace" : "io.confluent.ksql.avro_schemas",
"fields" : [ {
"name" : "FOO",
"type" : [ "null", "int" ],
"default" : null
} ],
"connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema"
},
"keyFormat" : "AVRO",
"valueFormat" : "AVRO",
"replicas" : 1,
"numPartitions" : 1
} ],
"statements" : [ "CREATE STREAM INPUT (foo INT) WITH (kafka_topic='input_topic', format='AVRO');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ],
"post" : {
"sources" : [ {
"name" : "INPUT",
"type" : "STREAM",
"schema" : "`ROWKEY` BIGINT KEY, `FOO` INTEGER",
"keyFormat" : {
"format" : "AVRO"
},
"valueFormat" : "AVRO",
"keyFeatures" : [ "UNWRAP_SINGLES" ],
"valueFeatures" : [ ]
}, {
"name" : "OUTPUT",
"type" : "STREAM",
"schema" : "`ROWKEY` BIGINT KEY, `FOO` INTEGER",
"keyFormat" : {
"format" : "AVRO"
},
"valueFormat" : "AVRO",
"keyFeatures" : [ "UNWRAP_SINGLES" ],
"valueFeatures" : [ ]
} ],
"topics" : {
"topics" : [ {
"name" : "input_topic",
"keyFormat" : {
"format" : "AVRO",
"features" : [ "UNWRAP_SINGLES" ]
},
"valueFormat" : {
"format" : "AVRO"
},
"partitions" : 1,
"keySchema" : "long",
"valueSchema" : {
"type" : "record",
"name" : "KsqlDataSourceSchema",
"namespace" : "io.confluent.ksql.avro_schemas",
"fields" : [ {
"name" : "FOO",
"type" : [ "null", "int" ],
"default" : null
} ],
"connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema"
}
}, {
"name" : "OUTPUT",
"keyFormat" : {
"format" : "AVRO",
"features" : [ "UNWRAP_SINGLES" ]
},
"valueFormat" : {
"format" : "AVRO"
},
"partitions" : 4,
"keySchema" : "long",
"valueSchema" : {
"type" : "record",
"name" : "KsqlDataSourceSchema",
"namespace" : "io.confluent.ksql.avro_schemas",
"fields" : [ {
"name" : "FOO",
"type" : [ "null", "int" ],
"default" : null
} ]
}
} ]
}
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,13 @@
Topologies:
Sub-topology: 0
Source: KSTREAM-SOURCE-0000000000 (topics: [input_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

Loading