diff --git a/docs/developer-guide/ksqldb-reference/scalar-functions.md b/docs/developer-guide/ksqldb-reference/scalar-functions.md index dbdbb39cabfe..7d594888960a 100644 --- a/docs/developer-guide/ksqldb-reference/scalar-functions.md +++ b/docs/developer-guide/ksqldb-reference/scalar-functions.md @@ -382,6 +382,26 @@ include both endpoints. ## Strings +### `CHR` + +```sql +CHR(decimal_code | utf_string) +``` + +Returns a single-character string representing the Unicode code-point described by the input. The input parameter can be either a decimal character code or a string representation of a UTF code. + +Returns NULL if the input is NULL or does not represent a valid code-point. + +Commonly used to insert control characters such as `Tab` (9), `Line Feed` (10), or `Carriage Return` (13) into strings. + +Examples: +```sql +CHR(75) => 'K' +CHR('\u004b') => 'K' +CHR(22909) => '好' +CHR('\u597d') => '好' +``` + ### `CONCAT` ```sql diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/string/Chr.java b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/string/Chr.java new file mode 100644 index 000000000000..73f68a8c7309 --- /dev/null +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/function/udf/string/Chr.java @@ -0,0 +1,48 @@ +/* + * Copyright 2020 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"; you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and limitations under the + * License. + */ + +package io.confluent.ksql.function.udf.string; + +import io.confluent.ksql.function.udf.Udf; +import io.confluent.ksql.function.udf.UdfDescription; +import io.confluent.ksql.function.udf.UdfParameter; +import org.apache.commons.lang3.StringEscapeUtils; + +@UdfDescription( + name = "Chr", + description = "Returns a single-character string corresponding to the input character code.") +public class Chr { + + @Udf + public String chr(@UdfParameter( + description = "Decimal codepoint") final Integer decimalCode) { + if (decimalCode == null) { + return null; + } + if (!Character.isValidCodePoint(decimalCode)) { + return null; + } + final char[] resultChars = Character.toChars(decimalCode.intValue()); + return String.valueOf(resultChars); + } + + @Udf + public String chr(@UdfParameter( + description = "UTF16 code for the desired character e.g. '\\u004b'") final String utf16Code) { + if (utf16Code == null || utf16Code.length() < 6 || !utf16Code.startsWith("\\u")) { + return null; + } + return StringEscapeUtils.unescapeJava(utf16Code); + } +} diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/string/ChrTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/string/ChrTest.java new file mode 100644 index 000000000000..defe9ca0648f --- /dev/null +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/function/udf/string/ChrTest.java @@ -0,0 +1,126 @@ +/* + * Copyright 2020 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"; you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and limitations under the + * License. + */ + +package io.confluent.ksql.function.udf.string; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; + +import org.junit.Test; + +public class ChrTest { + private final Chr udf = new Chr(); + + @Test + public void shouldConvertFromDecimal() { + final String result = udf.chr(75); + assertThat(result, is("K")); + } + + @Test + public void shouldConvertFromUTF16String() { + final String result = udf.chr("\\u004b"); + assertThat(result, is("K")); + } + + @Test + public void shouldConvertFromUTF16StringWithSlash() { + final String result = udf.chr("\\u004b"); + assertThat(result, is("K")); + } + + @Test + public void shouldConvertZhFromDecimal() { + final String result = udf.chr(22909); + assertThat(result, is("好")); + } + + @Test + public void shouldConvertZhFromUTF16() { + final String result = udf.chr("\\u597d"); + assertThat(result, is("好")); + } + + @Test + public void shouldConvertControlChar() { + final String result = udf.chr(9); + assertThat(result, is("\t")); + } + + @Test + public void shouldReturnNullForNullIntegerInput() { + final String result = udf.chr((Integer) null); + assertThat(result, is(nullValue())); + } + + @Test + public void shouldReturnNullForNullStringInput() { + final String result = udf.chr((String) null); + assertThat(result, is(nullValue())); + } + + @Test + public void shouldReturnNullForEmptyStringInput() { + final String result = udf.chr(""); + assertThat(result, is(nullValue())); + } + + @Test + public void shouldReturnNullForNegativeDecimalCode() { + final String result = udf.chr(-1); + assertThat(result, is(nullValue())); + } + + @Test + public void shouldReturnSingleCharForMaxBMPDecimal() { + final String result = udf.chr(65535); + assertThat(result.codePointAt(0), is(65535)); + assertThat(result.toCharArray().length, is(1)); + } + + @Test + public void shouldReturnTwoCharsForNonBMPDecimal() { + final String result = udf.chr(65536); + assertThat(result.codePointAt(0), is(65536)); + assertThat(result.toCharArray().length, is(2)); + } + + @Test + public void shouldReturnTwoCharsForMaxUnicodeDecimal() { + final String result = udf.chr(1_114_111); + assertThat(result.codePointAt(0), is(1_114_111)); + assertThat(result.toCharArray().length, is(2)); + } + + @Test + public void shouldReturnNullForOutOfRangeDecimal() { + final String result = udf.chr(1_114_112); + assertThat(result, is(nullValue())); + } + + @Test + public void shouldReturnNullForTooShortUTF16String() { + final String result = udf.chr("\\u065"); + assertThat(result, is(nullValue())); + } + + @Test + public void shouldReturnTwoCharsForNonBMPString() { + final String result = udf.chr("\\ud800\\udc01"); + assertThat(result.codePointAt(0), is(65537)); + assertThat(result.toCharArray().length, is(2)); + } + +} diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_AVRO/6.0.0_1591421496995/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_AVRO/6.0.0_1591421496995/plan.json new file mode 100644 index 000000000000..33111d1c51ed --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_AVRO/6.0.0_1591421496995/plan.json @@ -0,0 +1,126 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID STRING KEY, UTFCODE INTEGER) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` STRING KEY, `UTFCODE` INTEGER", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n INPUT.ID ID,\n CHR(INPUT.UTFCODE) RESULT\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` STRING KEY, `RESULT` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`ID` STRING KEY, `UTFCODE` INTEGER" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "CHR(UTFCODE) AS RESULT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "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.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "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.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_AVRO/6.0.0_1591421496995/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_AVRO/6.0.0_1591421496995/spec.json new file mode 100644 index 000000000000..e55dbca72163 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_AVRO/6.0.0_1591421496995/spec.json @@ -0,0 +1,123 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591421496995, + "path" : "query-validation-tests\\chr.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "codepoint from decimal code - AVRO", + "inputs" : [ { + "topic" : "test_topic", + "key" : "r1", + "value" : { + "utfcode" : 75 + } + }, { + "topic" : "test_topic", + "key" : "r2", + "value" : { + "utfcode" : 22909 + } + }, { + "topic" : "test_topic", + "key" : "r3", + "value" : { + "utfcode" : 99000 + } + }, { + "topic" : "test_topic", + "key" : "r4", + "value" : { + "utfcode" : -1 + } + }, { + "topic" : "test_topic", + "key" : "r5", + "value" : { + "utfcode" : null + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "r1", + "value" : { + "RESULT" : "K" + } + }, { + "topic" : "OUTPUT", + "key" : "r2", + "value" : { + "RESULT" : "好" + } + }, { + "topic" : "OUTPUT", + "key" : "r3", + "value" : { + "RESULT" : "𘊸" + } + }, { + "topic" : "OUTPUT", + "key" : "r4", + "value" : { + "RESULT" : null + } + }, { + "topic" : "OUTPUT", + "key" : "r5", + "value" : { + "RESULT" : null + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "schema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "UTFCODE", + "type" : [ "null", "int" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (id STRING KEY, utfcode INTEGER) WITH (kafka_topic='test_topic', value_format='AVRO');", "CREATE STREAM OUTPUT AS SELECT id, chr(utfcode) AS result FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_AVRO/6.0.0_1591421496995/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_AVRO/6.0.0_1591421496995/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_AVRO/6.0.0_1591421496995/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/historical_plans/chr_-_codepoint_from_decimal_code_-_JSON/6.0.0_1591421497086/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_JSON/6.0.0_1591421497086/plan.json new file mode 100644 index 000000000000..a34c2181cb9f --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_JSON/6.0.0_1591421497086/plan.json @@ -0,0 +1,126 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID STRING KEY, UTFCODE INTEGER) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` STRING KEY, `UTFCODE` INTEGER", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n INPUT.ID ID,\n CHR(INPUT.UTFCODE) RESULT\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` STRING KEY, `RESULT` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` STRING KEY, `UTFCODE` INTEGER" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "CHR(UTFCODE) AS RESULT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "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.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "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.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_JSON/6.0.0_1591421497086/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_JSON/6.0.0_1591421497086/spec.json new file mode 100644 index 000000000000..7cc21dbd3af8 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_JSON/6.0.0_1591421497086/spec.json @@ -0,0 +1,111 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591421497086, + "path" : "query-validation-tests\\chr.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "codepoint from decimal code - JSON", + "inputs" : [ { + "topic" : "test_topic", + "key" : "r1", + "value" : { + "utfcode" : 75 + } + }, { + "topic" : "test_topic", + "key" : "r2", + "value" : { + "utfcode" : 22909 + } + }, { + "topic" : "test_topic", + "key" : "r3", + "value" : { + "utfcode" : 99000 + } + }, { + "topic" : "test_topic", + "key" : "r4", + "value" : { + "utfcode" : -1 + } + }, { + "topic" : "test_topic", + "key" : "r5", + "value" : { + "utfcode" : null + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "r1", + "value" : { + "RESULT" : "K" + } + }, { + "topic" : "OUTPUT", + "key" : "r2", + "value" : { + "RESULT" : "好" + } + }, { + "topic" : "OUTPUT", + "key" : "r3", + "value" : { + "RESULT" : "𘊸" + } + }, { + "topic" : "OUTPUT", + "key" : "r4", + "value" : { + "RESULT" : null + } + }, { + "topic" : "OUTPUT", + "key" : "r5", + "value" : { + "RESULT" : null + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (id STRING KEY, utfcode INTEGER) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT id, chr(utfcode) AS result FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_JSON/6.0.0_1591421497086/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_JSON/6.0.0_1591421497086/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_decimal_code_-_JSON/6.0.0_1591421497086/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/historical_plans/chr_-_codepoint_from_text_code_-_AVRO/6.0.0_1591421497142/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_AVRO/6.0.0_1591421497142/plan.json new file mode 100644 index 000000000000..ac2fb854c475 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_AVRO/6.0.0_1591421497142/plan.json @@ -0,0 +1,126 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID STRING KEY, UTFCODE STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` STRING KEY, `UTFCODE` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n INPUT.ID ID,\n CHR(INPUT.UTFCODE) RESULT\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` STRING KEY, `RESULT` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`ID` STRING KEY, `UTFCODE` STRING" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "CHR(UTFCODE) AS RESULT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "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.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "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.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_AVRO/6.0.0_1591421497142/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_AVRO/6.0.0_1591421497142/spec.json new file mode 100644 index 000000000000..27ed7a1ea55a --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_AVRO/6.0.0_1591421497142/spec.json @@ -0,0 +1,159 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591421497142, + "path" : "query-validation-tests\\chr.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "codepoint from text code - AVRO", + "inputs" : [ { + "topic" : "test_topic", + "key" : "r1", + "value" : { + "utfcode" : "\\u004b" + } + }, { + "topic" : "test_topic", + "key" : "r2", + "value" : { + "utfcode" : "\\u597d" + } + }, { + "topic" : "test_topic", + "key" : "r3", + "value" : { + "utfcode" : "\\ud820\\udeb8" + } + }, { + "topic" : "test_topic", + "key" : "r4", + "value" : { + "utfcode" : "75" + } + }, { + "topic" : "test_topic", + "key" : "r5", + "value" : { + "utfcode" : "004b" + } + }, { + "topic" : "test_topic", + "key" : "r6", + "value" : { + "utfcode" : "bogus" + } + }, { + "topic" : "test_topic", + "key" : "r7", + "value" : { + "utfcode" : "" + } + }, { + "topic" : "test_topic", + "key" : "r8", + "value" : { + "utfcode" : null + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "r1", + "value" : { + "RESULT" : "K" + } + }, { + "topic" : "OUTPUT", + "key" : "r2", + "value" : { + "RESULT" : "好" + } + }, { + "topic" : "OUTPUT", + "key" : "r3", + "value" : { + "RESULT" : "𘊸" + } + }, { + "topic" : "OUTPUT", + "key" : "r4", + "value" : { + "RESULT" : null + } + }, { + "topic" : "OUTPUT", + "key" : "r5", + "value" : { + "RESULT" : null + } + }, { + "topic" : "OUTPUT", + "key" : "r6", + "value" : { + "RESULT" : null + } + }, { + "topic" : "OUTPUT", + "key" : "r7", + "value" : { + "RESULT" : null + } + }, { + "topic" : "OUTPUT", + "key" : "r8", + "value" : { + "RESULT" : null + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "schema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "UTFCODE", + "type" : [ "null", "string" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (id STRING KEY, utfcode STRING) WITH (kafka_topic='test_topic', value_format='AVRO');", "CREATE STREAM OUTPUT AS SELECT id, chr(utfcode) AS result FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_AVRO/6.0.0_1591421497142/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_AVRO/6.0.0_1591421497142/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_AVRO/6.0.0_1591421497142/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/historical_plans/chr_-_codepoint_from_text_code_-_JSON/6.0.0_1591421497199/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_JSON/6.0.0_1591421497199/plan.json new file mode 100644 index 000000000000..0218c9ec6ef8 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_JSON/6.0.0_1591421497199/plan.json @@ -0,0 +1,126 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID STRING KEY, UTFCODE STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` STRING KEY, `UTFCODE` STRING", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n INPUT.ID ID,\n CHR(INPUT.UTFCODE) RESULT\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` STRING KEY, `RESULT` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` STRING KEY, `UTFCODE` STRING" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "CHR(UTFCODE) AS RESULT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "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.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "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.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_JSON/6.0.0_1591421497199/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_JSON/6.0.0_1591421497199/spec.json new file mode 100644 index 000000000000..64d573b7c152 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_JSON/6.0.0_1591421497199/spec.json @@ -0,0 +1,147 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591421497199, + "path" : "query-validation-tests\\chr.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "codepoint from text code - JSON", + "inputs" : [ { + "topic" : "test_topic", + "key" : "r1", + "value" : { + "utfcode" : "\\u004b" + } + }, { + "topic" : "test_topic", + "key" : "r2", + "value" : { + "utfcode" : "\\u597d" + } + }, { + "topic" : "test_topic", + "key" : "r3", + "value" : { + "utfcode" : "\\ud820\\udeb8" + } + }, { + "topic" : "test_topic", + "key" : "r4", + "value" : { + "utfcode" : "75" + } + }, { + "topic" : "test_topic", + "key" : "r5", + "value" : { + "utfcode" : "004b" + } + }, { + "topic" : "test_topic", + "key" : "r6", + "value" : { + "utfcode" : "bogus" + } + }, { + "topic" : "test_topic", + "key" : "r7", + "value" : { + "utfcode" : "" + } + }, { + "topic" : "test_topic", + "key" : "r8", + "value" : { + "utfcode" : null + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "r1", + "value" : { + "RESULT" : "K" + } + }, { + "topic" : "OUTPUT", + "key" : "r2", + "value" : { + "RESULT" : "好" + } + }, { + "topic" : "OUTPUT", + "key" : "r3", + "value" : { + "RESULT" : "𘊸" + } + }, { + "topic" : "OUTPUT", + "key" : "r4", + "value" : { + "RESULT" : null + } + }, { + "topic" : "OUTPUT", + "key" : "r5", + "value" : { + "RESULT" : null + } + }, { + "topic" : "OUTPUT", + "key" : "r6", + "value" : { + "RESULT" : null + } + }, { + "topic" : "OUTPUT", + "key" : "r7", + "value" : { + "RESULT" : null + } + }, { + "topic" : "OUTPUT", + "key" : "r8", + "value" : { + "RESULT" : null + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (id STRING KEY, utfcode STRING) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT id, chr(utfcode) AS result FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_JSON/6.0.0_1591421497199/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_JSON/6.0.0_1591421497199/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_codepoint_from_text_code_-_JSON/6.0.0_1591421497199/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/historical_plans/chr_-_multiple_invocations_-_AVRO/6.0.0_1591421497279/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_AVRO/6.0.0_1591421497279/plan.json new file mode 100644 index 000000000000..7ed500c68343 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_AVRO/6.0.0_1591421497279/plan.json @@ -0,0 +1,126 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID STRING KEY, UTFCODE1 STRING, UTFCODE2 STRING, UTFCODE3 INTEGER, UTFCODE4 INTEGER) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` STRING KEY, `UTFCODE1` STRING, `UTFCODE2` STRING, `UTFCODE3` INTEGER, `UTFCODE4` INTEGER", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n INPUT.ID ID,\n CONCAT(CHR(INPUT.UTFCODE1), CHR(INPUT.UTFCODE2), CHR(INPUT.UTFCODE3), CHR(INPUT.UTFCODE4)) RESULT\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` STRING KEY, `RESULT` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "sourceSchema" : "`ID` STRING KEY, `UTFCODE1` STRING, `UTFCODE2` STRING, `UTFCODE3` INTEGER, `UTFCODE4` INTEGER" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "CONCAT(CHR(UTFCODE1), CHR(UTFCODE2), CHR(UTFCODE3), CHR(UTFCODE4)) AS RESULT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "AVRO" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "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.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "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.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_AVRO/6.0.0_1591421497279/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_AVRO/6.0.0_1591421497279/spec.json new file mode 100644 index 000000000000..34a34720d582 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_AVRO/6.0.0_1591421497279/spec.json @@ -0,0 +1,90 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591421497279, + "path" : "query-validation-tests\\chr.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "multiple invocations - AVRO", + "inputs" : [ { + "topic" : "test_topic", + "key" : "r1", + "value" : { + "utfcode1" : "\\u004b", + "utfcode2" : "\\u0053", + "utfcode3" : 81, + "utfcode4" : 76 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "r1", + "value" : { + "RESULT" : "KSQL" + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "schema" : { + "type" : "record", + "name" : "KsqlDataSourceSchema", + "namespace" : "io.confluent.ksql.avro_schemas", + "fields" : [ { + "name" : "UTFCODE1", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "UTFCODE2", + "type" : [ "null", "string" ], + "default" : null + }, { + "name" : "UTFCODE3", + "type" : [ "null", "int" ], + "default" : null + }, { + "name" : "UTFCODE4", + "type" : [ "null", "int" ], + "default" : null + } ], + "connect.name" : "io.confluent.ksql.avro_schemas.KsqlDataSourceSchema" + }, + "format" : "AVRO", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (id STRING KEY, utfcode1 STRING, utfcode2 STRING, utfcode3 INTEGER, utfcode4 INTEGER) WITH (kafka_topic='test_topic', value_format='AVRO');", "CREATE STREAM OUTPUT AS SELECT id, concat(chr(utfcode1), chr(utfcode2), chr(utfcode3), chr(utfcode4)) AS result FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "AVRO" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_AVRO/6.0.0_1591421497279/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_AVRO/6.0.0_1591421497279/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_AVRO/6.0.0_1591421497279/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/historical_plans/chr_-_multiple_invocations_-_JSON/6.0.0_1591421497385/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_JSON/6.0.0_1591421497385/plan.json new file mode 100644 index 000000000000..2c78f35e3004 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_JSON/6.0.0_1591421497385/plan.json @@ -0,0 +1,126 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID STRING KEY, UTFCODE1 STRING, UTFCODE2 STRING, UTFCODE3 INTEGER, UTFCODE4 INTEGER) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ID` STRING KEY, `UTFCODE1` STRING, `UTFCODE2` STRING, `UTFCODE3` INTEGER, `UTFCODE4` INTEGER", + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT\n INPUT.ID ID,\n CONCAT(CHR(INPUT.UTFCODE1), CHR(INPUT.UTFCODE2), CHR(INPUT.UTFCODE3), CHR(INPUT.UTFCODE4)) RESULT\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ID` STRING KEY, `RESULT` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + } + }, + "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" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`ID` STRING KEY, `UTFCODE1` STRING, `UTFCODE2` STRING, `UTFCODE3` INTEGER, `UTFCODE4` INTEGER" + }, + "keyColumnNames" : [ "ID" ], + "selectExpressions" : [ "CONCAT(CHR(UTFCODE1), CHR(UTFCODE2), CHR(UTFCODE3), CHR(UTFCODE4)) AS RESULT" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "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.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "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.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.error.classifier.regex" : "" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_JSON/6.0.0_1591421497385/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_JSON/6.0.0_1591421497385/spec.json new file mode 100644 index 000000000000..78c3f2b754db --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_JSON/6.0.0_1591421497385/spec.json @@ -0,0 +1,66 @@ +{ + "version" : "6.0.0", + "timestamp" : 1591421497385, + "path" : "query-validation-tests\\chr.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "testCase" : { + "name" : "multiple invocations - JSON", + "inputs" : [ { + "topic" : "test_topic", + "key" : "r1", + "value" : { + "utfcode1" : "\\u004b", + "utfcode2" : "\\u0053", + "utfcode3" : 81, + "utfcode4" : 76 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "r1", + "value" : { + "RESULT" : "KSQL" + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "test_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (id STRING KEY, utfcode1 STRING, utfcode2 STRING, utfcode3 INTEGER, utfcode4 INTEGER) WITH (kafka_topic='test_topic', value_format='JSON');", "CREATE STREAM OUTPUT AS SELECT id, concat(chr(utfcode1), chr(utfcode2), chr(utfcode3), chr(utfcode4)) AS result FROM INPUT;" ], + "post" : { + "topics" : { + "topics" : [ { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "test_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_JSON/6.0.0_1591421497385/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_JSON/6.0.0_1591421497385/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/chr_-_multiple_invocations_-_JSON/6.0.0_1591421497385/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/chr.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/chr.json new file mode 100644 index 000000000000..357cb174b2c3 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/chr.json @@ -0,0 +1,72 @@ +{ + "comments": [ + "Tests covering the use of the CHR function." + ], + "tests": [ + { + "name": "codepoint from decimal code", + "format": ["AVRO", "JSON"], + "statements": [ + "CREATE STREAM INPUT (id STRING KEY, utfcode INTEGER) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", + "CREATE STREAM OUTPUT AS SELECT id, chr(utfcode) AS result FROM INPUT;" + ], + "inputs": [ + {"topic": "test_topic", "key": "r1", "value": {"utfcode": 75}}, + {"topic": "test_topic", "key": "r2", "value": {"utfcode": 22909}}, + {"topic": "test_topic", "key": "r3", "value": {"utfcode": 99000}}, + {"topic": "test_topic", "key": "r4", "value": {"utfcode": -1}}, + {"topic": "test_topic", "key": "r5", "value": {"utfcode": null}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "r1", "value": {"RESULT": "K"}}, + {"topic": "OUTPUT", "key": "r2", "value": {"RESULT": "好"}}, + {"topic": "OUTPUT", "key": "r3", "value": {"RESULT": "𘊸"}}, + {"topic": "OUTPUT", "key": "r4", "value": {"RESULT": null}}, + {"topic": "OUTPUT", "key": "r5", "value": {"RESULT": null}} + ] + }, + { + "name": "codepoint from text code", + "format": ["AVRO", "JSON"], + "statements": [ + "CREATE STREAM INPUT (id STRING KEY, utfcode STRING) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", + "CREATE STREAM OUTPUT AS SELECT id, chr(utfcode) AS result FROM INPUT;" + ], + "inputs": [ + {"topic": "test_topic", "key": "r1", "value": {"utfcode": "\\u004b"}}, + {"topic": "test_topic", "key": "r2", "value": {"utfcode": "\\u597d"}}, + {"topic": "test_topic", "key": "r3", "value": {"utfcode": "\\ud820\\udeb8"}}, + {"topic": "test_topic", "key": "r4", "value": {"utfcode": "75"}}, + {"topic": "test_topic", "key": "r5", "value": {"utfcode": "004b"}}, + {"topic": "test_topic", "key": "r6", "value": {"utfcode": "bogus"}}, + {"topic": "test_topic", "key": "r7", "value": {"utfcode": ""}}, + {"topic": "test_topic", "key": "r8", "value": {"utfcode": null}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "r1", "value": {"RESULT": "K"}}, + {"topic": "OUTPUT", "key": "r2", "value": {"RESULT": "好"}}, + {"topic": "OUTPUT", "key": "r3", "value": {"RESULT": "𘊸"}}, + {"topic": "OUTPUT", "key": "r4", "value": {"RESULT": null}}, + {"topic": "OUTPUT", "key": "r5", "value": {"RESULT": null}}, + {"topic": "OUTPUT", "key": "r6", "value": {"RESULT": null}}, + {"topic": "OUTPUT", "key": "r7", "value": {"RESULT": null}}, + {"topic": "OUTPUT", "key": "r8", "value": {"RESULT": null}} + ] + }, + { + "name": "multiple invocations", + "format": ["AVRO", "JSON"], + "statements": [ + "CREATE STREAM INPUT (id STRING KEY, utfcode1 STRING, utfcode2 STRING, utfcode3 INTEGER, utfcode4 INTEGER) WITH (kafka_topic='test_topic', value_format='{FORMAT}');", + "CREATE STREAM OUTPUT AS SELECT id, concat(chr(utfcode1), chr(utfcode2), chr(utfcode3), chr(utfcode4)) AS result FROM INPUT;" + ], + "inputs": [ + {"topic": "test_topic", "key": "r1", "value": {"utfcode1": "\\u004b", "utfcode2": "\\u0053", "utfcode3": 81, "utfcode4": 76}} + ], + "outputs": [ + {"topic": "OUTPUT", "key": "r1", "value": {"RESULT": "KSQL"}} + ] + } + + ] +} \ No newline at end of file