From 7945d62c87f3b2c9496b686ba78ce6dc50667d53 Mon Sep 17 00:00:00 2001 From: Hasnat Ullah Date: Tue, 16 Oct 2018 00:50:19 +0100 Subject: [PATCH 01/46] Added optional for FORMAT_VALUE DELIMITED --- .../java/io/confluent/ksql/ddl/DdlConfig.java | 1 + .../ddl/commands/RegisterTopicCommand.java | 9 ++++-- .../delimited/KsqlDelimitedDeserializer.java | 9 +++++- .../delimited/KsqlDelimitedSerializer.java | 9 +++++- .../delimited/KsqlDelimitedTopicSerDe.java | 17 +++++++++-- .../KsqlDelimitedDeserializerTest.java | 30 +++++++++++++++++++ 6 files changed, 69 insertions(+), 6 deletions(-) diff --git a/ksql-common/src/main/java/io/confluent/ksql/ddl/DdlConfig.java b/ksql-common/src/main/java/io/confluent/ksql/ddl/DdlConfig.java index 44e07551fcbc..82f711d623e1 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/ddl/DdlConfig.java +++ b/ksql-common/src/main/java/io/confluent/ksql/ddl/DdlConfig.java @@ -17,6 +17,7 @@ public final class DdlConfig { public static final String VALUE_FORMAT_PROPERTY = "VALUE_FORMAT"; + public static final String DELIMITER_PROPERTY = "DELIMITER"; public static final String AVRO_SCHEMA_FILE = "AVROSCHEMAFILE"; public static final String AVRO_SCHEMA = "AVROSCHEMA"; public static final String KAFKA_TOPIC_NAME_PROPERTY = "KAFKA_TOPIC"; diff --git a/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/RegisterTopicCommand.java b/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/RegisterTopicCommand.java index 7e7ffb2be16c..236fe0afa33d 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/RegisterTopicCommand.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/RegisterTopicCommand.java @@ -51,11 +51,12 @@ public RegisterTopicCommand(final RegisterTopic registerTopic) { properties.get(DdlConfig.KAFKA_TOPIC_NAME_PROPERTY).toString()); final String serde = StringUtil.cleanQuotes( properties.get(DdlConfig.VALUE_FORMAT_PROPERTY).toString()); - this.topicSerDe = extractTopicSerDe(serde); + this.topicSerDe = extractTopicSerDe(serde, properties); this.notExists = notExist; } - private KsqlTopicSerDe extractTopicSerDe(final String serde) { + private KsqlTopicSerDe extractTopicSerDe(final String serde, + final Map properties) { // TODO: Find a way to avoid calling toUpperCase() here; // if the property can be an unquoted identifier, then capitalization will have already happened switch (serde.toUpperCase()) { @@ -64,6 +65,10 @@ private KsqlTopicSerDe extractTopicSerDe(final String serde) { case DataSource.JSON_SERDE_NAME: return new KsqlJsonTopicSerDe(); case DataSource.DELIMITED_SERDE_NAME: + if (properties.containsKey(DdlConfig.DELIMITER_PROPERTY)) { + return new KsqlDelimitedTopicSerDe( + properties.get(DdlConfig.DELIMITER_PROPERTY).toString()); + } return new KsqlDelimitedTopicSerDe(); default: throw new KsqlException("The specified topic serde is not supported."); diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializer.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializer.java index 7f161ef025bf..a0aaa974f04a 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializer.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializer.java @@ -30,9 +30,16 @@ public class KsqlDelimitedDeserializer implements Deserializer { private final Schema schema; + private final CSVFormat csvFormat; public KsqlDelimitedDeserializer(final Schema schema) { this.schema = schema; + this.csvFormat = CSVFormat.DEFAULT; + } + + public KsqlDelimitedDeserializer(final Schema schema, final CSVFormat csvFormat) { + this.schema = schema; + this.csvFormat = csvFormat; } @Override @@ -46,7 +53,7 @@ public GenericRow deserialize(final String topic, final byte[] bytes) { } final String recordCsvString = new String(bytes, StandardCharsets.UTF_8); try { - final List csvRecords = CSVParser.parse(recordCsvString, CSVFormat.DEFAULT) + final List csvRecords = CSVParser.parse(recordCsvString, csvFormat) .getRecords(); if (csvRecords == null || csvRecords.isEmpty()) { diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializer.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializer.java index 95ffe6494c88..fca1169cb6db 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializer.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializer.java @@ -28,9 +28,16 @@ public class KsqlDelimitedSerializer implements Serializer { private final Schema schema; + private final CSVFormat csvFormat; public KsqlDelimitedSerializer(final Schema schema) { this.schema = schema; + this.csvFormat = CSVFormat.DEFAULT; + } + + public KsqlDelimitedSerializer(final Schema schema, final CSVFormat csvFormat) { + this.schema = schema; + this.csvFormat = csvFormat; } @Override @@ -45,7 +52,7 @@ public byte[] serialize(final String topic, final GenericRow genericRow) { } try { final StringWriter stringWriter = new StringWriter(); - final CSVPrinter csvPrinter = new CSVPrinter(stringWriter, CSVFormat.DEFAULT); + final CSVPrinter csvPrinter = new CSVPrinter(stringWriter, csvFormat); csvPrinter.printRecord(genericRow.getColumns()); final String result = stringWriter.toString(); return result.substring(0, result.length() - 2).getBytes(StandardCharsets.UTF_8); diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java index d354d9b6caf2..91a4234a830e 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java @@ -23,6 +23,8 @@ import java.util.Map; import java.util.function.Supplier; +import org.apache.commons.csv.CSVFormat; + import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; @@ -30,10 +32,19 @@ import org.apache.kafka.connect.data.Schema; + public class KsqlDelimitedTopicSerDe extends KsqlTopicSerDe { + private final CSVFormat csvFormat; + public KsqlDelimitedTopicSerDe() { super(DataSource.DataSourceSerDe.DELIMITED); + this.csvFormat = CSVFormat.DEFAULT; + } + + public KsqlDelimitedTopicSerDe(final String csvFormat) { + super(DataSource.DataSourceSerDe.DELIMITED); + this.csvFormat = csvFormat == null ? CSVFormat.DEFAULT : CSVFormat.valueOf(csvFormat); } @Override @@ -44,10 +55,12 @@ public Serde getGenericRowSerde( final Supplier schemaRegistryClientFactory) { final Map serdeProps = new HashMap<>(); - final Serializer genericRowSerializer = new KsqlDelimitedSerializer(schema); + final Serializer genericRowSerializer = new KsqlDelimitedSerializer( + schema, csvFormat); genericRowSerializer.configure(serdeProps, false); - final Deserializer genericRowDeserializer = new KsqlDelimitedDeserializer(schema); + final Deserializer genericRowDeserializer = new KsqlDelimitedDeserializer( + schema, csvFormat); genericRowDeserializer.configure(serdeProps, false); return Serdes.serdeFrom(genericRowSerializer, genericRowDeserializer); diff --git a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java index ba65f6a8ba74..7fa1370bc915 100644 --- a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java +++ b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java @@ -20,6 +20,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import io.confluent.ksql.GenericRow; import java.nio.charset.StandardCharsets; +import org.apache.commons.csv.CSVFormat; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; import org.junit.Assert; @@ -70,4 +71,33 @@ public void shouldDeserializeJsonCorrectlyWithRedundantFields() throws JsonProce Assert.assertNull(genericRow.getColumns().get(3)); } + @Test + public void shouldDeserializeDelimitedCorrectlyWithOtherDelimiters() { + final String rowString = "1511897796092\t1\titem_1\t10.0\r\n"; + + final KsqlDelimitedDeserializer ksqlJsonDeserializer = new KsqlDelimitedDeserializer(orderSchema, CSVFormat.TDF); + + final GenericRow genericRow = ksqlJsonDeserializer.deserialize("", rowString.getBytes()); + assertThat(genericRow.getColumns().size(), equalTo(4)); + assertThat((Long) genericRow.getColumns().get(0), equalTo(1511897796092L)); + assertThat((Long) genericRow.getColumns().get(1), equalTo(1L)); + assertThat((String) genericRow.getColumns().get(2), equalTo("item_1")); + assertThat((Double) genericRow.getColumns().get(3), equalTo(10.0)); + } + + @Test + public void shouldDeserializeJsonCorrectlyWithRedundantFieldsWithOtherDelimiters() throws JsonProcessingException { + + final String rowString = "1511897796092\t1\titem_1\t\r\n"; + + final KsqlDelimitedDeserializer ksqlJsonDeserializer = new KsqlDelimitedDeserializer(orderSchema, CSVFormat.TDF); + + final GenericRow genericRow = ksqlJsonDeserializer.deserialize("", rowString.getBytes()); + assertThat(genericRow.getColumns().size(), equalTo(4)); + assertThat((Long) genericRow.getColumns().get(0), equalTo(1511897796092L)); + assertThat((Long) genericRow.getColumns().get(1), equalTo(1L)); + assertThat((String) genericRow.getColumns().get(2), equalTo("item_1")); + Assert.assertNull(genericRow.getColumns().get(3)); + } + } From 671035719f01daa60a3f556ff3e5f9a1ff352e13 Mon Sep 17 00:00:00 2001 From: Hasnat Ullah Date: Thu, 18 Oct 2018 23:57:28 +0100 Subject: [PATCH 02/46] Tests + Code review changes --- docs/developer-guide/syntax-reference.rst | 12 +++++ .../java/io/confluent/ksql/ddl/DdlConfig.java | 2 +- .../io/confluent/ksql/analyzer/Analyzer.java | 1 + .../commands/AbstractCreateStreamCommand.java | 1 + .../ddl/commands/RegisterTopicCommand.java | 25 +++++++--- .../delimited-value-format.json | 48 +++++++++++++++++++ .../delimited/KsqlDelimitedSerializer.java | 9 +--- .../delimited/KsqlDelimitedTopicSerDe.java | 3 +- .../KsqlDelimitedDeserializerTest.java | 11 +++-- 9 files changed, 90 insertions(+), 22 deletions(-) create mode 100644 ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json diff --git a/docs/developer-guide/syntax-reference.rst b/docs/developer-guide/syntax-reference.rst index f1c4cda68201..c2dca44222b6 100644 --- a/docs/developer-guide/syntax-reference.rst +++ b/docs/developer-guide/syntax-reference.rst @@ -218,6 +218,10 @@ The WITH clause supports the following properties: | VALUE_FORMAT (required) | Specifies the serialization format of the message value in the topic. Supported formats: | | | ``JSON``, ``DELIMITED`` (comma-separated value), and ``AVRO``. | +-------------------------+--------------------------------------------------------------------------------------------+ +| DELIMITER_FORMAT | Used when VALUE_FORMAT='DELIMITED'. Supported formats: DEFAULT, EXCEL, INFORMIX_UNLOAD, | +| | INFORMIX_UNLOAD_CSV, MYSQL, RFC4180, ORACLE, POSTGRESQL_CSV, POSTGRESQL_TEXT, TDF. | +| | See: org.apache.commons.csv.CSVFormat | ++-------------------------+--------------------------------------------------------------------------------------------+ | KEY | Optimization hint: If the Kafka message key is also present as a field/column in the Kafka | | | message value, you may set this property to associate the corresponding field/column with | | | the implicit ``ROWKEY`` column (message key). | @@ -321,6 +325,10 @@ The WITH clause supports the following properties: | VALUE_FORMAT (required) | Specifies the serialization format of message values in the topic. Supported formats: | | | ``JSON``, ``DELIMITED`` (comma-separated value), and ``AVRO``. | +-------------------------+--------------------------------------------------------------------------------------------+ +| DELIMITER_FORMAT | Used when VALUE_FORMAT='DELIMITED'. Supported formats: DEFAULT, EXCEL, INFORMIX_UNLOAD, | +| | INFORMIX_UNLOAD_CSV, MYSQL, RFC4180, ORACLE, POSTGRESQL_CSV, POSTGRESQL_TEXT, TDF. | +| | See: org.apache.commons.csv.CSVFormat | ++-------------------------+--------------------------------------------------------------------------------------------+ | KEY (required) | Associates a field/column within the Kafka message value with the implicit ``ROWKEY`` | | | column (message key) in the KSQL table. | | | | @@ -420,6 +428,10 @@ The WITH clause for the result supports the following properties: | | ``JSON``, ``DELIMITED`` (comma-separated value), and ``AVRO``. If this property is not | | | set, then the format of the input stream/table is used. | +-------------------------+------------------------------------------------------------------------------------------------------+ +| DELIMITER_FORMAT | Used when VALUE_FORMAT='DELIMITED'. Supported formats: DEFAULT, EXCEL, INFORMIX_UNLOAD, | +| | INFORMIX_UNLOAD_CSV, MYSQL, RFC4180, ORACLE, POSTGRESQL_CSV, POSTGRESQL_TEXT, TDF. | +| | See: org.apache.commons.csv.CSVFormat | ++-------------------------+------------------------------------------------------------------------------------------------------+ | PARTITIONS | The number of partitions in the backing topic. If this property is not set, then the number | | | of partitions is taken from the value of the ``ksql.sink.partitions`` property, which | | | defaults to four partitions. The ``ksql.sink.partitions`` property can be set in the | diff --git a/ksql-common/src/main/java/io/confluent/ksql/ddl/DdlConfig.java b/ksql-common/src/main/java/io/confluent/ksql/ddl/DdlConfig.java index 82f711d623e1..14568dfde6b8 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/ddl/DdlConfig.java +++ b/ksql-common/src/main/java/io/confluent/ksql/ddl/DdlConfig.java @@ -17,7 +17,7 @@ public final class DdlConfig { public static final String VALUE_FORMAT_PROPERTY = "VALUE_FORMAT"; - public static final String DELIMITER_PROPERTY = "DELIMITER"; + public static final String DELIMITER_FORMAT_PROPERTY = "DELIMITER_FORMAT"; public static final String AVRO_SCHEMA_FILE = "AVROSCHEMAFILE"; public static final String AVRO_SCHEMA = "AVROSCHEMA"; public static final String KAFKA_TOPIC_NAME_PROPERTY = "KAFKA_TOPIC"; diff --git a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java index ef3ab7d2da50..be89cdd5e927 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java @@ -667,6 +667,7 @@ private void validateWithClause(final Set withClauseVariables) { final Set validSet = new HashSet<>(); validSet.add(DdlConfig.VALUE_FORMAT_PROPERTY.toUpperCase()); validSet.add(DdlConfig.KAFKA_TOPIC_NAME_PROPERTY.toUpperCase()); + validSet.add(DdlConfig.DELIMITER_FORMAT_PROPERTY.toUpperCase()); validSet.add(DdlConfig.PARTITION_BY_PROPERTY.toUpperCase()); validSet.add(KsqlConstants.SINK_TIMESTAMP_COLUMN_NAME.toUpperCase()); validSet.add(KsqlConstants.SINK_NUMBER_OF_PARTITIONS.toUpperCase()); diff --git a/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/AbstractCreateStreamCommand.java b/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/AbstractCreateStreamCommand.java index 3c9421bfbb4b..5c4f83d384c0 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/AbstractCreateStreamCommand.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/AbstractCreateStreamCommand.java @@ -189,6 +189,7 @@ private void validateWithClause(final Set withClauseVariables) { final Set validSet = new HashSet<>(); validSet.add(DdlConfig.VALUE_FORMAT_PROPERTY.toUpperCase()); validSet.add(DdlConfig.KAFKA_TOPIC_NAME_PROPERTY.toUpperCase()); + validSet.add(DdlConfig.DELIMITER_FORMAT_PROPERTY.toUpperCase()); validSet.add(DdlConfig.KEY_NAME_PROPERTY.toUpperCase()); validSet.add(DdlConfig.WINDOW_TYPE_PROPERTY.toUpperCase()); validSet.add(DdlConfig.TIMESTAMP_NAME_PROPERTY.toUpperCase()); diff --git a/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/RegisterTopicCommand.java b/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/RegisterTopicCommand.java index 236fe0afa33d..20b3b777334c 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/RegisterTopicCommand.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/RegisterTopicCommand.java @@ -49,7 +49,7 @@ public RegisterTopicCommand(final RegisterTopic registerTopic) { enforceTopicProperties(properties); this.kafkaTopicName = StringUtil.cleanQuotes( properties.get(DdlConfig.KAFKA_TOPIC_NAME_PROPERTY).toString()); - final String serde = StringUtil.cleanQuotes( + final String serde = cleanQuotesAndUpperCase( properties.get(DdlConfig.VALUE_FORMAT_PROPERTY).toString()); this.topicSerDe = extractTopicSerDe(serde, properties); this.notExists = notExist; @@ -57,17 +57,16 @@ public RegisterTopicCommand(final RegisterTopic registerTopic) { private KsqlTopicSerDe extractTopicSerDe(final String serde, final Map properties) { - // TODO: Find a way to avoid calling toUpperCase() here; - // if the property can be an unquoted identifier, then capitalization will have already happened - switch (serde.toUpperCase()) { + switch (serde) { case DataSource.AVRO_SERDE_NAME: return new KsqlAvroTopicSerDe(); case DataSource.JSON_SERDE_NAME: return new KsqlJsonTopicSerDe(); case DataSource.DELIMITED_SERDE_NAME: - if (properties.containsKey(DdlConfig.DELIMITER_PROPERTY)) { + if (properties.containsKey(DdlConfig.DELIMITER_FORMAT_PROPERTY)) { return new KsqlDelimitedTopicSerDe( - properties.get(DdlConfig.DELIMITER_PROPERTY).toString()); + cleanQuotesAndUpperCase(properties.get(DdlConfig.DELIMITER_FORMAT_PROPERTY).toString()) + ); } return new KsqlDelimitedTopicSerDe(); default: @@ -75,6 +74,10 @@ private KsqlTopicSerDe extractTopicSerDe(final String serde, } } + private String cleanQuotesAndUpperCase(final String string) { + return StringUtil.cleanQuotes(string.toUpperCase()); + } + private void enforceTopicProperties(final Map properties) { if (properties.size() == 0) { throw new KsqlException("Register topic statement needs WITH clause."); @@ -84,6 +87,16 @@ private void enforceTopicProperties(final Map properties) { throw new KsqlException("Topic format(format) should be set in WITH clause."); } + if (properties.containsKey(DdlConfig.DELIMITER_FORMAT_PROPERTY) && ( + !properties.containsKey(DdlConfig.VALUE_FORMAT_PROPERTY) + || !cleanQuotesAndUpperCase(properties.get(DdlConfig.VALUE_FORMAT_PROPERTY).toString()) + .equals(DataSource.DELIMITED_SERDE_NAME))) { + throw new KsqlException( + DdlConfig.DELIMITER_FORMAT_PROPERTY + " can only be used with " + + DdlConfig.VALUE_FORMAT_PROPERTY + "='" + DataSource.DELIMITED_SERDE_NAME + "'" + ); + } + if (!properties.containsKey(DdlConfig.KAFKA_TOPIC_NAME_PROPERTY)) { throw new KsqlException("Corresponding kafka topic should be set in WITH clause."); } diff --git a/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json b/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json new file mode 100644 index 000000000000..da2239947658 --- /dev/null +++ b/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json @@ -0,0 +1,48 @@ +{ + "comments": [ + "When using value_format DELIMITED, we can define DELIMITER_FORMAT as per org.apache.commons.csv.CSVFormat ", + "Supported value DEFAULT, EXCEL, INFORMIX_UNLOAD, INFORMIX_UNLOAD_CSV, MYSQL, RFC4180, ORACLE, POSTGRESQL_CSV,", + ",POSTGRESQL_TEXT, TDF.", + "More details on https://commons.apache.org/proper/commons-csv/apidocs/org/apache/commons/csv/CSVFormat.html" + ], + "tests": [ + { + "name": "select delimited value_format", + "statements": [ + "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED');", + "CREATE TABLE S2 as SELECT id, name, value FROM test;" + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": "0,zero,0", "timestamp": 0}, + {"topic": "test_topic", "key": 0, "value": "0,100,100", "timestamp": 0}, + {"topic": "test_topic", "key": 100, "value": "100,100,500", "timestamp": 0}, + {"topic": "test_topic", "key": 100, "value": "100,100,100", "timestamp": 0} + ], + "outputs": [ + {"topic": "S2", "key": 0, "value": "0,zero,0", "timestamp": 0}, + {"topic": "S2", "key": 0, "value": "0,100,100", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100,100,500", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100,100,100", "timestamp": 0} + ] + }, + { + "name": "select delimited value_format with tab separated values", + "statements": [ + "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', delimiter_format='TDF');", + "CREATE TABLE S2 as SELECT id, name, value FROM test;" + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": "0\tzero\t0", "timestamp": 0}, + {"topic": "test_topic", "key": 0, "value": "0\t100\t100", "timestamp": 0}, + {"topic": "test_topic", "key": 100, "value": "100\t100\t500", "timestamp": 0}, + {"topic": "test_topic", "key": 100, "value": "100\t100\t100", "timestamp": 0} + ], + "outputs": [ + {"topic": "S2", "key": 0, "value": "0,zero,0", "timestamp": 0}, + {"topic": "S2", "key": 0, "value": "0,100,100", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100,100,500", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100,100,100", "timestamp": 0} + ] + } + ] +} \ No newline at end of file diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializer.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializer.java index fca1169cb6db..95ffe6494c88 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializer.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializer.java @@ -28,16 +28,9 @@ public class KsqlDelimitedSerializer implements Serializer { private final Schema schema; - private final CSVFormat csvFormat; public KsqlDelimitedSerializer(final Schema schema) { this.schema = schema; - this.csvFormat = CSVFormat.DEFAULT; - } - - public KsqlDelimitedSerializer(final Schema schema, final CSVFormat csvFormat) { - this.schema = schema; - this.csvFormat = csvFormat; } @Override @@ -52,7 +45,7 @@ public byte[] serialize(final String topic, final GenericRow genericRow) { } try { final StringWriter stringWriter = new StringWriter(); - final CSVPrinter csvPrinter = new CSVPrinter(stringWriter, csvFormat); + final CSVPrinter csvPrinter = new CSVPrinter(stringWriter, CSVFormat.DEFAULT); csvPrinter.printRecord(genericRow.getColumns()); final String result = stringWriter.toString(); return result.substring(0, result.length() - 2).getBytes(StandardCharsets.UTF_8); diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java index 91a4234a830e..1caeef6df732 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java @@ -55,8 +55,7 @@ public Serde getGenericRowSerde( final Supplier schemaRegistryClientFactory) { final Map serdeProps = new HashMap<>(); - final Serializer genericRowSerializer = new KsqlDelimitedSerializer( - schema, csvFormat); + final Serializer genericRowSerializer = new KsqlDelimitedSerializer(schema); genericRowSerializer.configure(serdeProps, false); final Deserializer genericRowDeserializer = new KsqlDelimitedDeserializer( diff --git a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java index 7fa1370bc915..4b21dee76fd8 100644 --- a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java +++ b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java @@ -72,7 +72,7 @@ public void shouldDeserializeJsonCorrectlyWithRedundantFields() throws JsonProce } @Test - public void shouldDeserializeDelimitedCorrectlyWithOtherDelimiters() { + public void shouldDeserializeDelimitedCorrectlyWithOtherDelimitersTDF() { final String rowString = "1511897796092\t1\titem_1\t10.0\r\n"; final KsqlDelimitedDeserializer ksqlJsonDeserializer = new KsqlDelimitedDeserializer(orderSchema, CSVFormat.TDF); @@ -85,10 +85,10 @@ public void shouldDeserializeDelimitedCorrectlyWithOtherDelimiters() { assertThat((Double) genericRow.getColumns().get(3), equalTo(10.0)); } - @Test - public void shouldDeserializeJsonCorrectlyWithRedundantFieldsWithOtherDelimiters() throws JsonProcessingException { - final String rowString = "1511897796092\t1\titem_1\t\r\n"; + @Test + public void shouldDeserializeDelimitedCorrectlyWithOtherDelimiters() { + final String rowString = "1511897796092\t1\titem_1\t10.0\r\n"; final KsqlDelimitedDeserializer ksqlJsonDeserializer = new KsqlDelimitedDeserializer(orderSchema, CSVFormat.TDF); @@ -97,7 +97,8 @@ public void shouldDeserializeJsonCorrectlyWithRedundantFieldsWithOtherDelimiters assertThat((Long) genericRow.getColumns().get(0), equalTo(1511897796092L)); assertThat((Long) genericRow.getColumns().get(1), equalTo(1L)); assertThat((String) genericRow.getColumns().get(2), equalTo("item_1")); - Assert.assertNull(genericRow.getColumns().get(3)); + assertThat((Double) genericRow.getColumns().get(3), equalTo(10.0)); } + } From e3481be66384df67e38390ed58b20b02ae0d8da5 Mon Sep 17 00:00:00 2001 From: Hasnat Ullah Date: Fri, 19 Oct 2018 08:01:36 +0100 Subject: [PATCH 03/46] Fix formatting in sytax reference docs --- docs/developer-guide/syntax-reference.rst | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/developer-guide/syntax-reference.rst b/docs/developer-guide/syntax-reference.rst index c2dca44222b6..0780ab9181ed 100644 --- a/docs/developer-guide/syntax-reference.rst +++ b/docs/developer-guide/syntax-reference.rst @@ -218,7 +218,7 @@ The WITH clause supports the following properties: | VALUE_FORMAT (required) | Specifies the serialization format of the message value in the topic. Supported formats: | | | ``JSON``, ``DELIMITED`` (comma-separated value), and ``AVRO``. | +-------------------------+--------------------------------------------------------------------------------------------+ -| DELIMITER_FORMAT | Used when VALUE_FORMAT='DELIMITED'. Supported formats: DEFAULT, EXCEL, INFORMIX_UNLOAD, | +| DELIMITER_FORMAT | Used when VALUE_FORMAT='DELIMITED'. Supported formats: DEFAULT, EXCEL, INFORMIX_UNLOAD, | | | INFORMIX_UNLOAD_CSV, MYSQL, RFC4180, ORACLE, POSTGRESQL_CSV, POSTGRESQL_TEXT, TDF. | | | See: org.apache.commons.csv.CSVFormat | +-------------------------+--------------------------------------------------------------------------------------------+ @@ -327,7 +327,7 @@ The WITH clause supports the following properties: +-------------------------+--------------------------------------------------------------------------------------------+ | DELIMITER_FORMAT | Used when VALUE_FORMAT='DELIMITED'. Supported formats: DEFAULT, EXCEL, INFORMIX_UNLOAD, | | | INFORMIX_UNLOAD_CSV, MYSQL, RFC4180, ORACLE, POSTGRESQL_CSV, POSTGRESQL_TEXT, TDF. | -| | See: org.apache.commons.csv.CSVFormat | +| | For more information,. see org.apache.commons.csv.CSVFormat | +-------------------------+--------------------------------------------------------------------------------------------+ | KEY (required) | Associates a field/column within the Kafka message value with the implicit ``ROWKEY`` | | | column (message key) in the KSQL table. | @@ -428,7 +428,7 @@ The WITH clause for the result supports the following properties: | | ``JSON``, ``DELIMITED`` (comma-separated value), and ``AVRO``. If this property is not | | | set, then the format of the input stream/table is used. | +-------------------------+------------------------------------------------------------------------------------------------------+ -| DELIMITER_FORMAT | Used when VALUE_FORMAT='DELIMITED'. Supported formats: DEFAULT, EXCEL, INFORMIX_UNLOAD, | +| DELIMITER_FORMAT | Used when VALUE_FORMAT='DELIMITED'. Supported formats: DEFAULT, EXCEL, INFORMIX_UNLOAD, | | | INFORMIX_UNLOAD_CSV, MYSQL, RFC4180, ORACLE, POSTGRESQL_CSV, POSTGRESQL_TEXT, TDF. | | | See: org.apache.commons.csv.CSVFormat | +-------------------------+------------------------------------------------------------------------------------------------------+ From eb230a4094cf8e4175c5a961ff19e373b44de012 Mon Sep 17 00:00:00 2001 From: Hasnat Ullah Date: Thu, 27 Dec 2018 08:57:48 +0000 Subject: [PATCH 04/46] Adds option to use custom delimiters when VALUE_FORMAT='DELIMITED' --- docs/developer-guide/syntax-reference.rst | 22 +++++--- .../java/io/confluent/ksql/ddl/DdlConfig.java | 2 +- .../io/confluent/ksql/analyzer/Analyzer.java | 2 +- .../commands/AbstractCreateStreamCommand.java | 2 +- .../ddl/commands/RegisterTopicCommand.java | 11 ++-- .../delimited-value-format.json | 51 ++++++++++++++++--- .../delimited/KsqlDelimitedTopicSerDe.java | 9 +++- .../KsqlDelimitedDeserializerTest.java | 7 +-- 8 files changed, 79 insertions(+), 27 deletions(-) diff --git a/docs/developer-guide/syntax-reference.rst b/docs/developer-guide/syntax-reference.rst index 0780ab9181ed..0ebdef950345 100644 --- a/docs/developer-guide/syntax-reference.rst +++ b/docs/developer-guide/syntax-reference.rst @@ -218,8 +218,9 @@ The WITH clause supports the following properties: | VALUE_FORMAT (required) | Specifies the serialization format of the message value in the topic. Supported formats: | | | ``JSON``, ``DELIMITED`` (comma-separated value), and ``AVRO``. | +-------------------------+--------------------------------------------------------------------------------------------+ -| DELIMITER_FORMAT | Used when VALUE_FORMAT='DELIMITED'. Supported formats: DEFAULT, EXCEL, INFORMIX_UNLOAD, | -| | INFORMIX_UNLOAD_CSV, MYSQL, RFC4180, ORACLE, POSTGRESQL_CSV, POSTGRESQL_TEXT, TDF. | +| VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supported formats: single character to be delimiter or | +| | DEFAULT, EXCEL, INFORMIX_UNLOAD, INFORMIX_UNLOAD_CSV, MYSQL, RFC4180, ORACLE, | +| | POSTGRESQL_CSV, POSTGRESQL_TEXT, TDF. | | | See: org.apache.commons.csv.CSVFormat | +-------------------------+--------------------------------------------------------------------------------------------+ | KEY | Optimization hint: If the Kafka message key is also present as a field/column in the Kafka | @@ -325,9 +326,10 @@ The WITH clause supports the following properties: | VALUE_FORMAT (required) | Specifies the serialization format of message values in the topic. Supported formats: | | | ``JSON``, ``DELIMITED`` (comma-separated value), and ``AVRO``. | +-------------------------+--------------------------------------------------------------------------------------------+ -| DELIMITER_FORMAT | Used when VALUE_FORMAT='DELIMITED'. Supported formats: DEFAULT, EXCEL, INFORMIX_UNLOAD, | -| | INFORMIX_UNLOAD_CSV, MYSQL, RFC4180, ORACLE, POSTGRESQL_CSV, POSTGRESQL_TEXT, TDF. | -| | For more information,. see org.apache.commons.csv.CSVFormat | +| VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supported formats: single character to be delimiter or | +| | DEFAULT, EXCEL, INFORMIX_UNLOAD, INFORMIX_UNLOAD_CSV, MYSQL, RFC4180, ORACLE, | +| | POSTGRESQL_CSV, POSTGRESQL_TEXT, TDF. | +| | See: org.apache.commons.csv.CSVFormat | +-------------------------+--------------------------------------------------------------------------------------------+ | KEY (required) | Associates a field/column within the Kafka message value with the implicit ``ROWKEY`` | | | column (message key) in the KSQL table. | @@ -428,8 +430,9 @@ The WITH clause for the result supports the following properties: | | ``JSON``, ``DELIMITED`` (comma-separated value), and ``AVRO``. If this property is not | | | set, then the format of the input stream/table is used. | +-------------------------+------------------------------------------------------------------------------------------------------+ -| DELIMITER_FORMAT | Used when VALUE_FORMAT='DELIMITED'. Supported formats: DEFAULT, EXCEL, INFORMIX_UNLOAD, | -| | INFORMIX_UNLOAD_CSV, MYSQL, RFC4180, ORACLE, POSTGRESQL_CSV, POSTGRESQL_TEXT, TDF. | +| VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supported formats: single character to be delimiter or | +| | DEFAULT, EXCEL, INFORMIX_UNLOAD, INFORMIX_UNLOAD_CSV, MYSQL, RFC4180, ORACLE, | +| | POSTGRESQL_CSV, POSTGRESQL_TEXT, TDF. | | | See: org.apache.commons.csv.CSVFormat | +-------------------------+------------------------------------------------------------------------------------------------------+ | PARTITIONS | The number of partitions in the backing topic. If this property is not set, then the number | @@ -513,6 +516,11 @@ The WITH clause supports the following properties: | | ``JSON``, ``DELIMITED`` (comma-separated value), and ``AVRO``. If this property is not | | | set, then the format of the input stream or table is used. | +-------------------------+------------------------------------------------------------------------------------------------------+ +| VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supported formats: single character to be delimiter or | +| | DEFAULT, EXCEL, INFORMIX_UNLOAD, INFORMIX_UNLOAD_CSV, MYSQL, RFC4180, ORACLE, | +| | POSTGRESQL_CSV, POSTGRESQL_TEXT, TDF. | +| | See: org.apache.commons.csv.CSVFormat | ++-------------------------+------------------------------------------------------------------------------------------------------+ | PARTITIONS | The number of partitions in the backing topic. If this property is not set, then the number | | | of partitions is taken from the value of the ``ksql.sink.partitions`` property, which | | | defaults to four partitions. The ``ksql.sink.partitions`` property can be set in the | diff --git a/ksql-common/src/main/java/io/confluent/ksql/ddl/DdlConfig.java b/ksql-common/src/main/java/io/confluent/ksql/ddl/DdlConfig.java index 14568dfde6b8..88ba72605487 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/ddl/DdlConfig.java +++ b/ksql-common/src/main/java/io/confluent/ksql/ddl/DdlConfig.java @@ -17,7 +17,7 @@ public final class DdlConfig { public static final String VALUE_FORMAT_PROPERTY = "VALUE_FORMAT"; - public static final String DELIMITER_FORMAT_PROPERTY = "DELIMITER_FORMAT"; + public static final String VALUE_DELIMITER_PROPERTY = "VALUE_DELIMITER"; public static final String AVRO_SCHEMA_FILE = "AVROSCHEMAFILE"; public static final String AVRO_SCHEMA = "AVROSCHEMA"; public static final String KAFKA_TOPIC_NAME_PROPERTY = "KAFKA_TOPIC"; diff --git a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java index be89cdd5e927..d35f80d91de6 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java @@ -667,7 +667,7 @@ private void validateWithClause(final Set withClauseVariables) { final Set validSet = new HashSet<>(); validSet.add(DdlConfig.VALUE_FORMAT_PROPERTY.toUpperCase()); validSet.add(DdlConfig.KAFKA_TOPIC_NAME_PROPERTY.toUpperCase()); - validSet.add(DdlConfig.DELIMITER_FORMAT_PROPERTY.toUpperCase()); + validSet.add(DdlConfig.VALUE_DELIMITER_PROPERTY.toUpperCase()); validSet.add(DdlConfig.PARTITION_BY_PROPERTY.toUpperCase()); validSet.add(KsqlConstants.SINK_TIMESTAMP_COLUMN_NAME.toUpperCase()); validSet.add(KsqlConstants.SINK_NUMBER_OF_PARTITIONS.toUpperCase()); diff --git a/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/AbstractCreateStreamCommand.java b/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/AbstractCreateStreamCommand.java index 5c4f83d384c0..dca0968d174d 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/AbstractCreateStreamCommand.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/AbstractCreateStreamCommand.java @@ -189,7 +189,7 @@ private void validateWithClause(final Set withClauseVariables) { final Set validSet = new HashSet<>(); validSet.add(DdlConfig.VALUE_FORMAT_PROPERTY.toUpperCase()); validSet.add(DdlConfig.KAFKA_TOPIC_NAME_PROPERTY.toUpperCase()); - validSet.add(DdlConfig.DELIMITER_FORMAT_PROPERTY.toUpperCase()); + validSet.add(DdlConfig.VALUE_DELIMITER_PROPERTY.toUpperCase()); validSet.add(DdlConfig.KEY_NAME_PROPERTY.toUpperCase()); validSet.add(DdlConfig.WINDOW_TYPE_PROPERTY.toUpperCase()); validSet.add(DdlConfig.TIMESTAMP_NAME_PROPERTY.toUpperCase()); diff --git a/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/RegisterTopicCommand.java b/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/RegisterTopicCommand.java index 20b3b777334c..5a228cdcb951 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/RegisterTopicCommand.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/RegisterTopicCommand.java @@ -63,10 +63,9 @@ private KsqlTopicSerDe extractTopicSerDe(final String serde, case DataSource.JSON_SERDE_NAME: return new KsqlJsonTopicSerDe(); case DataSource.DELIMITED_SERDE_NAME: - if (properties.containsKey(DdlConfig.DELIMITER_FORMAT_PROPERTY)) { - return new KsqlDelimitedTopicSerDe( - cleanQuotesAndUpperCase(properties.get(DdlConfig.DELIMITER_FORMAT_PROPERTY).toString()) - ); + if (properties.containsKey(DdlConfig.VALUE_DELIMITER_PROPERTY)) { + return new KsqlDelimitedTopicSerDe(StringUtil.cleanQuotes( + properties.get(DdlConfig.VALUE_DELIMITER_PROPERTY).toString())); } return new KsqlDelimitedTopicSerDe(); default: @@ -87,12 +86,12 @@ private void enforceTopicProperties(final Map properties) { throw new KsqlException("Topic format(format) should be set in WITH clause."); } - if (properties.containsKey(DdlConfig.DELIMITER_FORMAT_PROPERTY) && ( + if (properties.containsKey(DdlConfig.VALUE_DELIMITER_PROPERTY) && ( !properties.containsKey(DdlConfig.VALUE_FORMAT_PROPERTY) || !cleanQuotesAndUpperCase(properties.get(DdlConfig.VALUE_FORMAT_PROPERTY).toString()) .equals(DataSource.DELIMITED_SERDE_NAME))) { throw new KsqlException( - DdlConfig.DELIMITER_FORMAT_PROPERTY + " can only be used with " + DdlConfig.VALUE_DELIMITER_PROPERTY + " can only be used with " + DdlConfig.VALUE_FORMAT_PROPERTY + "='" + DataSource.DELIMITED_SERDE_NAME + "'" ); } diff --git a/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json b/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json index da2239947658..3b970aa1697a 100644 --- a/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json +++ b/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json @@ -1,8 +1,9 @@ { "comments": [ - "When using value_format DELIMITED, we can define DELIMITER_FORMAT as per org.apache.commons.csv.CSVFormat ", + "When using value_format DELIMITED, we can define VALUE_DELIMITER as custom character or ", + "one of the formats in org.apache.commons.csv.CSVFormat ", "Supported value DEFAULT, EXCEL, INFORMIX_UNLOAD, INFORMIX_UNLOAD_CSV, MYSQL, RFC4180, ORACLE, POSTGRESQL_CSV,", - ",POSTGRESQL_TEXT, TDF.", + ",POSTGRESQL_TEXT, TDF", "More details on https://commons.apache.org/proper/commons-csv/apidocs/org/apache/commons/csv/CSVFormat.html" ], "tests": [ @@ -10,7 +11,7 @@ "name": "select delimited value_format", "statements": [ "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED');", - "CREATE TABLE S2 as SELECT id, name, value FROM test;" + "CREATE STREAM S2 as SELECT id, name, value FROM test;" ], "inputs": [ {"topic": "test_topic", "key": 0, "value": "0,zero,0", "timestamp": 0}, @@ -26,10 +27,10 @@ ] }, { - "name": "select delimited value_format with tab separated values", + "name": "select delimited value_format with tab separated values using CsvFormat constant", "statements": [ - "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', delimiter_format='TDF');", - "CREATE TABLE S2 as SELECT id, name, value FROM test;" + "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='TDF');", + "CREATE STREAM S2 as SELECT id, name, value FROM test;" ], "inputs": [ {"topic": "test_topic", "key": 0, "value": "0\tzero\t0", "timestamp": 0}, @@ -43,6 +44,44 @@ {"topic": "S2", "key": 100, "value": "100,100,500", "timestamp": 0}, {"topic": "S2", "key": 100, "value": "100,100,100", "timestamp": 0} ] + }, + { + "name": "select delimited value_format with tab separated values using custom delimiter character", + "statements": [ + "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='\t');", + "CREATE STREAM S2 as SELECT id, name, value FROM test;" + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": "0\tzero\t0", "timestamp": 0}, + {"topic": "test_topic", "key": 0, "value": "0\t100\t100", "timestamp": 0}, + {"topic": "test_topic", "key": 100, "value": "100\t100\t500", "timestamp": 0}, + {"topic": "test_topic", "key": 100, "value": "100\t100\t100", "timestamp": 0} + ], + "outputs": [ + {"topic": "S2", "key": 0, "value": "0,zero,0", "timestamp": 0}, + {"topic": "S2", "key": 0, "value": "0,100,100", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100,100,500", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100,100,100", "timestamp": 0} + ] + }, + { + "name": "select delimited value_format with pipe separated values using custom delimiter character", + "statements": [ + "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='|');", + "CREATE STREAM S2 as SELECT id, name, value FROM test;" + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": "0|zero|0", "timestamp": 0}, + {"topic": "test_topic", "key": 0, "value": "0|100|100", "timestamp": 0}, + {"topic": "test_topic", "key": 100, "value": "100|100|500", "timestamp": 0}, + {"topic": "test_topic", "key": 100, "value": "100|100|100", "timestamp": 0} + ], + "outputs": [ + {"topic": "S2", "key": 0, "value": "0,zero,0", "timestamp": 0}, + {"topic": "S2", "key": 0, "value": "0,100,100", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100,100,500", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100,100,100", "timestamp": 0} + ] } ] } \ No newline at end of file diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java index 1caeef6df732..b88d07a7432c 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java @@ -42,9 +42,14 @@ public KsqlDelimitedTopicSerDe() { this.csvFormat = CSVFormat.DEFAULT; } - public KsqlDelimitedTopicSerDe(final String csvFormat) { + public KsqlDelimitedTopicSerDe(final String delimiter) { super(DataSource.DataSourceSerDe.DELIMITED); - this.csvFormat = csvFormat == null ? CSVFormat.DEFAULT : CSVFormat.valueOf(csvFormat); + + if (delimiter.length() == 1) { + this.csvFormat = CSVFormat.DEFAULT.withDelimiter(delimiter.charAt(0)); + } else { + this.csvFormat = CSVFormat.valueOf(delimiter); + } } @Override diff --git a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java index 4b21dee76fd8..fcc78fcf3836 100644 --- a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java +++ b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java @@ -87,10 +87,11 @@ public void shouldDeserializeDelimitedCorrectlyWithOtherDelimitersTDF() { @Test - public void shouldDeserializeDelimitedCorrectlyWithOtherDelimiters() { - final String rowString = "1511897796092\t1\titem_1\t10.0\r\n"; + public void shouldDeserializeDelimitedCorrectlyWithCustomDelimiters() { + final String rowString = "1511897796092|1|item_1|10.0\r\n"; - final KsqlDelimitedDeserializer ksqlJsonDeserializer = new KsqlDelimitedDeserializer(orderSchema, CSVFormat.TDF); + final KsqlDelimitedDeserializer ksqlJsonDeserializer = new KsqlDelimitedDeserializer( + orderSchema, CSVFormat.DEFAULT.withDelimiter('|')); final GenericRow genericRow = ksqlJsonDeserializer.deserialize("", rowString.getBytes()); assertThat(genericRow.getColumns().size(), equalTo(4)); From c6d0092c8d5c551db08ce76bcfa2f38450a44b64 Mon Sep 17 00:00:00 2001 From: Hasnat Ullah Date: Mon, 18 Feb 2019 18:53:42 +0000 Subject: [PATCH 05/46] Allow only single character delimiters --- docs/developer-guide/syntax-reference.rst | 22 +++++-------------- .../delimited-value-format.json | 18 ++++++++++----- .../delimited/KsqlDelimitedTopicSerDe.java | 3 ++- 3 files changed, 21 insertions(+), 22 deletions(-) diff --git a/docs/developer-guide/syntax-reference.rst b/docs/developer-guide/syntax-reference.rst index 6e4bbf24f840..a32562330c0e 100644 --- a/docs/developer-guide/syntax-reference.rst +++ b/docs/developer-guide/syntax-reference.rst @@ -227,10 +227,8 @@ The WITH clause supports the following properties: | VALUE_FORMAT (required) | Specifies the serialization format of the message value in the topic. Supported formats: | | | ``JSON``, ``DELIMITED`` (comma-separated value), and ``AVRO``. | +-------------------------+--------------------------------------------------------------------------------------------+ -| VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supported formats: single character to be delimiter or | -| | DEFAULT, EXCEL, INFORMIX_UNLOAD, INFORMIX_UNLOAD_CSV, MYSQL, RFC4180, ORACLE, | -| | POSTGRESQL_CSV, POSTGRESQL_TEXT, TDF. | -| | See: org.apache.commons.csv.CSVFormat | +| VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supports single character to be a delimiter, | +| | defaults to ','. | +-------------------------+--------------------------------------------------------------------------------------------+ | KEY | Optimization hint: If the Kafka message key is also present as a field/column in the Kafka | | | message value, you may set this property to associate the corresponding field/column with | @@ -335,10 +333,8 @@ The WITH clause supports the following properties: | VALUE_FORMAT (required) | Specifies the serialization format of message values in the topic. Supported formats: | | | ``JSON``, ``DELIMITED`` (comma-separated value), and ``AVRO``. | +-------------------------+--------------------------------------------------------------------------------------------+ -| VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supported formats: single character to be delimiter or | -| | DEFAULT, EXCEL, INFORMIX_UNLOAD, INFORMIX_UNLOAD_CSV, MYSQL, RFC4180, ORACLE, | -| | POSTGRESQL_CSV, POSTGRESQL_TEXT, TDF. | -| | See: org.apache.commons.csv.CSVFormat | +| VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supports single character to be a delimiter, | +| | defaults to ','. | +-------------------------+--------------------------------------------------------------------------------------------+ | KEY (required) | Associates a field/column within the Kafka message value with the implicit ``ROWKEY`` | | | column (message key) in the KSQL table. | @@ -439,10 +435,7 @@ The WITH clause for the result supports the following properties: | | ``JSON``, ``DELIMITED`` (comma-separated value), and ``AVRO``. If this property is not | | | set, then the format of the input stream/table is used. | +-------------------------+------------------------------------------------------------------------------------------------------+ -| VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supported formats: single character to be delimiter or | -| | DEFAULT, EXCEL, INFORMIX_UNLOAD, INFORMIX_UNLOAD_CSV, MYSQL, RFC4180, ORACLE, | -| | POSTGRESQL_CSV, POSTGRESQL_TEXT, TDF. | -| | See: org.apache.commons.csv.CSVFormat | +| VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supports single character to be a delimiter, defaults to ','. | +-------------------------+------------------------------------------------------------------------------------------------------+ | PARTITIONS | The number of partitions in the backing topic. If this property is not set, then the number | | | of partitions is taken from the value of the ``ksql.sink.partitions`` property, which | @@ -525,10 +518,7 @@ The WITH clause supports the following properties: | | ``JSON``, ``DELIMITED`` (comma-separated value), and ``AVRO``. If this property is not | | | set, then the format of the input stream or table is used. | +-------------------------+------------------------------------------------------------------------------------------------------+ -| VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supported formats: single character to be delimiter or | -| | DEFAULT, EXCEL, INFORMIX_UNLOAD, INFORMIX_UNLOAD_CSV, MYSQL, RFC4180, ORACLE, | -| | POSTGRESQL_CSV, POSTGRESQL_TEXT, TDF. | -| | See: org.apache.commons.csv.CSVFormat | +| VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supports single character to be a delimiter, defaults to ','. | +-------------------------+------------------------------------------------------------------------------------------------------+ | PARTITIONS | The number of partitions in the backing topic. If this property is not set, then the number | | | of partitions is taken from the value of the ``ksql.sink.partitions`` property, which | diff --git a/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json b/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json index 3b970aa1697a..ac4849b9f0ef 100644 --- a/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json +++ b/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json @@ -1,10 +1,6 @@ { "comments": [ - "When using value_format DELIMITED, we can define VALUE_DELIMITER as custom character or ", - "one of the formats in org.apache.commons.csv.CSVFormat ", - "Supported value DEFAULT, EXCEL, INFORMIX_UNLOAD, INFORMIX_UNLOAD_CSV, MYSQL, RFC4180, ORACLE, POSTGRESQL_CSV,", - ",POSTGRESQL_TEXT, TDF", - "More details on https://commons.apache.org/proper/commons-csv/apidocs/org/apache/commons/csv/CSVFormat.html" + "When using value_format DELIMITED, we can define VALUE_DELIMITER as custom character." ], "tests": [ { @@ -26,6 +22,18 @@ {"topic": "S2", "key": 100, "value": "100,100,100", "timestamp": 0} ] }, + { + "name": "validate value_delimiter to be single character", + "statements": [ + "CREATE STREAM TEST WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='<~>');" + ], + "expectedException": { + "type": "io.confluent.ksql.util.KsqlException", + "message": "Only single characters are supported for VALUE_DELIMITER." + }, + "inputs": [], + "outputs": [] + }, { "name": "select delimited value_format with tab separated values using CsvFormat constant", "statements": [ diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java index b88d07a7432c..1045058912fd 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java @@ -23,6 +23,7 @@ import java.util.Map; import java.util.function.Supplier; +import io.confluent.ksql.util.KsqlException; import org.apache.commons.csv.CSVFormat; import org.apache.kafka.common.serialization.Deserializer; @@ -48,7 +49,7 @@ public KsqlDelimitedTopicSerDe(final String delimiter) { if (delimiter.length() == 1) { this.csvFormat = CSVFormat.DEFAULT.withDelimiter(delimiter.charAt(0)); } else { - this.csvFormat = CSVFormat.valueOf(delimiter); + throw new KsqlException("Only single characters are supported for VALUE_DELIMITER."); } } From c774b098c2923693d190425d6d2d37d116082aa5 Mon Sep 17 00:00:00 2001 From: Hasnat Ullah Date: Mon, 18 Feb 2019 19:11:59 +0000 Subject: [PATCH 06/46] Updates to master merge --- .../ksql/serde/delimited/KsqlDelimitedDeserializer.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializer.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializer.java index 0529889f133f..77e0da8d2254 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializer.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializer.java @@ -44,16 +44,18 @@ public KsqlDelimitedDeserializer(final Schema schema) { this.csvFormat = CSVFormat.DEFAULT; } - public KsqlDelimitedDeserializer(final Schema schema, final CSVFormat csvFormat) { + KsqlDelimitedDeserializer(final Schema schema, final CSVFormat csvFormat) { this.schema = schema; this.csvFormat = csvFormat; } KsqlDelimitedDeserializer( final Schema schema, + final CSVFormat csvFormat, final StructuredLogger recordLogger, final ProcessingLogContext processingLogContext) { this.schema = Objects.requireNonNull(schema); + this.csvFormat = Objects.requireNonNull(csvFormat); this.recordLogger = Objects.requireNonNull(recordLogger); this.processingLogContext = Objects.requireNonNull(processingLogContext); } From aaf6fa42ca5f98327cdd8c581520f32fae11ca33 Mon Sep 17 00:00:00 2001 From: Hasnat Ullah Date: Sun, 3 Mar 2019 23:52:06 +0000 Subject: [PATCH 07/46] Adds seralizer with custom delimiter --- .../java/io/confluent/ksql/ddl/DdlConfig.java | 1 + .../io/confluent/ksql/analyzer/Analyzer.java | 2 +- .../ddl/commands/RegisterTopicCommand.java | 12 +++++----- .../integration/IntegrationTestHarness.java | 3 ++- .../delimited-value-format.json | 19 +++++++++++++++ .../io/confluent/ksql/datagen/DataGen.java | 23 +++++++++++++++++-- .../ksql/datagen/DelimitedProducer.java | 9 +++++++- .../ksql/datagen/ProducerFactory.java | 3 ++- .../confluent/ksql/datagen/DataGenTest.java | 9 ++++++++ .../delimited/KsqlDelimitedDeserializer.java | 10 -------- .../delimited/KsqlDelimitedSerializer.java | 6 +++-- .../delimited/KsqlDelimitedTopicSerDe.java | 7 +----- .../KsqlDelimitedDeserializerTest.java | 14 +++++++++-- .../KsqlDelimitedSerializerTest.java | 17 ++++++++++++-- 14 files changed, 101 insertions(+), 34 deletions(-) diff --git a/ksql-common/src/main/java/io/confluent/ksql/ddl/DdlConfig.java b/ksql-common/src/main/java/io/confluent/ksql/ddl/DdlConfig.java index 3461d9d47bb4..812f66384ac3 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/ddl/DdlConfig.java +++ b/ksql-common/src/main/java/io/confluent/ksql/ddl/DdlConfig.java @@ -29,6 +29,7 @@ public final class DdlConfig { public static final String TIMESTAMP_NAME_PROPERTY = "TIMESTAMP"; public static final String PARTITION_BY_PROPERTY = "PARTITION_BY"; public static final String TIMESTAMP_FORMAT_PROPERTY = "TIMESTAMP_FORMAT"; + public static final String VALUE_DELIMITER_DEFAULT = ","; private DdlConfig() { } diff --git a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java index b69dd19f1f62..1c78e66de0ba 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java @@ -148,7 +148,7 @@ private void analyzeNonStdOutSink(final boolean doCreateInto) { intoTopicSerde = new KsqlJsonTopicSerDe(); break; case DataSource.DELIMITED_SERDE_NAME: - intoTopicSerde = new KsqlDelimitedTopicSerDe(); + intoTopicSerde = new KsqlDelimitedTopicSerDe(DdlConfig.VALUE_DELIMITER_DEFAULT); break; default: throw new KsqlException( diff --git a/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/RegisterTopicCommand.java b/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/RegisterTopicCommand.java index 151976cc42d8..f7f596057858 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/RegisterTopicCommand.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/RegisterTopicCommand.java @@ -21,6 +21,7 @@ import io.confluent.ksql.metastore.StructuredDataSource; import io.confluent.ksql.parser.tree.Expression; import io.confluent.ksql.parser.tree.RegisterTopic; +import io.confluent.ksql.parser.tree.StringLiteral; import io.confluent.ksql.serde.DataSource; import io.confluent.ksql.serde.KsqlTopicSerDe; import io.confluent.ksql.serde.avro.KsqlAvroTopicSerDe; @@ -78,17 +79,16 @@ private KsqlTopicSerDe extractTopicSerDe( case DataSource.JSON_SERDE_NAME: return new KsqlJsonTopicSerDe(); case DataSource.DELIMITED_SERDE_NAME: - if (properties.containsKey(DdlConfig.VALUE_DELIMITER_PROPERTY)) { - return new KsqlDelimitedTopicSerDe(StringUtil.cleanQuotes( - properties.get(DdlConfig.VALUE_DELIMITER_PROPERTY).toString())); - } - return new KsqlDelimitedTopicSerDe(); + final String delimiter = StringUtil.cleanQuotes(properties.getOrDefault( + DdlConfig.VALUE_DELIMITER_PROPERTY, + new StringLiteral(DdlConfig.VALUE_DELIMITER_DEFAULT)).toString()); + return new KsqlDelimitedTopicSerDe(delimiter); default: throw new KsqlException("The specified topic serde is not supported."); } } - private String cleanQuotesAndUpperCase(final String string) { + private static String cleanQuotesAndUpperCase(final String string) { return StringUtil.cleanQuotes(string.toUpperCase()); } diff --git a/ksql-engine/src/test/java/io/confluent/ksql/integration/IntegrationTestHarness.java b/ksql-engine/src/test/java/io/confluent/ksql/integration/IntegrationTestHarness.java index b805c517986b..6f5a767bcaf9 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/integration/IntegrationTestHarness.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/integration/IntegrationTestHarness.java @@ -23,6 +23,7 @@ import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import io.confluent.ksql.GenericRow; import io.confluent.ksql.KsqlContextTestUtil; +import io.confluent.ksql.ddl.DdlConfig; import io.confluent.ksql.processing.log.ProcessingLogContext; import io.confluent.ksql.serde.DataSource; import io.confluent.ksql.serde.DataSource.DataSourceSerDe; @@ -480,7 +481,7 @@ private static KsqlTopicSerDe getSerde( case AVRO: return new KsqlAvroTopicSerDe(KsqlConstants.DEFAULT_AVRO_SCHEMA_FULL_NAME); case DELIMITED: - return new KsqlDelimitedTopicSerDe(); + return new KsqlDelimitedTopicSerDe(DdlConfig.VALUE_DELIMITER_DEFAULT); default: throw new RuntimeException("Format not supported: " + dataSourceSerDe); } diff --git a/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json b/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json index ac4849b9f0ef..689803184369 100644 --- a/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json +++ b/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json @@ -90,6 +90,25 @@ {"topic": "S2", "key": 100, "value": "100,100,500", "timestamp": 0}, {"topic": "S2", "key": 100, "value": "100,100,100", "timestamp": 0} ] + }, + { + "name": "select delimited value_format with pipe separated values using custom delimiter character", + "statements": [ + "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='|');", + "CREATE STREAM S2 WITH(value_delimiter='$') AS SELECT * FROM test;" + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": "0|zero|0", "timestamp": 0}, + {"topic": "test_topic", "key": 0, "value": "0|100|100", "timestamp": 0}, + {"topic": "test_topic", "key": 100, "value": "100|100|500", "timestamp": 0}, + {"topic": "test_topic", "key": 100, "value": "100|100|100", "timestamp": 0} + ], + "outputs": [ + {"topic": "S2", "key": 0, "value": "0$zero$0", "timestamp": 0}, + {"topic": "S2", "key": 0, "value": "0$100$100", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100$100$500", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100$100$100", "timestamp": 0} + ] } ] } \ No newline at end of file diff --git a/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java b/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java index b2fbebefb79b..025998defb02 100644 --- a/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java +++ b/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java @@ -57,7 +57,7 @@ static void run(final String... args) throws IOException { final Generator generator = new Generator(arguments.schemaFile, new Random()); final DataGenProducer dataProducer = new ProducerFactory() - .getProducer(arguments.format, arguments.schemaRegistryUrl); + .getProducer(arguments.format, arguments.valueDelimiter, arguments.schemaRegistryUrl); final Properties props = getProperties(arguments); dataProducer.populateTopic( @@ -92,6 +92,8 @@ private static void usage() { + "schema= " + "[schemaRegistryUrl= (defaults to http://localhost:8081)] " + "format= (case-insensitive; one of 'avro', 'json', or 'delimited') " + + "[valueDelimiter= (used only when format is 'delimited', only " + + "single characters, defaults to ',' )] " + "topic= " + "key= " + "[iterations= (defaults to 1,000,000)] " @@ -108,6 +110,7 @@ public enum Format { AVRO, JSON, DELIMITED } private final String bootstrapServer; private final InputStream schemaFile; private final Format format; + private final String valueDelimiter; private final String topicName; private final String keyName; private final int iterations; @@ -122,6 +125,7 @@ public enum Format { AVRO, JSON, DELIMITED } final Format format, final String topicName, final String keyName, + final String valueDelimiter, final int iterations, final long maxInterval, final String schemaRegistryUrl, @@ -131,6 +135,7 @@ public enum Format { AVRO, JSON, DELIMITED } this.bootstrapServer = bootstrapServer; this.schemaFile = schemaFile; this.format = format; + this.valueDelimiter = valueDelimiter; this.topicName = topicName; this.keyName = keyName; this.iterations = iterations; @@ -154,6 +159,7 @@ private static final class Builder { .put("bootstrap-server", (builder, argVal) -> builder.bootstrapServer = argVal) .put("schema", (builder, argVal) -> builder.schemaFile = toFileInputStream(argVal)) .put("format", (builder, argVal) -> builder.format = parseFormat(argVal)) + .put("valueDelimiter", (builder, argVal) -> builder.valueDelimiter = parseValueDelimiter(argVal)) .put("topic", (builder, argVal) -> builder.topicName = argVal) .put("key", (builder, argVal) -> builder.keyName = argVal) .put("iterations", (builder, argVal) -> builder.iterations = parseIterations(argVal)) @@ -170,6 +176,7 @@ private static final class Builder { private String bootstrapServer; private InputStream schemaFile; private Format format; + private String valueDelimiter; private String topicName; private String keyName; private int iterations; @@ -183,6 +190,7 @@ private Builder() { bootstrapServer = "localhost:9092"; schemaFile = null; format = null; + valueDelimiter = ","; topicName = null; keyName = null; iterations = 1000000; @@ -231,7 +239,7 @@ public Format getFormat() { Arguments build() { if (help) { - return new Arguments(true, null, null, null, null, null, 0, -1, null, null); + return new Arguments(true, null, null, null, null, null, null, 0, -1, null, null); } if (quickstart != null) { @@ -254,6 +262,7 @@ Arguments build() { bootstrapServer, schemaFile, format, + valueDelimiter, topicName, keyName, iterations, @@ -352,6 +361,16 @@ private static Format parseFormat(final String formatString) { } } + private static String parseValueDelimiter(final String valueDelimiterString) { + if (valueDelimiterString.length() > 1) { + throw new ArgumentParseException(String.format( + "Invalid value '%s' for valueDelimiter; was expecting single character value", + valueDelimiterString + )); + } + return valueDelimiterString; + } + private static int parseIterations(final String iterationsString) { try { final int result = Integer.valueOf(iterationsString, 10); diff --git a/ksql-examples/src/main/java/io/confluent/ksql/datagen/DelimitedProducer.java b/ksql-examples/src/main/java/io/confluent/ksql/datagen/DelimitedProducer.java index aa3e28dc2e16..27eea40bf26f 100644 --- a/ksql-examples/src/main/java/io/confluent/ksql/datagen/DelimitedProducer.java +++ b/ksql-examples/src/main/java/io/confluent/ksql/datagen/DelimitedProducer.java @@ -17,16 +17,23 @@ import io.confluent.ksql.GenericRow; import io.confluent.ksql.serde.delimited.KsqlDelimitedSerializer; import org.apache.avro.Schema; +import org.apache.commons.csv.CSVFormat; import org.apache.kafka.common.serialization.Serializer; public class DelimitedProducer extends DataGenProducer { + private final CSVFormat csvFormat; + public DelimitedProducer(final String valueDelimiter) { + super(); + this.csvFormat = CSVFormat.newFormat(valueDelimiter.charAt(0)); + } + @Override protected Serializer getSerializer( final Schema avroSchema, final org.apache.kafka.connect.data.Schema kafkaSchema, final String topicName ) { - return new KsqlDelimitedSerializer(kafkaSchema); + return new KsqlDelimitedSerializer(kafkaSchema, csvFormat); } } diff --git a/ksql-examples/src/main/java/io/confluent/ksql/datagen/ProducerFactory.java b/ksql-examples/src/main/java/io/confluent/ksql/datagen/ProducerFactory.java index e85e96b5e08a..9c805c21550d 100644 --- a/ksql-examples/src/main/java/io/confluent/ksql/datagen/ProducerFactory.java +++ b/ksql-examples/src/main/java/io/confluent/ksql/datagen/ProducerFactory.java @@ -20,6 +20,7 @@ class ProducerFactory { DataGenProducer getProducer(final Format format, + final String valueDelimiter, final String schemaRegistryUrl) { switch (format) { case AVRO: @@ -36,7 +37,7 @@ DataGenProducer getProducer(final Format format, return new JsonProducer(); case DELIMITED: - return new DelimitedProducer(); + return new DelimitedProducer(valueDelimiter); default: throw new IllegalArgumentException("Invalid format in '" + format diff --git a/ksql-examples/src/test/java/io/confluent/ksql/datagen/DataGenTest.java b/ksql-examples/src/test/java/io/confluent/ksql/datagen/DataGenTest.java index 858515962577..0ee1fb59c0c3 100644 --- a/ksql-examples/src/test/java/io/confluent/ksql/datagen/DataGenTest.java +++ b/ksql-examples/src/test/java/io/confluent/ksql/datagen/DataGenTest.java @@ -61,4 +61,13 @@ public void shouldThrowOnUnknownQuickStart() throws Exception { "format=avro", "topic=foo"); } + + @Test(expected = IllegalArgumentException.class) + public void valueDelimiterCanOnlyBeSingleCharacter() throws Exception { + DataGen.run( + "schema=./src/main/resources/purchase.avro", + "format=delimited", + "valueDelimiter=@@", + "topic=foo"); + } } \ No newline at end of file diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializer.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializer.java index 77e0da8d2254..831f0f40023e 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializer.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializer.java @@ -39,16 +39,6 @@ public class KsqlDelimitedDeserializer implements Deserializer { private final StructuredLogger recordLogger; private final ProcessingLogContext processingLogContext; - public KsqlDelimitedDeserializer(final Schema schema) { - this.schema = schema; - this.csvFormat = CSVFormat.DEFAULT; - } - - KsqlDelimitedDeserializer(final Schema schema, final CSVFormat csvFormat) { - this.schema = schema; - this.csvFormat = csvFormat; - } - KsqlDelimitedDeserializer( final Schema schema, final CSVFormat csvFormat, diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializer.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializer.java index 95ffe6494c88..7348c86622b1 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializer.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializer.java @@ -28,9 +28,11 @@ public class KsqlDelimitedSerializer implements Serializer { private final Schema schema; + private final CSVFormat csvFormat; - public KsqlDelimitedSerializer(final Schema schema) { + public KsqlDelimitedSerializer(final Schema schema, final CSVFormat csvFormat) { this.schema = schema; + this.csvFormat = csvFormat; } @Override @@ -45,7 +47,7 @@ public byte[] serialize(final String topic, final GenericRow genericRow) { } try { final StringWriter stringWriter = new StringWriter(); - final CSVPrinter csvPrinter = new CSVPrinter(stringWriter, CSVFormat.DEFAULT); + final CSVPrinter csvPrinter = new CSVPrinter(stringWriter, csvFormat); csvPrinter.printRecord(genericRow.getColumns()); final String result = stringWriter.toString(); return result.substring(0, result.length() - 2).getBytes(StandardCharsets.UTF_8); diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java index cdc4f8a2e9b2..32ff61530ac9 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java @@ -41,11 +41,6 @@ public class KsqlDelimitedTopicSerDe extends KsqlTopicSerDe { private final CSVFormat csvFormat; - public KsqlDelimitedTopicSerDe() { - super(DataSource.DataSourceSerDe.DELIMITED); - this.csvFormat = CSVFormat.DEFAULT; - } - public KsqlDelimitedTopicSerDe(final String delimiter) { super(DataSource.DataSourceSerDe.DELIMITED); @@ -66,7 +61,7 @@ public Serde getGenericRowSerde( final ProcessingLogContext processingLogContext) { final Map serdeProps = new HashMap<>(); - final Serializer genericRowSerializer = new KsqlDelimitedSerializer(schema); + final Serializer genericRowSerializer = new KsqlDelimitedSerializer(schema, csvFormat); genericRowSerializer.configure(serdeProps, false); final Deserializer genericRowDeserializer = new KsqlDelimitedDeserializer( diff --git a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java index f5ccd35da8b8..7949d1b209c8 100644 --- a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java +++ b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java @@ -14,6 +14,7 @@ package io.confluent.ksql.serde.delimited; +import static org.easymock.EasyMock.mock; import static org.hamcrest.CoreMatchers.equalTo; import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; @@ -113,7 +114,12 @@ public void shouldDeserializeJsonCorrectlyWithRedundantFields() { public void shouldDeserializeDelimitedCorrectlyWithOtherDelimitersTDF() { final String rowString = "1511897796092\t1\titem_1\t10.0\r\n"; - final KsqlDelimitedDeserializer ksqlJsonDeserializer = new KsqlDelimitedDeserializer(orderSchema, CSVFormat.TDF); + final KsqlDelimitedDeserializer ksqlJsonDeserializer = new KsqlDelimitedDeserializer( + orderSchema, + CSVFormat.TDF, + mock(StructuredLogger.class), + mock(ProcessingLogContext.class) + ); final GenericRow genericRow = ksqlJsonDeserializer.deserialize("", rowString.getBytes()); assertThat(genericRow.getColumns().size(), equalTo(4)); @@ -129,7 +135,11 @@ public void shouldDeserializeDelimitedCorrectlyWithCustomDelimiters() { final String rowString = "1511897796092|1|item_1|10.0\r\n"; final KsqlDelimitedDeserializer ksqlJsonDeserializer = new KsqlDelimitedDeserializer( - orderSchema, CSVFormat.DEFAULT.withDelimiter('|')); + orderSchema, + CSVFormat.DEFAULT.withDelimiter('|'), + mock(StructuredLogger.class), + mock(ProcessingLogContext.class) + ); final GenericRow genericRow = ksqlJsonDeserializer.deserialize("", rowString.getBytes()); assertThat(genericRow.getColumns().size(), equalTo(4)); diff --git a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java index b40b06b12fe0..52a82d28ea76 100644 --- a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java +++ b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java @@ -21,6 +21,8 @@ import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.List; + +import org.apache.commons.csv.CSVFormat; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; import org.junit.Before; @@ -46,18 +48,29 @@ public void before() { public void shouldSerializeRowCorrectly() { final List columns = Arrays.asList(1511897796092L, 1L, "item_1", 10.0); final GenericRow genericRow = new GenericRow(columns); - final KsqlDelimitedSerializer ksqlDelimitedSerializer = new KsqlDelimitedSerializer(orderSchema); + final KsqlDelimitedSerializer ksqlDelimitedSerializer = new KsqlDelimitedSerializer(orderSchema, CSVFormat.DEFAULT); final byte[] bytes = ksqlDelimitedSerializer.serialize("t1", genericRow); final String delimitedString = new String(bytes, StandardCharsets.UTF_8); assertThat("Incorrect serialization.", delimitedString, equalTo("1511897796092,1,item_1,10.0")); } + @Test + public void shouldSerializeRowWithCustomDelimiter() { + final List columns = Arrays.asList(1511897796092L, 1L, "item_1", 10.0); + final GenericRow genericRow = new GenericRow(columns); + final KsqlDelimitedSerializer ksqlDelimitedSerializer = new KsqlDelimitedSerializer(orderSchema, CSVFormat.newFormat('^')); + final byte[] bytes = ksqlDelimitedSerializer.serialize("t1", genericRow); + + final String delimitedString = new String(bytes, StandardCharsets.UTF_8); + assertThat("Incorrect serialization.", delimitedString, equalTo("1511897796092^1^item_1^10.0")); + } + @Test public void shouldSerializeRowWithNull() { final List columns = Arrays.asList(1511897796092L, 1L, "item_1", null); final GenericRow genericRow = new GenericRow(columns); - final KsqlDelimitedSerializer ksqlDelimitedSerializer = new KsqlDelimitedSerializer(orderSchema); + final KsqlDelimitedSerializer ksqlDelimitedSerializer = new KsqlDelimitedSerializer(orderSchema, CSVFormat.DEFAULT); final byte[] bytes = ksqlDelimitedSerializer.serialize("t1", genericRow); final String delimitedString = new String(bytes, StandardCharsets.UTF_8); From 077699de556756d3eb3c920eae5faf7f78d04366 Mon Sep 17 00:00:00 2001 From: Hasnat Ullah Date: Mon, 4 Mar 2019 22:39:01 +0000 Subject: [PATCH 08/46] Fix max line issue --- .../ksql/serde/delimited/KsqlDelimitedTopicSerDe.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java index 3780854defe9..e1a98dd42c78 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java @@ -62,7 +62,9 @@ public Serde getGenericRowSerde( final ProcessingLogContext processingLogContext) { final Map serdeProps = new HashMap<>(); - final Serializer genericRowSerializer = new KsqlDelimitedSerializer(schema, csvFormat); + final Serializer genericRowSerializer = new KsqlDelimitedSerializer( + schema, csvFormat + ); genericRowSerializer.configure(serdeProps, false); final Deserializer genericRowDeserializer = new KsqlDelimitedDeserializer( From ab21e530319490d62bf6c6c1abf3ad54d18e088f Mon Sep 17 00:00:00 2001 From: Hasnat Ullah Date: Mon, 4 Mar 2019 22:59:18 +0000 Subject: [PATCH 09/46] Fix logger classes --- .../ksql/serde/delimited/KsqlDelimitedDeserializer.java | 3 ++- .../ksql/serde/delimited/KsqlDelimitedDeserializerTest.java | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializer.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializer.java index 5cbd103bca8b..f31d777dbd6f 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializer.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializer.java @@ -16,6 +16,7 @@ package io.confluent.ksql.serde.delimited; import io.confluent.ksql.GenericRow; +import io.confluent.ksql.logging.processing.ProcessingLogger; import io.confluent.ksql.serde.util.SerdeProcessingLogMessageFactory; import io.confluent.ksql.util.KsqlException; import java.nio.charset.StandardCharsets; @@ -35,7 +36,7 @@ public class KsqlDelimitedDeserializer implements Deserializer { private final Schema schema; private final CSVFormat csvFormat; - private final StructuredLogger recordLogger; + private final ProcessingLogger recordLogger; KsqlDelimitedDeserializer( final Schema schema, diff --git a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java index f770e6505d06..d205ab67d93d 100644 --- a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java +++ b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java @@ -120,7 +120,7 @@ public void shouldDeserializeDelimitedCorrectlyWithOtherDelimitersTDF() { final KsqlDelimitedDeserializer ksqlJsonDeserializer = new KsqlDelimitedDeserializer( orderSchema, CSVFormat.TDF, - mock(StructuredLogger.class) + mock(ProcessingLogger.class) ); final GenericRow genericRow = ksqlJsonDeserializer.deserialize("", rowString.getBytes()); @@ -139,7 +139,7 @@ public void shouldDeserializeDelimitedCorrectlyWithCustomDelimiters() { final KsqlDelimitedDeserializer ksqlJsonDeserializer = new KsqlDelimitedDeserializer( orderSchema, CSVFormat.DEFAULT.withDelimiter('|'), - mock(StructuredLogger.class) + mock(ProcessingLogger.class) ); final GenericRow genericRow = ksqlJsonDeserializer.deserialize("", rowString.getBytes()); From 7794be0d760734839cf8f3d49d5739bd6b95b006 Mon Sep 17 00:00:00 2001 From: Hasnat Ullah Date: Tue, 5 Mar 2019 08:38:59 +0000 Subject: [PATCH 10/46] Try default delimiter type with custom character --- .../ksql/serde/delimited/KsqlDelimitedSerializerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java index 0a6f6327bc00..2d0df9007ef8 100644 --- a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java +++ b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java @@ -60,7 +60,7 @@ public void shouldSerializeRowCorrectly() { public void shouldSerializeRowWithCustomDelimiter() { final List columns = Arrays.asList(1511897796092L, 1L, "item_1", 10.0); final GenericRow genericRow = new GenericRow(columns); - final KsqlDelimitedSerializer ksqlDelimitedSerializer = new KsqlDelimitedSerializer(orderSchema, CSVFormat.newFormat('^')); + final KsqlDelimitedSerializer ksqlDelimitedSerializer = new KsqlDelimitedSerializer(orderSchema, CSVFormat.DEFAULT.withDelimiter('^')); final byte[] bytes = ksqlDelimitedSerializer.serialize("t1", genericRow); final String delimitedString = new String(bytes, StandardCharsets.UTF_8); From ef4ca5617f1f58f82796decbb2ff28548ec8b625 Mon Sep 17 00:00:00 2001 From: Hasnat Ullah Date: Tue, 5 Mar 2019 19:19:14 +0000 Subject: [PATCH 11/46] Use utf 8 --- .../serde/delimited/KsqlDelimitedDeserializerTest.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java index d205ab67d93d..753f63bfa4c5 100644 --- a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java +++ b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java @@ -123,7 +123,9 @@ public void shouldDeserializeDelimitedCorrectlyWithOtherDelimitersTDF() { mock(ProcessingLogger.class) ); - final GenericRow genericRow = ksqlJsonDeserializer.deserialize("", rowString.getBytes()); + final GenericRow genericRow = ksqlJsonDeserializer.deserialize( + "", + rowString.getBytes(StandardCharsets.UTF_8)); assertThat(genericRow.getColumns().size(), equalTo(4)); assertThat((Long) genericRow.getColumns().get(0), equalTo(1511897796092L)); assertThat((Long) genericRow.getColumns().get(1), equalTo(1L)); @@ -142,7 +144,9 @@ public void shouldDeserializeDelimitedCorrectlyWithCustomDelimiters() { mock(ProcessingLogger.class) ); - final GenericRow genericRow = ksqlJsonDeserializer.deserialize("", rowString.getBytes()); + final GenericRow genericRow = ksqlJsonDeserializer.deserialize( + "", + rowString.getBytes(StandardCharsets.UTF_8)); assertThat(genericRow.getColumns().size(), equalTo(4)); assertThat((Long) genericRow.getColumns().get(0), equalTo(1511897796092L)); assertThat((Long) genericRow.getColumns().get(1), equalTo(1L)); From 2b2491af0501100160a79223fab850126ad9c62c Mon Sep 17 00:00:00 2001 From: Hasnat Ullah Date: Tue, 5 Mar 2019 20:30:02 +0000 Subject: [PATCH 12/46] fix query --- .../query-validation-tests/delimited-value-format.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json b/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json index 689803184369..ce3f5937b137 100644 --- a/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json +++ b/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json @@ -95,7 +95,7 @@ "name": "select delimited value_format with pipe separated values using custom delimiter character", "statements": [ "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='|');", - "CREATE STREAM S2 WITH(value_delimiter='$') AS SELECT * FROM test;" + "CREATE STREAM S2 WITH(value_format='DELIMITED', value_delimiter='$') AS SELECT * FROM test;" ], "inputs": [ {"topic": "test_topic", "key": 0, "value": "0|zero|0", "timestamp": 0}, From d1a64e3719517664df7ce84f25151f8a614931ce Mon Sep 17 00:00:00 2001 From: Hasnat Ullah Date: Tue, 5 Mar 2019 21:55:50 +0000 Subject: [PATCH 13/46] Fix value delimiter WITH argument --- .../src/main/java/io/confluent/ksql/datagen/DataGen.java | 6 +++--- .../test/java/io/confluent/ksql/datagen/DataGenTest.java | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java b/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java index 7c4210fb6112..a288873e5b57 100644 --- a/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java +++ b/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java @@ -93,7 +93,7 @@ private static void usage() { + "schema= " + "[schemaRegistryUrl= (defaults to http://localhost:8081)] " + "format= (case-insensitive; one of 'avro', 'json', or 'delimited') " - + "[valueDelimiter= (used only when format is 'delimited', only " + + "[value_delimiter= (used only when format is 'delimited', only " + "single characters, defaults to ',' )] " + "topic= " + "key= " @@ -160,7 +160,7 @@ private static final class Builder { .put("bootstrap-server", (builder, argVal) -> builder.bootstrapServer = argVal) .put("schema", (builder, argVal) -> builder.schemaFile = toFileInputStream(argVal)) .put("format", (builder, argVal) -> builder.format = parseFormat(argVal)) - .put("valueDelimiter", (builder, argVal) -> builder.valueDelimiter = parseValueDelimiter(argVal)) + .put("value_delimiter", (builder, argVal) -> builder.valueDelimiter = parseValueDelimiter(argVal)) .put("topic", (builder, argVal) -> builder.topicName = argVal) .put("key", (builder, argVal) -> builder.keyName = argVal) .put("iterations", (builder, argVal) -> builder.iterations = parseIterations(argVal)) @@ -365,7 +365,7 @@ private static Format parseFormat(final String formatString) { private static String parseValueDelimiter(final String valueDelimiterString) { if (valueDelimiterString.length() > 1) { throw new ArgumentParseException(String.format( - "Invalid value '%s' for valueDelimiter; was expecting single character value", + "Invalid value '%s' for value_delimiter; was expecting single character value", valueDelimiterString )); } diff --git a/ksql-examples/src/test/java/io/confluent/ksql/datagen/DataGenTest.java b/ksql-examples/src/test/java/io/confluent/ksql/datagen/DataGenTest.java index f4e08a89b2e6..40b2e461778e 100644 --- a/ksql-examples/src/test/java/io/confluent/ksql/datagen/DataGenTest.java +++ b/ksql-examples/src/test/java/io/confluent/ksql/datagen/DataGenTest.java @@ -68,7 +68,7 @@ public void valueDelimiterCanOnlyBeSingleCharacter() throws Exception { DataGen.run( "schema=./src/main/resources/purchase.avro", "format=delimited", - "valueDelimiter=@@", + "value_delimiter=@@", "topic=foo"); } } \ No newline at end of file From 5c58d252142176e1dddb74637fc7141e6942238a Mon Sep 17 00:00:00 2001 From: Hasnat Ullah Date: Tue, 5 Mar 2019 22:19:00 +0000 Subject: [PATCH 14/46] use default with delimiter --- .../main/java/io/confluent/ksql/datagen/DelimitedProducer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ksql-examples/src/main/java/io/confluent/ksql/datagen/DelimitedProducer.java b/ksql-examples/src/main/java/io/confluent/ksql/datagen/DelimitedProducer.java index cda11e8e597e..98507e40632f 100644 --- a/ksql-examples/src/main/java/io/confluent/ksql/datagen/DelimitedProducer.java +++ b/ksql-examples/src/main/java/io/confluent/ksql/datagen/DelimitedProducer.java @@ -26,7 +26,7 @@ public class DelimitedProducer extends DataGenProducer { private final CSVFormat csvFormat; public DelimitedProducer(final String valueDelimiter) { super(); - this.csvFormat = CSVFormat.newFormat(valueDelimiter.charAt(0)); + this.csvFormat = CSVFormat.DEFAULT.withDelimiter(valueDelimiter.charAt(0)); } @Override From 8eecc8dbaf396ed472cf3479e0b02c3ead1929f8 Mon Sep 17 00:00:00 2001 From: Hasnat Ullah Date: Sun, 17 Mar 2019 11:09:27 +0000 Subject: [PATCH 15/46] remove old test --- .../delimited-value-format.json | 23 ++----------------- 1 file changed, 2 insertions(+), 21 deletions(-) diff --git a/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json b/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json index ce3f5937b137..42f565d43ec3 100644 --- a/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json +++ b/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json @@ -34,25 +34,6 @@ "inputs": [], "outputs": [] }, - { - "name": "select delimited value_format with tab separated values using CsvFormat constant", - "statements": [ - "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='TDF');", - "CREATE STREAM S2 as SELECT id, name, value FROM test;" - ], - "inputs": [ - {"topic": "test_topic", "key": 0, "value": "0\tzero\t0", "timestamp": 0}, - {"topic": "test_topic", "key": 0, "value": "0\t100\t100", "timestamp": 0}, - {"topic": "test_topic", "key": 100, "value": "100\t100\t500", "timestamp": 0}, - {"topic": "test_topic", "key": 100, "value": "100\t100\t100", "timestamp": 0} - ], - "outputs": [ - {"topic": "S2", "key": 0, "value": "0,zero,0", "timestamp": 0}, - {"topic": "S2", "key": 0, "value": "0,100,100", "timestamp": 0}, - {"topic": "S2", "key": 100, "value": "100,100,500", "timestamp": 0}, - {"topic": "S2", "key": 100, "value": "100,100,100", "timestamp": 0} - ] - }, { "name": "select delimited value_format with tab separated values using custom delimiter character", "statements": [ @@ -92,10 +73,10 @@ ] }, { - "name": "select delimited value_format with pipe separated values using custom delimiter character", + "name": "Serialize delimited value_format with $", "statements": [ "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='|');", - "CREATE STREAM S2 WITH(value_format='DELIMITED', value_delimiter='$') AS SELECT * FROM test;" + "CREATE STREAM S2 WITH(value_delimiter='$') AS SELECT * FROM test;" ], "inputs": [ {"topic": "test_topic", "key": 0, "value": "0|zero|0", "timestamp": 0}, From 66ab934eecea5f0ccdc2d042dc36cd72595da179 Mon Sep 17 00:00:00 2001 From: Hasnat Ullah Date: Mon, 6 May 2019 10:53:54 +0100 Subject: [PATCH 16/46] style fix --- .../ksql/serde/delimited/KsqlDelimitedTopicSerDe.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java index 230e8c304766..a2a674e79b18 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedTopicSerDe.java @@ -45,7 +45,7 @@ public class KsqlDelimitedTopicSerDe extends KsqlTopicSerDe { private final CSVFormat csvFormat; public KsqlDelimitedTopicSerDe(final String delimiter) { - super(Format.DELIMITED); + super(Format.DELIMITED); if (delimiter.length() == 1) { this.csvFormat = CSVFormat.DEFAULT.withDelimiter(delimiter.charAt(0)); @@ -64,9 +64,7 @@ public Serde getGenericRowSerde( ) { final Map serdeProps = new HashMap<>(); - final Serializer genericRowSerializer = new KsqlDelimitedSerializer( - schema, csvFormat - ); + final Serializer genericRowSerializer = new KsqlDelimitedSerializer(csvFormat); genericRowSerializer.configure(serdeProps, false); final Deserializer genericRowDeserializer = new KsqlDelimitedDeserializer( From 3dce9e9568bd69d9e6937ef8d1da03ba1ba8a99e Mon Sep 17 00:00:00 2001 From: Hasnat Ullah Date: Mon, 6 May 2019 11:12:10 +0100 Subject: [PATCH 17/46] Missing default --- .../src/main/java/io/confluent/ksql/analyzer/Analyzer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java index a019fdfefc6e..5f7fdaa67115 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java @@ -217,7 +217,7 @@ private KsqlTopicSerDe getIntoValueSerde() { return new KsqlJsonTopicSerDe(); case DELIMITED: - return new KsqlDelimitedTopicSerDe(); + return new KsqlDelimitedTopicSerDe(DdlConfig.VALUE_DELIMITER_DEFAULT); default: throw new KsqlException( From 9676843696582642afe58b9b16de9d4bdef2f2dc Mon Sep 17 00:00:00 2001 From: Hasnat Ullah Date: Mon, 6 May 2019 11:35:48 +0100 Subject: [PATCH 18/46] bug fixes --- .../io/confluent/ksql/ddl/commands/RegisterTopicCommand.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/RegisterTopicCommand.java b/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/RegisterTopicCommand.java index ea5eda178bca..fc18cb18d72b 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/RegisterTopicCommand.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/ddl/commands/RegisterTopicCommand.java @@ -106,7 +106,7 @@ private static void enforceTopicProperties(final Map properties) { if (properties.containsKey(DdlConfig.VALUE_DELIMITER_PROPERTY) && ( !properties.containsKey(DdlConfig.VALUE_FORMAT_PROPERTY) || !cleanQuotesAndUpperCase(properties.get(DdlConfig.VALUE_FORMAT_PROPERTY).toString()) - .equals(Format.DELIMITED))) { + .equals(Format.DELIMITED.toString()))) { throw new KsqlException( DdlConfig.VALUE_DELIMITER_PROPERTY + " can only be used with " + DdlConfig.VALUE_FORMAT_PROPERTY + "='" + Format.DELIMITED + "'" From a5b496a00f53d132c9aaa304a043f78b38ae3777 Mon Sep 17 00:00:00 2001 From: Hasnat Ullah Date: Mon, 6 May 2019 12:10:51 +0100 Subject: [PATCH 19/46] more style fixes --- .../java/io/confluent/ksql/datagen/DataGen.java | 15 ++++++++++----- .../confluent/ksql/datagen/DelimitedProducer.java | 3 ++- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java b/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java index bef5a218ad0a..5e2e36073b28 100644 --- a/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java +++ b/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java @@ -95,10 +95,12 @@ private static void usage() { + "[quickstart= (case-insensitive; one of 'orders', 'users', or " + "'pageviews')] " + newLine + "schema= " + newLine - + "[schemaRegistryUrl= (defaults to http://localhost:8081)] " + newLine - + "format= (case-insensitive; one of 'avro', 'json', or 'delimited') " + newLine - + "[value_delimiter= (used only when format is 'delimited', only " - + "single characters, defaults to ',' )] " + newLine + + "[schemaRegistryUrl= " + + "(defaults to http://localhost:8081)] " + newLine + + "format= (case-insensitive; one of 'avro', " + + "'json', or 'delimited') " + newLine + + "[value_delimiter= (used only when format is " + + "'delimited', only single characters, defaults to ',' )] " + newLine + "(defaults to http://localhost:8081)] " + newLine + "format= (case-insensitive; one of 'avro', 'json', or " + "'delimited') " + newLine @@ -126,6 +128,7 @@ public enum Format { AVRO, JSON, DELIMITED } private final String schemaRegistryUrl; private final InputStream propertiesFile; + // CHECKSTYLE_RULES.OFF: ParameterNumberCheck Arguments( final boolean help, final String bootstrapServer, @@ -139,6 +142,7 @@ public enum Format { AVRO, JSON, DELIMITED } final String schemaRegistryUrl, final InputStream propertiesFile ) { + // CHECKSTYLE_RULES.ON: ParameterNumberCheck this.help = help; this.bootstrapServer = bootstrapServer; this.schemaFile = schemaFile; @@ -167,7 +171,8 @@ private static final class Builder { .put("bootstrap-server", (builder, argVal) -> builder.bootstrapServer = argVal) .put("schema", (builder, argVal) -> builder.schemaFile = toFileInputStream(argVal)) .put("format", (builder, argVal) -> builder.format = parseFormat(argVal)) - .put("value_delimiter", (builder, argVal) -> builder.valueDelimiter = parseValueDelimiter(argVal)) + .put("value_delimiter", + (builder, argVal) -> builder.valueDelimiter = parseValueDelimiter(argVal)) .put("topic", (builder, argVal) -> builder.topicName = argVal) .put("key", (builder, argVal) -> builder.keyName = argVal) .put("iterations", (builder, argVal) -> builder.iterations = parseIterations(argVal)) diff --git a/ksql-examples/src/main/java/io/confluent/ksql/datagen/DelimitedProducer.java b/ksql-examples/src/main/java/io/confluent/ksql/datagen/DelimitedProducer.java index 98507e40632f..90540e3bae10 100644 --- a/ksql-examples/src/main/java/io/confluent/ksql/datagen/DelimitedProducer.java +++ b/ksql-examples/src/main/java/io/confluent/ksql/datagen/DelimitedProducer.java @@ -24,6 +24,7 @@ public class DelimitedProducer extends DataGenProducer { private final CSVFormat csvFormat; + public DelimitedProducer(final String valueDelimiter) { super(); this.csvFormat = CSVFormat.DEFAULT.withDelimiter(valueDelimiter.charAt(0)); @@ -35,6 +36,6 @@ protected Serializer getSerializer( final org.apache.kafka.connect.data.Schema kafkaSchema, final String topicName ) { - return new KsqlDelimitedSerializer(kafkaSchema, csvFormat); + return new KsqlDelimitedSerializer(csvFormat); } } From b1075d439218bec797dca38437a70209c3650373 Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Fri, 20 Sep 2019 14:29:29 +0100 Subject: [PATCH 20/46] fix: Get the PR to work as the original author intended against current master --- .../properties/with/CommonCreateConfigs.java | 12 ++- .../io/confluent/ksql/serde/FormatInfo.java | 35 +++++-- .../io/confluent/ksql/serde/KeyFormat.java | 3 +- .../confluent/ksql/serde/FormatInfoTest.java | 2 +- .../confluent/ksql/serde/ValueFormatTest.java | 8 +- .../io/confluent/ksql/analyzer/Analyzer.java | 3 +- .../ksql/engine/InsertValuesExecutor.java | 1 + .../io/confluent/ksql/engine/KsqlEngine.java | 1 + .../io/confluent/ksql/topic/TopicFactory.java | 3 +- .../confluent/ksql/analyzer/AnalyzerTest.java | 2 +- .../io/confluent/ksql/datagen/DataGen.java | 23 ++++- .../confluent/ksql/datagen/DataGenTest.java | 19 ++-- .../ksql/test/tools/TestCaseBuilderUtil.java | 5 +- .../delimited-value-format.json | 45 +++++---- .../properties/with/WithClauseProperties.java | 23 +++++ .../ksql/serde/KsqlSerdeFactories.java | 2 +- .../delimited/KsqlDelimitedSerdeFactory.java | 14 ++- .../delimited/KsqlDelimitedSerializer.java | 4 - .../KsqlDelimitedDeserializerTest.java | 97 +++++++++---------- .../KsqlDelimitedSerdeFactoryTest.java | 3 +- .../KsqlDelimitedSerializerTest.java | 77 ++++++++------- 21 files changed, 231 insertions(+), 151 deletions(-) rename {ksql-engine => ksql-functional-tests}/src/test/resources/query-validation-tests/delimited-value-format.json (75%) diff --git a/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java b/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java index 2d95740ffb77..37563ce19c3e 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java +++ b/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java @@ -41,9 +41,6 @@ public final class CommonCreateConfigs { public static final String WRAP_SINGLE_VALUE = "WRAP_SINGLE_VALUE"; public static final String VALUE_DELIMITER_PROPERTY = "VALUE_DELIMITER"; - public static final String AVRO_SCHEMA_FILE = "AVROSCHEMAFILE"; - public static final String AVRO_SCHEMA = "AVROSCHEMA"; - public static final String VALUE_DELIMITER_DEFAULT = ","; static void addToConfigDef( final ConfigDef configDef, @@ -122,7 +119,14 @@ static void addToConfigDef( ConfigDef.Type.STRING, null, Importance.LOW, - "The fully qualified name of the Avro schema to use"); + "The fully qualified name of the Avro schema to use" + ) + .define( + VALUE_DELIMITER_PROPERTY, + ConfigDef.Type.STRING, + null, + Importance.LOW, + "The delimiter to use when VALUE_FORMAT='DELIMITED'."); } private CommonCreateConfigs() { diff --git a/ksql-common/src/main/java/io/confluent/ksql/serde/FormatInfo.java b/ksql-common/src/main/java/io/confluent/ksql/serde/FormatInfo.java index f5c4a17ee576..d330d362229a 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/serde/FormatInfo.java +++ b/ksql-common/src/main/java/io/confluent/ksql/serde/FormatInfo.java @@ -28,21 +28,31 @@ public final class FormatInfo { private final Format format; private final Optional avroFullSchemaName; + private final Optional delimiter; public static FormatInfo of(final Format format) { - return FormatInfo.of(format, Optional.empty()); + return FormatInfo.of(format, Optional.empty(), Optional.empty()); } public static FormatInfo of( final Format format, final Optional avroFullSchemaName ) { - return new FormatInfo(format, avroFullSchemaName); + return FormatInfo.of(format, avroFullSchemaName, Optional.empty()); + } + + public static FormatInfo of( + final Format format, + final Optional avroFullSchemaName, + final Optional delimiter + ) { + return new FormatInfo(format, avroFullSchemaName, delimiter); } private FormatInfo( final Format format, - final Optional avroFullSchemaName + final Optional avroFullSchemaName, + final Optional delimiter ) { this.format = Objects.requireNonNull(format, "format"); this.avroFullSchemaName = Objects.requireNonNull(avroFullSchemaName, "avroFullSchemaName"); @@ -52,8 +62,15 @@ private FormatInfo( } if (avroFullSchemaName.map(name -> name.trim().isEmpty()).orElse(false)) { - throw new KsqlException("Schema name can not be empty"); + throw new KsqlException("Schema name cannot be empty"); } + + this.delimiter = Objects.requireNonNull(delimiter, "delimiter"); + + if (format != Format.DELIMITED && delimiter.isPresent()) { + throw new KsqlException("Delimeter only supported with DELIMITED format"); + } + } public Format getFormat() { @@ -64,6 +81,10 @@ public Optional getAvroFullSchemaName() { return avroFullSchemaName; } + public Optional getDelimiter() { + return delimiter; + } + @Override public boolean equals(final Object o) { if (this == o) { @@ -74,12 +95,13 @@ public boolean equals(final Object o) { } final FormatInfo that = (FormatInfo) o; return format == that.format - && Objects.equals(avroFullSchemaName, that.avroFullSchemaName); + && Objects.equals(avroFullSchemaName, that.avroFullSchemaName) + && Objects.equals(delimiter, that.delimiter); } @Override public int hashCode() { - return Objects.hash(format, avroFullSchemaName); + return Objects.hash(format, avroFullSchemaName, delimiter); } @Override @@ -87,6 +109,7 @@ public String toString() { return "FormatInfo{" + "format=" + format + ", avroFullSchemaName=" + avroFullSchemaName + + ", delimiter=" + delimiter + '}'; } } diff --git a/ksql-common/src/main/java/io/confluent/ksql/serde/KeyFormat.java b/ksql-common/src/main/java/io/confluent/ksql/serde/KeyFormat.java index da2636d7767a..8e7dc92db60d 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/serde/KeyFormat.java +++ b/ksql-common/src/main/java/io/confluent/ksql/serde/KeyFormat.java @@ -37,10 +37,11 @@ public static KeyFormat nonWindowed(final FormatInfo format) { public static KeyFormat windowed( final Format format, final Optional avroSchemaName, + final Optional delimiter, final WindowInfo windowInfo ) { return new KeyFormat( - FormatInfo.of(format, avroSchemaName), + FormatInfo.of(format, avroSchemaName, delimiter), Optional.of(windowInfo) ); } diff --git a/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java b/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java index 4f7956a635ff..cdebc74980c6 100644 --- a/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java +++ b/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java @@ -84,7 +84,7 @@ public void shouldThrowOnNonAvroWithAvroSchemName() { public void shouldThrowOnEmptyAvroSchemaName() { // Then: expectedException.expect(KsqlException.class); - expectedException.expectMessage("Schema name can not be empty"); + expectedException.expectMessage("Schema name cannot be empty"); // When: FormatInfo.of(Format.AVRO, Optional.of("")); diff --git a/ksql-common/src/test/java/io/confluent/ksql/serde/ValueFormatTest.java b/ksql-common/src/test/java/io/confluent/ksql/serde/ValueFormatTest.java index 2b582b17f6cd..28c9410625b2 100644 --- a/ksql-common/src/test/java/io/confluent/ksql/serde/ValueFormatTest.java +++ b/ksql-common/src/test/java/io/confluent/ksql/serde/ValueFormatTest.java @@ -28,7 +28,11 @@ public class ValueFormatTest { - private static final FormatInfo FORMAT_INFO = FormatInfo.of(AVRO, Optional.of("something")); + private static final FormatInfo FORMAT_INFO = + FormatInfo.of( + AVRO, Optional.of("something"), + Optional.empty() + ); @Test public void shouldThrowNPEs() { @@ -44,7 +48,7 @@ public void shouldImplementEquals() { ValueFormat.of(FORMAT_INFO) ) .addEqualityGroup( - ValueFormat.of(FormatInfo.of(JSON, Optional.empty())) + ValueFormat.of(FormatInfo.of(JSON, Optional.empty(), Optional.empty())) ) .testEquals(); } diff --git a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java index 52278fa96c08..42cae762e087 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java @@ -176,7 +176,8 @@ private void analyzeNonStdOutSink(final Sink sink) { final ValueFormat valueFormat = ValueFormat.of(FormatInfo.of( getValueFormat(sink), - sink.getProperties().getValueAvroSchemaName() + sink.getProperties().getValueAvroSchemaName(), + sink.getProperties().getValueDelimiter() )); final KsqlTopic intoKsqlTopic = new KsqlTopic( diff --git a/ksql-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java b/ksql-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java index e36c14f0e668..f731200ed0da 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java @@ -137,6 +137,7 @@ public void execute( final ServiceContext serviceContext ) { final InsertValues insertValues = statement.getStatement(); + final KsqlConfig config = statement.getConfig() .cloneWithPropertyOverwrite(statement.getOverrides()); diff --git a/ksql-engine/src/main/java/io/confluent/ksql/engine/KsqlEngine.java b/ksql-engine/src/main/java/io/confluent/ksql/engine/KsqlEngine.java index a4bbb6eff6a8..98eb1302a22f 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/engine/KsqlEngine.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/engine/KsqlEngine.java @@ -168,6 +168,7 @@ public ExecuteResult execute( final ServiceContext serviceContext, final ConfiguredStatement statement ) { + final ExecuteResult result = EngineExecutor .create(primaryContext, serviceContext, statement.getConfig(), statement.getOverrides()) .execute(statement); diff --git a/ksql-engine/src/main/java/io/confluent/ksql/topic/TopicFactory.java b/ksql-engine/src/main/java/io/confluent/ksql/topic/TopicFactory.java index d23f67ab05b4..c4e36b394c1d 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/topic/TopicFactory.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/topic/TopicFactory.java @@ -45,7 +45,8 @@ public static KsqlTopic create(final CreateSourceProperties properties) { final ValueFormat valueFormat = ValueFormat.of(FormatInfo.of( properties.getValueFormat(), - properties.getValueAvroSchemaName() + properties.getValueAvroSchemaName(), + properties.getValueDelimiter() )); return new KsqlTopic( diff --git a/ksql-engine/src/test/java/io/confluent/ksql/analyzer/AnalyzerTest.java b/ksql-engine/src/test/java/io/confluent/ksql/analyzer/AnalyzerTest.java index e4143e430416..089608d6d8ee 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/analyzer/AnalyzerTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/analyzer/AnalyzerTest.java @@ -409,7 +409,7 @@ public void shouldFailIfExplicitNamespaceIsProvidedButEmpty() { final Analyzer analyzer = new Analyzer(jsonMetaStore, "", DEFAULT_SERDE_OPTIONS); expectedException.expect(KsqlException.class); - expectedException.expectMessage("Schema name can not be empty"); + expectedException.expectMessage("Schema name cannot be empty"); analyzer.analyze(query, Optional.of(createStreamAsSelect.getSink())); } diff --git a/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java b/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java index f4efe4347b94..19a3785cc32a 100644 --- a/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java +++ b/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java @@ -166,7 +166,7 @@ static class Arguments { private final Supplier schemaFile; private final Format keyFormat; private final Format valueFormat; - private final String valueDelimiter; + private final Character valueDelimiter; private final String topicName; private final String keyName; private final int iterations; @@ -184,9 +184,9 @@ static class Arguments { final Supplier schemaFile, final Format keyFormat, final Format valueFormat, + final Character valueDelimiter, final String topicName, final String keyName, - final String valueDelimiter, final int iterations, final long maxInterval, final String schemaRegistryUrl, @@ -253,7 +253,7 @@ private static final class Builder { private Supplier schemaFile; private Format keyFormat; private Format valueFormat; - private String valueDelimiter; + private char valueDelimiter; private String topicName; private String keyName; private int iterations; @@ -271,7 +271,7 @@ private Builder() { schemaFile = null; keyFormat = Format.KAFKA; valueFormat = null; - valueDelimiter = ","; + valueDelimiter = ','; topicName = null; keyName = null; iterations = -1; @@ -334,6 +334,7 @@ Arguments build() { null, null, null, + null, 0, -1, null, @@ -471,6 +472,20 @@ private static Format parseFormat(final String formatString) { } } + private static Character parseValueDelimiter(final String valueDelimiterString) { + if (valueDelimiterString == null) { + return null; + } else { + if (valueDelimiterString.length() != 1) { + throw new ArgumentParseException(String.format( + "Invalid value_delimiter; was expecting a single character, got '%s'", + valueDelimiterString + )); + } + return valueDelimiterString.charAt(0); + } + } + private static int parseNumThreads(final String numThreadsString) { try { final int result = Integer.valueOf(numThreadsString, 10); diff --git a/ksql-examples/src/test/java/io/confluent/ksql/datagen/DataGenTest.java b/ksql-examples/src/test/java/io/confluent/ksql/datagen/DataGenTest.java index 7ddd697392f6..944a5cef4b50 100644 --- a/ksql-examples/src/test/java/io/confluent/ksql/datagen/DataGenTest.java +++ b/ksql-examples/src/test/java/io/confluent/ksql/datagen/DataGenTest.java @@ -1,4 +1,4 @@ -\/* +/* * Copyright 2018 Confluent Inc. * * Licensed under the Confluent Community License (the "License"); you may not use @@ -19,11 +19,13 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsString; +import io.confluent.ksql.datagen.DataGen; import io.confluent.ksql.util.KsqlConfig; import java.util.Properties; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import scala.reflect.internal.Trees.Throw; public class DataGenTest { @Rule @@ -67,14 +69,6 @@ public void shouldThrowOnUnknownQuickStart() throws Throwable { "topic=foo"); } - @Test(expected = IllegalArgumentException.class) - public void valueDelimiterCanOnlyBeSingleCharacter() throws Exception { - DataGen.run( - "schema=./src/main/resources/purchase.avro", - "format=delimited", - "value_delimiter=@@", - "topic=foo"); - } @Test public void shouldPassSchemaRegistryUrl() throws Exception { final DataGen.Arguments args = new DataGen.Arguments( @@ -83,9 +77,9 @@ public void shouldPassSchemaRegistryUrl() throws Exception { null, null, null, + null, "topic", "key", - null, 0, 0L, "srUrl", @@ -99,10 +93,11 @@ public void shouldPassSchemaRegistryUrl() throws Exception { assertThat(props.getProperty(KsqlConfig.SCHEMA_REGISTRY_URL_PROPERTY), equalTo("srUrl")); } - @Test(expected = IllegalArgumentException.class) - public void valueDelimiterCanOnlyBeSingleCharacter() throws Exception { + @Test(expected = DataGen.Arguments.ArgumentParseException.class) + public void valueDelimiterCanOnlyBeSingleCharacter() throws Throwable { DataGen.run( "schema=./src/main/resources/purchase.avro", + "key=id", "format=delimited", "value_delimiter=@@", "topic=foo"); diff --git a/ksql-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestCaseBuilderUtil.java b/ksql-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestCaseBuilderUtil.java index 69cda2502732..e26c1daef9fa 100644 --- a/ksql-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestCaseBuilderUtil.java +++ b/ksql-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestCaseBuilderUtil.java @@ -137,7 +137,7 @@ public static Map getTopicsByName( .map(recordNode -> new Topic( recordNode.topicName(), Optional.empty(), - getKeySedeSupplier(recordNode.getWindow()), + getKeySerdeSupplier(recordNode.getWindow()), defaultValueSerdeSupplier, 4, 1, @@ -222,7 +222,7 @@ private static Topic createTopicFromStatement( } } - private static SerdeSupplier getKeySedeSupplier(final Optional windowDataInfo) { + private static SerdeSupplier getKeySerdeSupplier(final Optional windowDataInfo) { if (windowDataInfo.isPresent()) { final WindowData windowData = windowDataInfo.get(); final WindowType windowType = WindowType.of((windowData.type == Type.SESSION) @@ -231,6 +231,7 @@ private static SerdeSupplier getKeySedeSupplier(final Optional wi final KeyFormat windowKeyFormat = KeyFormat.windowed( Format.KAFKA, Optional.empty(), + Optional.empty(), WindowInfo.of( windowType, windowType == WindowType.SESSION diff --git a/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json b/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json similarity index 75% rename from ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json rename to ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json index 42f565d43ec3..ab2585d9489b 100644 --- a/ksql-engine/src/test/resources/query-validation-tests/delimited-value-format.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json @@ -9,6 +9,12 @@ "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED');", "CREATE STREAM S2 as SELECT id, name, value FROM test;" ], + "topics": [ + { + "name": "test_topic", + "format": "DELIMITED" + } + ], "inputs": [ {"topic": "test_topic", "key": 0, "value": "0,zero,0", "timestamp": 0}, {"topic": "test_topic", "key": 0, "value": "0,100,100", "timestamp": 0}, @@ -27,38 +33,31 @@ "statements": [ "CREATE STREAM TEST WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='<~>');" ], + "topics": [ + { + "name": "test_topic", + "format": "DELIMITED" + } + ], "expectedException": { "type": "io.confluent.ksql.util.KsqlException", - "message": "Only single characters are supported for VALUE_DELIMITER." + "message": "Error in WITH clause property 'VALUE_DELIMITER': Delimiter must be a single character.\nExample valid value: ';'" }, "inputs": [], "outputs": [] }, - { - "name": "select delimited value_format with tab separated values using custom delimiter character", - "statements": [ - "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='\t');", - "CREATE STREAM S2 as SELECT id, name, value FROM test;" - ], - "inputs": [ - {"topic": "test_topic", "key": 0, "value": "0\tzero\t0", "timestamp": 0}, - {"topic": "test_topic", "key": 0, "value": "0\t100\t100", "timestamp": 0}, - {"topic": "test_topic", "key": 100, "value": "100\t100\t500", "timestamp": 0}, - {"topic": "test_topic", "key": 100, "value": "100\t100\t100", "timestamp": 0} - ], - "outputs": [ - {"topic": "S2", "key": 0, "value": "0,zero,0", "timestamp": 0}, - {"topic": "S2", "key": 0, "value": "0,100,100", "timestamp": 0}, - {"topic": "S2", "key": 100, "value": "100,100,500", "timestamp": 0}, - {"topic": "S2", "key": 100, "value": "100,100,100", "timestamp": 0} - ] - }, { "name": "select delimited value_format with pipe separated values using custom delimiter character", "statements": [ "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='|');", "CREATE STREAM S2 as SELECT id, name, value FROM test;" ], + "topics": [ + { + "name": "test_topic", + "format": "DELIMITED" + } + ], "inputs": [ {"topic": "test_topic", "key": 0, "value": "0|zero|0", "timestamp": 0}, {"topic": "test_topic", "key": 0, "value": "0|100|100", "timestamp": 0}, @@ -78,6 +77,12 @@ "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='|');", "CREATE STREAM S2 WITH(value_delimiter='$') AS SELECT * FROM test;" ], + "topics": [ + { + "name": "test_topic", + "format": "DELIMITED" + } + ], "inputs": [ {"topic": "test_topic", "key": 0, "value": "0|zero|0", "timestamp": 0}, {"topic": "test_topic", "key": 0, "value": "0|100|100", "timestamp": 0}, diff --git a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java index 43c2d344fa34..d580554d747a 100644 --- a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java +++ b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java @@ -20,12 +20,14 @@ import com.google.common.collect.Sets.SetView; import com.google.errorprone.annotations.Immutable; import io.confluent.ksql.execution.expression.tree.Literal; +import io.confluent.ksql.properties.with.CommonCreateConfigs; import io.confluent.ksql.properties.with.ConfigMetaData; import io.confluent.ksql.util.KsqlException; import java.time.format.DateTimeFormatter; import java.util.HashMap; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; import org.apache.kafka.common.config.AbstractConfig; @@ -117,4 +119,25 @@ private static void throwOnUnknownProperty( + String.join(",", onlyInProvided)); } } + + public Optional getValueDelimiter() { + final String providedValueDelimiter = getString(CommonCreateConfigs.VALUE_DELIMITER_PROPERTY); + if (providedValueDelimiter != null && providedValueDelimiter.equalsIgnoreCase("$")) { + System.out.println("foo"); + } + if (providedValueDelimiter != null) { + if (providedValueDelimiter.length() != 1) { + throw new KsqlException("Error in WITH clause property '" + + CommonCreateConfigs.VALUE_DELIMITER_PROPERTY + + "': Delimiter must be a single character." + + System.lineSeparator() + + "Example valid value: ';'" + ); + } + return Optional.of(providedValueDelimiter.charAt(0)); + } else { + return Optional.empty(); + } + } + } diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/KsqlSerdeFactories.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/KsqlSerdeFactories.java index d680c550dd2e..624b461d7c27 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/KsqlSerdeFactories.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/KsqlSerdeFactories.java @@ -71,7 +71,7 @@ static KsqlSerdeFactory create(final FormatInfo format) { return new KsqlJsonSerdeFactory(); case DELIMITED: - return new KsqlDelimitedSerdeFactory(DdlConfig.VALUE_DELIMITER_DEFAULT); + return new KsqlDelimitedSerdeFactory(format.getDelimiter()); case KAFKA: return new KafkaSerdeFactory(); diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java index 2e72cc84b495..1b69438d6150 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java @@ -24,7 +24,9 @@ import io.confluent.ksql.util.DecimalUtil; import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.KsqlException; +import java.util.Optional; import java.util.function.Supplier; +import org.apache.commons.csv.CSVFormat; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.connect.data.ConnectSchema; @@ -35,6 +37,14 @@ @Immutable public class KsqlDelimitedSerdeFactory implements KsqlSerdeFactory { + public static final char DEFAULT_DELIMITER = ','; + + private final CSVFormat csvFormat; + + public KsqlDelimitedSerdeFactory(final Optional delimiter) { + this.csvFormat = CSVFormat.DEFAULT.withDelimiter(delimiter.orElse(DEFAULT_DELIMITER)); + } + @Override public void validate(final PersistenceSchema schema) { final ConnectSchema connectSchema = schema.serializedSchema(); @@ -54,8 +64,8 @@ public Serde createSerde( validate(schema); return Serdes.serdeFrom( - new KsqlDelimitedSerializer(), - new KsqlDelimitedDeserializer(schema) + new KsqlDelimitedSerializer(csvFormat), + new KsqlDelimitedDeserializer(schema, csvFormat) ); } diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializer.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializer.java index 60272c99184f..d38040a43936 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializer.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializer.java @@ -36,10 +36,6 @@ public class KsqlDelimitedSerializer implements Serializer { private final CSVFormat csvFormat; - public KsqlDelimitedSerializer() { - this.csvFormat = CSVFormat.DEFAULT; - } - public KsqlDelimitedSerializer(final CSVFormat csvFormat) { this.csvFormat = csvFormat; } diff --git a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java index e24b58b9e626..ad380ab5cb20 100644 --- a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java +++ b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java @@ -15,8 +15,6 @@ package io.confluent.ksql.serde.delimited; -import static org.easymock.EasyMock.mock; -import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; @@ -29,8 +27,6 @@ import java.math.BigDecimal; import java.nio.charset.StandardCharsets; import org.apache.commons.csv.CSVFormat; -import java.util.Collections; -import java.util.Optional; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.connect.data.ConnectSchema; import org.apache.kafka.connect.data.Schema; @@ -144,7 +140,7 @@ public void shouldThrowIfTopLevelNotStruct() { expectedException.expectMessage("DELIMITED expects all top level schemas to be STRUCTs"); // When: - new KsqlDelimitedDeserializer(schema); + new KsqlDelimitedDeserializer(schema, CSVFormat.DEFAULT.withDelimiter(',')); } @Test @@ -156,7 +152,8 @@ public void shouldDeserializedTopLevelPrimitiveTypeIfSchemaHasOnlySingleField() .build() ); - final KsqlDelimitedDeserializer deserializer = new KsqlDelimitedDeserializer(schema); + final KsqlDelimitedDeserializer deserializer = + createDeserializer(schema); final byte[] bytes = "10".getBytes(StandardCharsets.UTF_8); @@ -175,7 +172,8 @@ public void shouldDeserializeDecimal() { .field("cost", DecimalUtil.builder(4, 2)) .build() ); - final KsqlDelimitedDeserializer deserializer = new KsqlDelimitedDeserializer(schema); + final KsqlDelimitedDeserializer deserializer = + createDeserializer(schema); final byte[] bytes = "01.12".getBytes(StandardCharsets.UTF_8); @@ -194,7 +192,8 @@ public void shouldDeserializeDecimalWithoutLeadingZeros() { .field("cost", DecimalUtil.builder(4, 2)) .build() ); - final KsqlDelimitedDeserializer deserializer = new KsqlDelimitedDeserializer(schema); + final KsqlDelimitedDeserializer deserializer = + createDeserializer(schema); final byte[] bytes = "1.12".getBytes(StandardCharsets.UTF_8); @@ -205,6 +204,35 @@ public void shouldDeserializeDecimalWithoutLeadingZeros() { assertThat(result.get("cost"), is(new BigDecimal("01.12"))); } + @Test + public void shouldDeserializeDelimitedCorrectlyWithTabDelimiter() { + shouldDeserializeDelimitedCorrectlyWithNonDefaultDelimiter('\t'); + } + + @Test + public void shouldDeserializeDelimitedCorrectlyWithBarDelimiter() { + shouldDeserializeDelimitedCorrectlyWithNonDefaultDelimiter('|'); + } + + private void shouldDeserializeDelimitedCorrectlyWithNonDefaultDelimiter(char delimiter) { + // Given: + final byte[] bytes = "1511897796092\t1\titem_1\t10.0\t10.10\r\n".getBytes(StandardCharsets.UTF_8); + + final KsqlDelimitedDeserializer deserializer = + new KsqlDelimitedDeserializer(ORDER_SCHEMA, CSVFormat.DEFAULT.withDelimiter('\t')); + + // When: + final Struct struct = deserializer.deserialize("", bytes); + + // Then: + assertThat(struct.schema(), is(ORDER_SCHEMA.serializedSchema())); + assertThat(struct.get("ORDERTIME"), is(1511897796092L)); + assertThat(struct.get("ORDERID"), is(1L)); + assertThat(struct.get("ITEMID"), is("item_1")); + assertThat(struct.get("ORDERUNITS"), is(10.0)); + assertThat(struct.get("COST"), is(new BigDecimal("10.10"))); + } + @Test public void shouldThrowOnDeserializedTopLevelPrimitiveWhenSchemaHasMoreThanOneField() { // Given: @@ -215,7 +243,8 @@ public void shouldThrowOnDeserializedTopLevelPrimitiveWhenSchemaHasMoreThanOneFi .build() ); - final KsqlDelimitedDeserializer deserializer = new KsqlDelimitedDeserializer(schema); + final KsqlDelimitedDeserializer deserializer = + createDeserializer(schema); final byte[] bytes = "10".getBytes(StandardCharsets.UTF_8); @@ -229,7 +258,6 @@ public void shouldThrowOnDeserializedTopLevelPrimitiveWhenSchemaHasMoreThanOneFi deserializer.deserialize("", bytes); } -<<<<<<< HEAD @Test public void shouldThrowOnArrayTypes() { // Given: @@ -247,7 +275,7 @@ public void shouldThrowOnArrayTypes() { expectedException.expectMessage("DELIMITED does not support type: ARRAY, field: ids"); // When: - new KsqlDelimitedDeserializer(schema); + createDeserializer(schema); } @Test @@ -267,7 +295,7 @@ public void shouldThrowOnMapTypes() { expectedException.expectMessage("DELIMITED does not support type: MAP, field: ids"); // When: - new KsqlDelimitedDeserializer(schema); + createDeserializer(schema); } @Test @@ -288,52 +316,17 @@ public void shouldThrowOnStructTypes() { expectedException.expectMessage("DELIMITED does not support type: STRUCT, field: ids"); // When: - new KsqlDelimitedDeserializer(schema); + createDeserializer(schema); } - @Test - public void shouldDeserializeDelimitedCorrectlyWithOtherDelimitersTDF() { - final String rowString = "1511897796092\t1\titem_1\t10.0\r\n"; - final KsqlDelimitedDeserializer ksqlJsonDeserializer = new KsqlDelimitedDeserializer( - ORDER_SCHEMA, - CSVFormat.TDF, - mock(ProcessingLogger.class) - ); - - final GenericRow genericRow = ksqlJsonDeserializer.deserialize( - "", - rowString.getBytes(StandardCharsets.UTF_8)); - assertThat(genericRow.getColumns().size(), equalTo(4)); - assertThat((Long) genericRow.getColumns().get(0), equalTo(1511897796092L)); - assertThat((Long) genericRow.getColumns().get(1), equalTo(1L)); - assertThat((String) genericRow.getColumns().get(2), equalTo("item_1")); - assertThat((Double) genericRow.getColumns().get(3), equalTo(10.0)); + private static PersistenceSchema persistenceSchema(final Schema connectSchema) { + return PersistenceSchema.from((ConnectSchema) connectSchema, false); } - - @Test - public void shouldDeserializeDelimitedCorrectlyWithCustomDelimiters() { - final String rowString = "1511897796092|1|item_1|10.0\r\n"; - - final KsqlDelimitedDeserializer ksqlJsonDeserializer = new KsqlDelimitedDeserializer( - ORDER_SCHEMA, - CSVFormat.DEFAULT.withDelimiter('|'), - mock(ProcessingLogger.class) - ); - - final GenericRow genericRow = ksqlJsonDeserializer.deserialize( - "", - rowString.getBytes(StandardCharsets.UTF_8)); - assertThat(genericRow.getColumns().size(), equalTo(4)); - assertThat((Long) genericRow.getColumns().get(0), equalTo(1511897796092L)); - assertThat((Long) genericRow.getColumns().get(1), equalTo(1L)); - assertThat((String) genericRow.getColumns().get(2), equalTo("item_1")); - assertThat((Double) genericRow.getColumns().get(3), equalTo(10.0)); + private static KsqlDelimitedDeserializer createDeserializer(PersistenceSchema schema) { + return new KsqlDelimitedDeserializer(schema, CSVFormat.DEFAULT.withDelimiter(',')); } - private static PersistenceSchema persistenceSchema(final Schema connectSchema) { - return PersistenceSchema.from((ConnectSchema) connectSchema, false); - } } diff --git a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactoryTest.java b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactoryTest.java index 8218d6ccc1a2..5563f9c8ead0 100644 --- a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactoryTest.java +++ b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactoryTest.java @@ -22,6 +22,7 @@ import io.confluent.ksql.schema.ksql.types.SqlTypes; import io.confluent.ksql.serde.SerdeOption; import io.confluent.ksql.util.KsqlException; +import java.util.Optional; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -36,7 +37,7 @@ public class KsqlDelimitedSerdeFactoryTest { @Before public void setUp() { - factory = new KsqlDelimitedSerdeFactory(); + factory = new KsqlDelimitedSerdeFactory(Optional.of(',')); } @Test diff --git a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java index f8d9ea3093e2..bab40f0851df 100644 --- a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java +++ b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java @@ -23,20 +23,12 @@ import io.confluent.ksql.util.DecimalUtil; import java.math.BigDecimal; import java.nio.charset.StandardCharsets; -<<<<<<< HEAD import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; -======= -import java.util.Arrays; -import java.util.List; - import org.apache.commons.csv.CSVFormat; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; ->>>>>>> a5b496a00f53d132c9aaa304a043f78b38ae3777 import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -59,7 +51,7 @@ public class KsqlDelimitedSerializerTest { @Before public void before() { - serializer = new KsqlDelimitedSerializer(); + serializer = new KsqlDelimitedSerializer(CSVFormat.DEFAULT.withDelimiter(',')); } @Test @@ -89,17 +81,6 @@ public void shouldSerializeRowCorrectly() { assertThat(delimitedString, equalTo("1511897796092,1,item_1,10.0")); } - @Test - public void shouldSerializeRowWithCustomDelimiter() { - - final GenericRow genericRow = new GenericRow(Arrays.asList(1511897796092L, 1L, "item_1", 10.0)); - final KsqlDelimitedSerializer ksqlDelimitedSerializer = new KsqlDelimitedSerializer(CSVFormat.DEFAULT.withDelimiter('^')); - final byte[] bytes = ksqlDelimitedSerializer.serialize("t1", genericRow); - - final String delimitedString = new String(bytes, StandardCharsets.UTF_8); - assertThat("Incorrect serialization.", delimitedString, equalTo("1511897796092^1^item_1^10.0")); - } - @Test public void shouldSerializeRowWithNull() { // Given: @@ -124,7 +105,7 @@ public void shouldSerializedTopLevelPrimitiveIfValueHasOneField() { .field("id", Schema.OPTIONAL_INT64_SCHEMA) .build(); - final Serializer serializer = new KsqlDelimitedSerializer(); + final Serializer serializer = createSerializer(); final Struct value = new Struct(schema) .put("id", 10L); @@ -143,7 +124,7 @@ public void shouldSerializeDecimal() { .field("id", DecimalUtil.builder(4, 2).build()) .build(); - final Serializer serializer = new KsqlDelimitedSerializer(); + final Serializer serializer = createSerializer(); final Struct value = new Struct(schema) .put("id", new BigDecimal("11.12")); @@ -162,7 +143,7 @@ public void shouldSerializeDecimalWithPaddedZeros() { .field("id", DecimalUtil.builder(4, 2).build()) .build(); - final Serializer serializer = new KsqlDelimitedSerializer(); + final Serializer serializer = createSerializer(); final Struct value = new Struct(schema) .put("id", new BigDecimal("1.12")); @@ -181,7 +162,7 @@ public void shouldSerializeZeroDecimalWithPaddedZeros() { .field("id", DecimalUtil.builder(4, 2).build()) .build(); - final Serializer serializer = new KsqlDelimitedSerializer(); + final Serializer serializer = createSerializer(); final Struct value = new Struct(schema) .put("id", BigDecimal.ZERO); @@ -200,7 +181,7 @@ public void shouldSerializeOneHalfDecimalWithPaddedZeros() { .field("id", DecimalUtil.builder(4, 2).build()) .build(); - final Serializer serializer = new KsqlDelimitedSerializer(); + final Serializer serializer = createSerializer(); final Struct value = new Struct(schema) .put("id", new BigDecimal(0.5)); @@ -219,7 +200,7 @@ public void shouldSerializeNegativeOneHalfDecimalWithPaddedZeros() { .field("id", DecimalUtil.builder(4, 2).build()) .build(); - final Serializer serializer = new KsqlDelimitedSerializer(); + final Serializer serializer = createSerializer(); final Struct value = new Struct(schema) .put("id", new BigDecimal(-0.5)); @@ -238,7 +219,7 @@ public void shouldSerializeNegativeDecimalWithPaddedZeros() { .field("id", DecimalUtil.builder(4, 2).build()) .build(); - final Serializer serializer = new KsqlDelimitedSerializer(); + final Serializer serializer = createSerializer(); final Struct value = new Struct(schema) .put("id", new BigDecimal("-1.12")); @@ -250,6 +231,36 @@ public void shouldSerializeNegativeDecimalWithPaddedZeros() { assertThat(new String(bytes, StandardCharsets.UTF_8), is("\"-01.12\"")); } + @Test + public void shouldSerializeRowCorrectlyWithTabDelimeter() { + shouldSerializeRowCorrectlyWithNonDefaultDelimeter('\t'); + } + + @Test + public void shouldSerializeRowCorrectlyWithBarDelimeter() { + shouldSerializeRowCorrectlyWithNonDefaultDelimeter('|'); + } + + private void shouldSerializeRowCorrectlyWithNonDefaultDelimeter(char delimiter) { + // Given: + final Struct data = new Struct(SCHEMA) + .put("ORDERTIME", 1511897796092L) + .put("ORDERID", 1L) + .put("ITEMID", "item_1") + .put("ORDERUNITS", 10.0); + + KsqlDelimitedSerializer serializer = + new KsqlDelimitedSerializer(CSVFormat.DEFAULT.withDelimiter(delimiter)); + + // When: + final byte[] bytes = serializer.serialize("t1", data); + + // Then: + final String delimitedString = new String(bytes, StandardCharsets.UTF_8); + assertThat(delimitedString, equalTo( + "1511897796092" + delimiter +"1" + delimiter + "item_1" + delimiter + "10.0")); + } + @Test public void shouldThrowOnArrayField() { // Given: @@ -317,14 +328,8 @@ public void shouldThrowOnStructField() { serializer.serialize("t1", data); } - @Test - public void shouldSerializeRowWithCustomDelimiter() { - - final GenericRow genericRow = new GenericRow(Arrays.asList(1511897796092L, 1L, "item_1", 10.0)); - final KsqlDelimitedSerializer ksqlDelimitedSerializer = new KsqlDelimitedSerializer(CSVFormat.DEFAULT.withDelimiter('^')); - final byte[] bytes = ksqlDelimitedSerializer.serialize("t1", genericRow); - - final String delimitedString = new String(bytes, StandardCharsets.UTF_8); - assertThat("Incorrect serialization.", delimitedString, equalTo("1511897796092^1^item_1^10.0")); + private KsqlDelimitedSerializer createSerializer() { + return new KsqlDelimitedSerializer(CSVFormat.DEFAULT.withDelimiter(',')); } + } From 3af05b5fb747faac913a3774ca3b5425fca90837 Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Fri, 20 Sep 2019 15:36:20 +0100 Subject: [PATCH 21/46] fix: mainly nits --- .../io/confluent/ksql/serde/FormatInfo.java | 2 +- .../confluent/ksql/serde/FormatInfoTest.java | 40 ++++++++++++++++++- .../confluent/ksql/serde/ValueFormatTest.java | 3 +- .../ksql/engine/InsertValuesExecutor.java | 1 - .../io/confluent/ksql/engine/KsqlEngine.java | 1 - .../delimited/KsqlDelimitedSerdeFactory.java | 2 +- 6 files changed, 42 insertions(+), 7 deletions(-) diff --git a/ksql-common/src/main/java/io/confluent/ksql/serde/FormatInfo.java b/ksql-common/src/main/java/io/confluent/ksql/serde/FormatInfo.java index d330d362229a..627319990105 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/serde/FormatInfo.java +++ b/ksql-common/src/main/java/io/confluent/ksql/serde/FormatInfo.java @@ -61,7 +61,7 @@ private FormatInfo( throw new KsqlException("Full schema name only supported with AVRO format"); } - if (avroFullSchemaName.map(name -> name.trim().isEmpty()).orElse(false)) { + if (format == Format.AVRO && avroFullSchemaName.map(name -> name.trim().isEmpty()).orElse(false)) { throw new KsqlException("Schema name cannot be empty"); } diff --git a/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java b/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java index cdebc74980c6..26e63bc56d59 100644 --- a/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java +++ b/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java @@ -16,6 +16,7 @@ package io.confluent.ksql.serde; import static io.confluent.ksql.serde.Format.AVRO; +import static io.confluent.ksql.serde.Format.DELIMITED; import static io.confluent.ksql.serde.Format.KAFKA; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsString; @@ -43,6 +44,10 @@ public void shouldThrowNPEs() { @Test public void shouldImplementEquals() { new EqualsTester() + .addEqualityGroup( + FormatInfo.of(Format.DELIMITED, Optional.empty(), Optional.of('x')), + FormatInfo.of(Format.DELIMITED, Optional.empty(), Optional.of('x')) + ) .addEqualityGroup( FormatInfo.of(Format.AVRO, Optional.of("something")), FormatInfo.of(Format.AVRO, Optional.of("something")) @@ -58,7 +63,7 @@ public void shouldImplementEquals() { } @Test - public void shouldImplementToString() { + public void shouldImplementToStringAvro() { // Given: final FormatInfo info = FormatInfo.of(AVRO, Optional.of("something")); @@ -70,6 +75,19 @@ public void shouldImplementToString() { assertThat(result, containsString("something")); } + @Test + public void shouldImplementToStringDelimited() { + // Given: + final FormatInfo info = FormatInfo.of(DELIMITED, Optional.empty(), Optional.of('~')); + + // When: + final String result = info.toString(); + + // Then: + assertThat(result, containsString("DELIMITED")); + assertThat(result, containsString("~")); + } + @Test public void shouldThrowOnNonAvroWithAvroSchemName() { // Then: @@ -103,4 +121,24 @@ public void shouldGetAvroSchemaName() { assertThat(FormatInfo.of(AVRO, Optional.empty()).getAvroFullSchemaName(), is(Optional.empty())); } + + @Test + public void shouldThrowWhenAttemptingToUseValueDelimeterWithAvroFormat() { + // Then: + expectedException.expect(KsqlException.class); + expectedException.expectMessage("Delimeter only supported with DELIMITED format"); + + // When: + FormatInfo.of(Format.AVRO, Optional.of("something"), Optional.of('x')); + } + + @Test + public void shouldThrowWhenAttemptingToUseValueDelimeterWithJsonFormat() { + // Then: + expectedException.expect(KsqlException.class); + expectedException.expectMessage("Delimeter only supported with DELIMITED format"); + + // When: + FormatInfo.of(Format.JSON, Optional.empty(), Optional.of('x')); + } } \ No newline at end of file diff --git a/ksql-common/src/test/java/io/confluent/ksql/serde/ValueFormatTest.java b/ksql-common/src/test/java/io/confluent/ksql/serde/ValueFormatTest.java index 28c9410625b2..c174eaef7251 100644 --- a/ksql-common/src/test/java/io/confluent/ksql/serde/ValueFormatTest.java +++ b/ksql-common/src/test/java/io/confluent/ksql/serde/ValueFormatTest.java @@ -30,8 +30,7 @@ public class ValueFormatTest { private static final FormatInfo FORMAT_INFO = FormatInfo.of( - AVRO, Optional.of("something"), - Optional.empty() + AVRO, Optional.of("something") ); @Test diff --git a/ksql-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java b/ksql-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java index f731200ed0da..e36c14f0e668 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java @@ -137,7 +137,6 @@ public void execute( final ServiceContext serviceContext ) { final InsertValues insertValues = statement.getStatement(); - final KsqlConfig config = statement.getConfig() .cloneWithPropertyOverwrite(statement.getOverrides()); diff --git a/ksql-engine/src/main/java/io/confluent/ksql/engine/KsqlEngine.java b/ksql-engine/src/main/java/io/confluent/ksql/engine/KsqlEngine.java index 98eb1302a22f..a4bbb6eff6a8 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/engine/KsqlEngine.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/engine/KsqlEngine.java @@ -168,7 +168,6 @@ public ExecuteResult execute( final ServiceContext serviceContext, final ConfiguredStatement statement ) { - final ExecuteResult result = EngineExecutor .create(primaryContext, serviceContext, statement.getConfig(), statement.getOverrides()) .execute(statement); diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java index 1b69438d6150..6b09857edf7f 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java @@ -37,7 +37,7 @@ @Immutable public class KsqlDelimitedSerdeFactory implements KsqlSerdeFactory { - public static final char DEFAULT_DELIMITER = ','; + private static final char DEFAULT_DELIMITER = ','; private final CSVFormat csvFormat; From 0038c3ef9a0ca77d17793dc981a7d2adbfead5f0 Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Fri, 20 Sep 2019 16:39:06 +0100 Subject: [PATCH 22/46] fix: checkstyle issue --- .../src/main/java/io/confluent/ksql/serde/FormatInfo.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ksql-common/src/main/java/io/confluent/ksql/serde/FormatInfo.java b/ksql-common/src/main/java/io/confluent/ksql/serde/FormatInfo.java index 627319990105..42d4bc16ffcc 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/serde/FormatInfo.java +++ b/ksql-common/src/main/java/io/confluent/ksql/serde/FormatInfo.java @@ -61,7 +61,8 @@ private FormatInfo( throw new KsqlException("Full schema name only supported with AVRO format"); } - if (format == Format.AVRO && avroFullSchemaName.map(name -> name.trim().isEmpty()).orElse(false)) { + if (format == Format.AVRO + && avroFullSchemaName.map(name -> name.trim().isEmpty()).orElse(false)) { throw new KsqlException("Schema name cannot be empty"); } From c469f85af773865296a79184c03276d2c470f06d Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Fri, 20 Sep 2019 18:21:39 +0100 Subject: [PATCH 23/46] Removed 2 param of() method for FormatInfo that was only used in tests --- .../ksql/benchmark/SerdeBenchmark.java | 4 +- .../io/confluent/ksql/serde/FormatInfo.java | 12 +-- .../confluent/ksql/serde/FormatInfoTest.java | 21 ++-- .../confluent/ksql/serde/KeyFormatTest.java | 18 ++-- .../confluent/ksql/serde/ValueFormatTest.java | 4 +- .../io/confluent/ksql/topic/TopicFactory.java | 6 +- .../ksql/analyzer/AnalyzerFunctionalTest.java | 7 +- .../ddl/commands/CommandFactoriesTest.java | 4 +- .../ksql/engine/InsertValuesExecutorTest.java | 4 +- .../integration/IntegrationTestHarness.java | 4 +- .../KsqlStructuredDataOutputNodeTest.java | 3 +- .../structured/SchemaKGroupedTableTest.java | 2 +- .../ksql/topic/TopicDeleteInjectorTest.java | 2 +- .../io/confluent/ksql/util/TopicConsumer.java | 2 +- .../io/confluent/ksql/util/TopicProducer.java | 2 +- .../builder/KsqlQueryBuilderTest.java | 2 +- .../query-validation-tests/scratch.json | 100 ++++++++++++++++++ .../ksql/serde/GenericRowSerDeTest.java | 16 +-- .../ksql/serde/KsqlSerdeFactoriesTest.java | 8 +- 19 files changed, 160 insertions(+), 61 deletions(-) create mode 100644 ksql-functional-tests/src/test/resources/query-validation-tests/scratch.json diff --git a/ksql-benchmark/src/main/java/io/confluent/ksql/benchmark/SerdeBenchmark.java b/ksql-benchmark/src/main/java/io/confluent/ksql/benchmark/SerdeBenchmark.java index dd2c6d535cff..7a96b355fe19 100644 --- a/ksql-benchmark/src/main/java/io/confluent/ksql/benchmark/SerdeBenchmark.java +++ b/ksql-benchmark/src/main/java/io/confluent/ksql/benchmark/SerdeBenchmark.java @@ -163,7 +163,7 @@ private static Serde getJsonSerdeHelper( final org.apache.kafka.connect.data.Schema schema ) { return getGenericRowSerde( - FormatInfo.of(Format.JSON, Optional.empty()), + FormatInfo.of(Format.JSON), schema, () -> null ); @@ -175,7 +175,7 @@ private static Serde getAvroSerde( final SchemaRegistryClient schemaRegistryClient = new MockSchemaRegistryClient(); return getGenericRowSerde( - FormatInfo.of(Format.AVRO, Optional.of("benchmarkSchema")), + FormatInfo.of(Format.AVRO, Optional.of("benchmarkSchema"), Optional.empty()), schema, () -> schemaRegistryClient ); diff --git a/ksql-common/src/main/java/io/confluent/ksql/serde/FormatInfo.java b/ksql-common/src/main/java/io/confluent/ksql/serde/FormatInfo.java index 42d4bc16ffcc..460fe879acc4 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/serde/FormatInfo.java +++ b/ksql-common/src/main/java/io/confluent/ksql/serde/FormatInfo.java @@ -34,19 +34,11 @@ public static FormatInfo of(final Format format) { return FormatInfo.of(format, Optional.empty(), Optional.empty()); } - public static FormatInfo of( - final Format format, - final Optional avroFullSchemaName - ) { - return FormatInfo.of(format, avroFullSchemaName, Optional.empty()); - } - public static FormatInfo of( final Format format, final Optional avroFullSchemaName, - final Optional delimiter - ) { - return new FormatInfo(format, avroFullSchemaName, delimiter); + final Optional valueDelimiter) { + return new FormatInfo(format, avroFullSchemaName, valueDelimiter); } private FormatInfo( diff --git a/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java b/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java index 26e63bc56d59..6b2bce8cdfb5 100644 --- a/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java +++ b/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java @@ -49,15 +49,16 @@ public void shouldImplementEquals() { FormatInfo.of(Format.DELIMITED, Optional.empty(), Optional.of('x')) ) .addEqualityGroup( - FormatInfo.of(Format.AVRO, Optional.of("something")), - FormatInfo.of(Format.AVRO, Optional.of("something")) + FormatInfo.of(Format.AVRO, Optional.of("something"), Optional.empty()), + FormatInfo.of(Format.AVRO, Optional.of("something"), Optional.empty()) ) .addEqualityGroup( - FormatInfo.of(Format.AVRO, Optional.empty()), + FormatInfo.of(Format.AVRO, Optional.empty(), Optional.empty()), FormatInfo.of(Format.AVRO) ) .addEqualityGroup( - FormatInfo.of(Format.JSON, Optional.empty()) + FormatInfo.of(Format.JSON, Optional.empty(), Optional.empty()), + FormatInfo.of(Format.JSON) ) .testEquals(); } @@ -65,7 +66,7 @@ public void shouldImplementEquals() { @Test public void shouldImplementToStringAvro() { // Given: - final FormatInfo info = FormatInfo.of(AVRO, Optional.of("something")); + final FormatInfo info = FormatInfo.of(AVRO, Optional.of("something"), Optional.empty()); // When: final String result = info.toString(); @@ -95,7 +96,7 @@ public void shouldThrowOnNonAvroWithAvroSchemName() { expectedException.expectMessage("Full schema name only supported with AVRO format"); // When: - FormatInfo.of(Format.JSON, Optional.of("thing")); + FormatInfo.of(Format.JSON, Optional.of("thing"), Optional.empty()); } @Test @@ -105,20 +106,20 @@ public void shouldThrowOnEmptyAvroSchemaName() { expectedException.expectMessage("Schema name cannot be empty"); // When: - FormatInfo.of(Format.AVRO, Optional.of("")); + FormatInfo.of(Format.AVRO, Optional.of(""), Optional.empty()); } @Test public void shouldGetFormat() { - assertThat(FormatInfo.of(KAFKA, Optional.empty()).getFormat(), is(KAFKA)); + assertThat(FormatInfo.of(KAFKA, Optional.empty(), Optional.empty()).getFormat(), is(KAFKA)); } @Test public void shouldGetAvroSchemaName() { - assertThat(FormatInfo.of(AVRO, Optional.of("Something")).getAvroFullSchemaName(), + assertThat(FormatInfo.of(AVRO, Optional.of("Something"), Optional.empty()).getAvroFullSchemaName(), is(Optional.of("Something"))); - assertThat(FormatInfo.of(AVRO, Optional.empty()).getAvroFullSchemaName(), + assertThat(FormatInfo.of(AVRO, Optional.empty(), Optional.empty()).getAvroFullSchemaName(), is(Optional.empty())); } diff --git a/ksql-common/src/test/java/io/confluent/ksql/serde/KeyFormatTest.java b/ksql-common/src/test/java/io/confluent/ksql/serde/KeyFormatTest.java index e8de9320b3f8..8a798ff6ebc6 100644 --- a/ksql-common/src/test/java/io/confluent/ksql/serde/KeyFormatTest.java +++ b/ksql-common/src/test/java/io/confluent/ksql/serde/KeyFormatTest.java @@ -44,8 +44,8 @@ public void shouldThrowNPEs() { @Test public void shouldImplementEquals() { - final FormatInfo format1 = FormatInfo.of(AVRO, Optional.empty()); - final FormatInfo format2 = FormatInfo.of(JSON, Optional.empty()); + final FormatInfo format1 = FormatInfo.of(AVRO, Optional.empty(), Optional.empty()); + final FormatInfo format2 = FormatInfo.of(JSON, Optional.empty(), Optional.empty()); final WindowInfo window1 = WindowInfo.of(SESSION, Optional.empty()); final WindowInfo window2 = WindowInfo.of(HOPPING, Optional.of(Duration.ofMillis(1000))); @@ -74,7 +74,7 @@ public void shouldImplementEquals() { @Test public void shouldImplementToString() { // Given: - final FormatInfo formatInfo = FormatInfo.of(AVRO, Optional.of("something")); + final FormatInfo formatInfo = FormatInfo.of(AVRO, Optional.of("something"), Optional.empty()); final WindowInfo windowInfo = WindowInfo.of(HOPPING, Optional.of(Duration.ofMillis(10101))); final KeyFormat keyFormat = KeyFormat.windowed(formatInfo, windowInfo); @@ -90,7 +90,7 @@ public void shouldImplementToString() { @Test public void shouldGetFormat() { // Given: - final FormatInfo format = FormatInfo.of(DELIMITED, Optional.empty()); + final FormatInfo format = FormatInfo.of(DELIMITED, Optional.empty(), Optional.empty()); final KeyFormat keyFormat = KeyFormat.nonWindowed(format); // When: @@ -103,7 +103,7 @@ public void shouldGetFormat() { @Test public void shouldGetFormatInfo() { // Given: - final FormatInfo format = FormatInfo.of(AVRO, Optional.of("something")); + final FormatInfo format = FormatInfo.of(AVRO, Optional.of("something"), Optional.empty()); final KeyFormat keyFormat = KeyFormat.nonWindowed(format); // When: @@ -116,7 +116,8 @@ public void shouldGetFormatInfo() { @Test public void shouldHandleNoneWindowedFunctionsForNonWindowed() { // Given: - final KeyFormat keyFormat = KeyFormat.nonWindowed(FormatInfo.of(JSON, Optional.empty())); + final KeyFormat keyFormat = KeyFormat.nonWindowed(FormatInfo.of(JSON, Optional.empty(), + Optional.empty())); // Then: assertThat(keyFormat.isWindowed(), is(false)); @@ -142,12 +143,13 @@ public void shouldHandleWindowedFunctionsForWindowed() { public void shouldHandleWindowedWithAvroSchemaName() { // Given: final KeyFormat keyFormat = KeyFormat.windowed( - FormatInfo.of(AVRO, Optional.of("something")), + FormatInfo.of(AVRO, Optional.of("something"), Optional.empty()), WindowInfo.of(HOPPING, Optional.of(Duration.ofMinutes(4))) ); // Then: - assertThat(keyFormat.getFormatInfo(), is(FormatInfo.of(AVRO, Optional.of("something")))); + assertThat(keyFormat.getFormatInfo(), is(FormatInfo.of(AVRO, Optional.of("something"), + Optional.empty()))); } @Test diff --git a/ksql-common/src/test/java/io/confluent/ksql/serde/ValueFormatTest.java b/ksql-common/src/test/java/io/confluent/ksql/serde/ValueFormatTest.java index c174eaef7251..21adf12b0e18 100644 --- a/ksql-common/src/test/java/io/confluent/ksql/serde/ValueFormatTest.java +++ b/ksql-common/src/test/java/io/confluent/ksql/serde/ValueFormatTest.java @@ -30,8 +30,8 @@ public class ValueFormatTest { private static final FormatInfo FORMAT_INFO = FormatInfo.of( - AVRO, Optional.of("something") - ); + AVRO, Optional.of("something"), + Optional.empty()); @Test public void shouldThrowNPEs() { diff --git a/ksql-engine/src/main/java/io/confluent/ksql/topic/TopicFactory.java b/ksql-engine/src/main/java/io/confluent/ksql/topic/TopicFactory.java index c4e36b394c1d..90cf4ad5c6f1 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/topic/TopicFactory.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/topic/TopicFactory.java @@ -41,12 +41,14 @@ public static KsqlTopic create(final CreateSourceProperties properties) { .map(type -> KeyFormat .windowed(FormatInfo.of(Format.KAFKA), WindowInfo.of(type, windowSize))) .orElseGet(() -> KeyFormat - .nonWindowed(FormatInfo.of(Format.KAFKA, Optional.empty()))); + .nonWindowed(FormatInfo.of(Format.KAFKA))); + + Optional valueDelimiter = properties.getValueDelimiter(); final ValueFormat valueFormat = ValueFormat.of(FormatInfo.of( properties.getValueFormat(), properties.getValueAvroSchemaName(), - properties.getValueDelimiter() + valueDelimiter )); return new KsqlTopic( diff --git a/ksql-engine/src/test/java/io/confluent/ksql/analyzer/AnalyzerFunctionalTest.java b/ksql-engine/src/test/java/io/confluent/ksql/analyzer/AnalyzerFunctionalTest.java index c774ba30fa96..0e9161efca08 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/analyzer/AnalyzerFunctionalTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/analyzer/AnalyzerFunctionalTest.java @@ -299,7 +299,8 @@ public void shouldUseExplicitNamespaceForAvroSchema() { assertThat(analysis.getInto(), is(not(Optional.empty()))); assertThat(analysis.getInto().get().getKsqlTopic().getValueFormat(), - is(ValueFormat.of(FormatInfo.of(Format.AVRO, Optional.of("com.custom.schema"))))); + is(ValueFormat.of(FormatInfo.of(Format.AVRO, Optional.of("com.custom.schema"), + Optional.empty())))); } @Test @@ -330,7 +331,7 @@ public void shouldUseExplicitNamespaceWhenFormatIsInheritedForAvro() { assertThat(analysis.getInto(), is(not(Optional.empty()))); assertThat(analysis.getInto().get().getKsqlTopic().getValueFormat(), - is(ValueFormat.of(FormatInfo.of(Format.AVRO, Optional.of("org.ac.s1"))))); + is(ValueFormat.of(FormatInfo.of(Format.AVRO, Optional.of("org.ac.s1"), Optional.empty())))); } @Test @@ -342,7 +343,7 @@ public void shouldNotInheritNamespaceExplicitlySetUpstreamForAvro() { final KsqlTopic ksqlTopic = new KsqlTopic( "s0", KeyFormat.nonWindowed(FormatInfo.of(Format.KAFKA)), - ValueFormat.of(FormatInfo.of(Format.AVRO, Optional.of("org.ac.s1"))), + ValueFormat.of(FormatInfo.of(Format.AVRO, Optional.of("org.ac.s1"), Optional.empty())), false); final LogicalSchema schema = LogicalSchema.builder() diff --git a/ksql-engine/src/test/java/io/confluent/ksql/ddl/commands/CommandFactoriesTest.java b/ksql-engine/src/test/java/io/confluent/ksql/ddl/commands/CommandFactoriesTest.java index f10f1a434479..67dba4315b2a 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/ddl/commands/CommandFactoriesTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/ddl/commands/CommandFactoriesTest.java @@ -668,7 +668,7 @@ public void shouldCreateSerdeToValidateValueFormatCanHandleValueSchemaForStream( // Then: verify(serdeFactory).create( - FormatInfo.of(JSON, Optional.empty()), + FormatInfo.of(JSON, Optional.empty(), Optional.empty()), PersistenceSchema.from(schema.valueConnectSchema(), false), ksqlConfig, serviceContext.getSchemaRegistryClientFactory(), @@ -711,7 +711,7 @@ public void shouldHandleValueAvroSchemaNameForStream() { // Then: assertThat(cmd.getTopic().getValueFormat(), - is(ValueFormat.of(FormatInfo.of(AVRO, Optional.of("full.schema.name"))))); + is(ValueFormat.of(FormatInfo.of(AVRO, Optional.of("full.schema.name"), Optional.empty())))); } @Test diff --git a/ksql-engine/src/test/java/io/confluent/ksql/engine/InsertValuesExecutorTest.java b/ksql-engine/src/test/java/io/confluent/ksql/engine/InsertValuesExecutorTest.java index f80d38cb8641..8bd71198c5e6 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/engine/InsertValuesExecutorTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/engine/InsertValuesExecutorTest.java @@ -654,7 +654,7 @@ public void shouldBuildCorrectSerde() { // Then: verify(keySerdeFactory).create( - FormatInfo.of(Format.KAFKA, Optional.empty()), + FormatInfo.of(Format.KAFKA, Optional.empty(), Optional.empty()), PersistenceSchema.from(SCHEMA.keyConnectSchema(), false), new KsqlConfig(ImmutableMap.of()), srClientFactory, @@ -663,7 +663,7 @@ public void shouldBuildCorrectSerde() { ); verify(valueSerdeFactory).create( - FormatInfo.of(Format.JSON, Optional.empty()), + FormatInfo.of(Format.JSON, Optional.empty(), Optional.empty()), PersistenceSchema.from(SCHEMA.valueConnectSchema(), false), new KsqlConfig(ImmutableMap.of()), srClientFactory, diff --git a/ksql-engine/src/test/java/io/confluent/ksql/integration/IntegrationTestHarness.java b/ksql-engine/src/test/java/io/confluent/ksql/integration/IntegrationTestHarness.java index 58b9f0103ffd..687ff76e0c6a 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/integration/IntegrationTestHarness.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/integration/IntegrationTestHarness.java @@ -528,7 +528,7 @@ private Serializer getSerializer( final PhysicalSchema schema ) { return GenericRowSerDe.from( - FormatInfo.of(format, Optional.empty()), + FormatInfo.of(format, Optional.empty(), Optional.empty()), schema.valueSchema(), new KsqlConfig(Collections.emptyMap()), serviceContext.get().getSchemaRegistryClientFactory(), @@ -542,7 +542,7 @@ private Deserializer getDeserializer( final PhysicalSchema schema ) { return GenericRowSerDe.from( - FormatInfo.of(format, Optional.empty()), + FormatInfo.of(format, Optional.empty(), Optional.empty()), schema.valueSchema(), new KsqlConfig(Collections.emptyMap()), serviceContext.get().getSchemaRegistryClientFactory(), diff --git a/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNodeTest.java b/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNodeTest.java index ae4497e8e58a..9e7e247f0ab4 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNodeTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNodeTest.java @@ -304,7 +304,8 @@ public void shouldBuildOutputNodeForInsertIntoAvroFromNonAvro() { // Given: givenInsertIntoNode(); - final ValueFormat valueFormat = ValueFormat.of(FormatInfo.of(Format.AVRO, Optional.of("name"))); + final ValueFormat valueFormat = ValueFormat.of(FormatInfo.of(Format.AVRO, Optional.of("name"), + Optional.empty())); when(ksqlTopic.getValueFormat()).thenReturn(valueFormat); diff --git a/ksql-engine/src/test/java/io/confluent/ksql/structured/SchemaKGroupedTableTest.java b/ksql-engine/src/test/java/io/confluent/ksql/structured/SchemaKGroupedTableTest.java index 6a8b3137310f..c647de9be638 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/structured/SchemaKGroupedTableTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/structured/SchemaKGroupedTableTest.java @@ -266,7 +266,7 @@ public void shouldFailUnsupportedAggregateFunction() { null, valueFormat, GenericRowSerDe.from( - FormatInfo.of(Format.JSON, Optional.empty()), + FormatInfo.of(Format.JSON, Optional.empty(), Optional.empty()), PersistenceSchema.from(ksqlTable.getSchema().valueConnectSchema(), false), ksqlConfig, () -> null, diff --git a/ksql-engine/src/test/java/io/confluent/ksql/topic/TopicDeleteInjectorTest.java b/ksql-engine/src/test/java/io/confluent/ksql/topic/TopicDeleteInjectorTest.java index c8a6601a6e12..9538365c1934 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/topic/TopicDeleteInjectorTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/topic/TopicDeleteInjectorTest.java @@ -247,7 +247,7 @@ public void shouldThrowIfTopicDoesNotExist() { public void shouldNotThrowIfSchemaIsMissing() throws IOException, RestClientException { // Given: when(topic.getValueFormat()) - .thenReturn(ValueFormat.of(FormatInfo.of(Format.AVRO, Optional.of("foo")))); + .thenReturn(ValueFormat.of(FormatInfo.of(Format.AVRO, Optional.of("foo"), Optional.empty()))); doThrow(new RestClientException("Subject not found.", 404, 40401)) .when(registryClient).deleteSubject("something" + KsqlConstants.SCHEMA_REGISTRY_VALUE_SUFFIX); diff --git a/ksql-engine/src/test/java/io/confluent/ksql/util/TopicConsumer.java b/ksql-engine/src/test/java/io/confluent/ksql/util/TopicConsumer.java index 7d3fa8d29471..0003b476e001 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/util/TopicConsumer.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/util/TopicConsumer.java @@ -97,7 +97,7 @@ public Map readResults( final Deserializer keyDeserializer ) { final Deserializer deserializer = GenericRowSerDe.from( - FormatInfo.of(Format.JSON, Optional.empty()), + FormatInfo.of(Format.JSON, Optional.empty(), Optional.empty()), schema.valueSchema(), new KsqlConfig(ImmutableMap.of()), () -> null, diff --git a/ksql-engine/src/test/java/io/confluent/ksql/util/TopicProducer.java b/ksql-engine/src/test/java/io/confluent/ksql/util/TopicProducer.java index 9c5603bdcda6..9ac711029611 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/util/TopicProducer.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/util/TopicProducer.java @@ -68,7 +68,7 @@ public Map produceInputData( ) throws InterruptedException, TimeoutException, ExecutionException { final Serializer serializer = GenericRowSerDe.from( - FormatInfo.of(Format.JSON, Optional.empty()), + FormatInfo.of(Format.JSON, Optional.empty(), Optional.empty()), schema.valueSchema(), new KsqlConfig(ImmutableMap.of()), () -> null, diff --git a/ksql-execution/src/test/java/io/confluent/ksql/execution/builder/KsqlQueryBuilderTest.java b/ksql-execution/src/test/java/io/confluent/ksql/execution/builder/KsqlQueryBuilderTest.java index d26159a50979..75bad8d68ed9 100644 --- a/ksql-execution/src/test/java/io/confluent/ksql/execution/builder/KsqlQueryBuilderTest.java +++ b/ksql-execution/src/test/java/io/confluent/ksql/execution/builder/KsqlQueryBuilderTest.java @@ -71,7 +71,7 @@ public class KsqlQueryBuilderTest { private static final QueryId QUERY_ID = new QueryId("fred"); private static final FormatInfo FORMAT_INFO = FormatInfo - .of(Format.AVRO, Optional.of("io.confluent.ksql")); + .of(Format.AVRO, Optional.of("io.confluent.ksql"), Optional.empty()); private static final WindowInfo WINDOW_INFO = WindowInfo .of(WindowType.TUMBLING, Optional.of(Duration.ofMillis(1000))); diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/scratch.json b/ksql-functional-tests/src/test/resources/query-validation-tests/scratch.json new file mode 100644 index 000000000000..ab2585d9489b --- /dev/null +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/scratch.json @@ -0,0 +1,100 @@ +{ + "comments": [ + "When using value_format DELIMITED, we can define VALUE_DELIMITER as custom character." + ], + "tests": [ + { + "name": "select delimited value_format", + "statements": [ + "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED');", + "CREATE STREAM S2 as SELECT id, name, value FROM test;" + ], + "topics": [ + { + "name": "test_topic", + "format": "DELIMITED" + } + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": "0,zero,0", "timestamp": 0}, + {"topic": "test_topic", "key": 0, "value": "0,100,100", "timestamp": 0}, + {"topic": "test_topic", "key": 100, "value": "100,100,500", "timestamp": 0}, + {"topic": "test_topic", "key": 100, "value": "100,100,100", "timestamp": 0} + ], + "outputs": [ + {"topic": "S2", "key": 0, "value": "0,zero,0", "timestamp": 0}, + {"topic": "S2", "key": 0, "value": "0,100,100", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100,100,500", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100,100,100", "timestamp": 0} + ] + }, + { + "name": "validate value_delimiter to be single character", + "statements": [ + "CREATE STREAM TEST WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='<~>');" + ], + "topics": [ + { + "name": "test_topic", + "format": "DELIMITED" + } + ], + "expectedException": { + "type": "io.confluent.ksql.util.KsqlException", + "message": "Error in WITH clause property 'VALUE_DELIMITER': Delimiter must be a single character.\nExample valid value: ';'" + }, + "inputs": [], + "outputs": [] + }, + { + "name": "select delimited value_format with pipe separated values using custom delimiter character", + "statements": [ + "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='|');", + "CREATE STREAM S2 as SELECT id, name, value FROM test;" + ], + "topics": [ + { + "name": "test_topic", + "format": "DELIMITED" + } + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": "0|zero|0", "timestamp": 0}, + {"topic": "test_topic", "key": 0, "value": "0|100|100", "timestamp": 0}, + {"topic": "test_topic", "key": 100, "value": "100|100|500", "timestamp": 0}, + {"topic": "test_topic", "key": 100, "value": "100|100|100", "timestamp": 0} + ], + "outputs": [ + {"topic": "S2", "key": 0, "value": "0,zero,0", "timestamp": 0}, + {"topic": "S2", "key": 0, "value": "0,100,100", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100,100,500", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100,100,100", "timestamp": 0} + ] + }, + { + "name": "Serialize delimited value_format with $", + "statements": [ + "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='|');", + "CREATE STREAM S2 WITH(value_delimiter='$') AS SELECT * FROM test;" + ], + "topics": [ + { + "name": "test_topic", + "format": "DELIMITED" + } + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": "0|zero|0", "timestamp": 0}, + {"topic": "test_topic", "key": 0, "value": "0|100|100", "timestamp": 0}, + {"topic": "test_topic", "key": 100, "value": "100|100|500", "timestamp": 0}, + {"topic": "test_topic", "key": 100, "value": "100|100|100", "timestamp": 0} + ], + "outputs": [ + {"topic": "S2", "key": 0, "value": "0$zero$0", "timestamp": 0}, + {"topic": "S2", "key": 0, "value": "0$100$100", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100$100$500", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100$100$100", "timestamp": 0} + ] + } + ] +} \ No newline at end of file diff --git a/ksql-serde/src/test/java/io/confluent/ksql/serde/GenericRowSerDeTest.java b/ksql-serde/src/test/java/io/confluent/ksql/serde/GenericRowSerDeTest.java index 5662e1b96bd0..0875125a08cb 100644 --- a/ksql-serde/src/test/java/io/confluent/ksql/serde/GenericRowSerDeTest.java +++ b/ksql-serde/src/test/java/io/confluent/ksql/serde/GenericRowSerDeTest.java @@ -123,7 +123,7 @@ public void setUp() { public void shouldGetStructSerdeOnConstruction() { // When: valueSerde.create( - FormatInfo.of(Format.JSON, Optional.empty()), + FormatInfo.of(Format.JSON, Optional.empty(), Optional.empty()), MUTLI_FIELD_SCHEMA, ksqlConfig, srClientFactory, @@ -133,7 +133,7 @@ public void shouldGetStructSerdeOnConstruction() { // Then: verify(serdesFactories).create( - FormatInfo.of(Format.JSON, Optional.empty()), + FormatInfo.of(Format.JSON, Optional.empty(), Optional.empty()), MUTLI_FIELD_SCHEMA, ksqlConfig, srClientFactory, @@ -145,7 +145,7 @@ public void shouldGetStructSerdeOnConstruction() { public void shouldGetStringSerdeOnConstruction() { // When: valueSerde.create( - FormatInfo.of(Format.JSON, Optional.empty()), + FormatInfo.of(Format.JSON, Optional.empty(), Optional.empty()), UNWRAPPED_SINGLE_FIELD_SCHEMA, ksqlConfig, srClientFactory, @@ -155,7 +155,7 @@ public void shouldGetStringSerdeOnConstruction() { // Then: verify(serdesFactories).create( - FormatInfo.of(Format.JSON, Optional.empty()), + FormatInfo.of(Format.JSON, Optional.empty(), Optional.empty()), UNWRAPPED_SINGLE_FIELD_SCHEMA, ksqlConfig, srClientFactory, @@ -170,7 +170,7 @@ public void shouldThrowOnNullStructSerde() { // When: valueSerde.create( - FormatInfo.of(Format.JSON, Optional.empty()), + FormatInfo.of(Format.JSON, Optional.empty(), Optional.empty()), MUTLI_FIELD_SCHEMA, ksqlConfig, srClientFactory, @@ -183,7 +183,7 @@ public void shouldThrowOnNullStructSerde() { public void shouldThrowOnNullSchema() { // When: GenericRowSerDe.from( - FormatInfo.of(Format.JSON, Optional.empty()), + FormatInfo.of(Format.JSON, Optional.empty(), Optional.empty()), null, ksqlConfig, srClientFactory, @@ -196,7 +196,7 @@ public void shouldThrowOnNullSchema() { public void shouldCreateProcessingLoggerWithCorrectName() { // When: GenericRowSerDe.from( - FormatInfo.of(Format.JSON, Optional.empty()), + FormatInfo.of(Format.JSON, Optional.empty(), Optional.empty()), MUTLI_FIELD_SCHEMA, ksqlConfig, srClientFactory, @@ -494,7 +494,7 @@ public void shouldDeserializeNullUnwrappedSingleFieldGenericRow() { private Serde givenSerdeForSchema(final PersistenceSchema schema) { return valueSerde.create( - FormatInfo.of(Format.JSON, Optional.empty()), + FormatInfo.of(Format.JSON, Optional.empty(), Optional.empty()), schema, ksqlConfig, srClientFactory, diff --git a/ksql-serde/src/test/java/io/confluent/ksql/serde/KsqlSerdeFactoriesTest.java b/ksql-serde/src/test/java/io/confluent/ksql/serde/KsqlSerdeFactoriesTest.java index 0b66eb33dcc3..16cbae5c996f 100644 --- a/ksql-serde/src/test/java/io/confluent/ksql/serde/KsqlSerdeFactoriesTest.java +++ b/ksql-serde/src/test/java/io/confluent/ksql/serde/KsqlSerdeFactoriesTest.java @@ -136,7 +136,7 @@ public void shouldCreateSerde() { public void shouldHandleAvro() { // When: final KsqlSerdeFactory result = KsqlSerdeFactories - .create(FormatInfo.of(AVRO, Optional.empty())); + .create(FormatInfo.of(AVRO, Optional.empty(), Optional.empty())); // Then: assertThat(result, instanceOf(KsqlAvroSerdeFactory.class)); @@ -146,7 +146,7 @@ public void shouldHandleAvro() { public void shouldHandleJson() { // When: final KsqlSerdeFactory result = KsqlSerdeFactories - .create(FormatInfo.of(JSON, Optional.empty())); + .create(FormatInfo.of(JSON, Optional.empty(), Optional.empty())); // Then: assertThat(result, instanceOf(KsqlJsonSerdeFactory.class)); @@ -156,7 +156,7 @@ public void shouldHandleJson() { public void shouldHandleDelimited() { // When: final KsqlSerdeFactory result = KsqlSerdeFactories - .create(FormatInfo.of(DELIMITED, Optional.empty())); + .create(FormatInfo.of(DELIMITED, Optional.empty(), Optional.empty())); // Then: assertThat(result, instanceOf(KsqlDelimitedSerdeFactory.class)); @@ -166,7 +166,7 @@ public void shouldHandleDelimited() { public void shouldHandleKafka() { // When: final KsqlSerdeFactory result = KsqlSerdeFactories - .create(FormatInfo.of(KAFKA, Optional.empty())); + .create(FormatInfo.of(KAFKA, Optional.empty(), Optional.empty())); // Then: assertThat(result, instanceOf(KafkaSerdeFactory.class)); From b1344ed6cfcef08d81575d805e926b0cff823a04 Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Fri, 20 Sep 2019 18:27:38 +0100 Subject: [PATCH 24/46] Add special handling for TAB and SPACE --- .../delimited-value-format.json | 52 ++++++++- .../query-validation-tests/scratch.json | 100 ------------------ .../properties/with/WithClauseProperties.java | 29 ++++- 3 files changed, 76 insertions(+), 105 deletions(-) delete mode 100644 ksql-functional-tests/src/test/resources/query-validation-tests/scratch.json diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json b/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json index ab2585d9489b..14619406270c 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json @@ -41,7 +41,7 @@ ], "expectedException": { "type": "io.confluent.ksql.util.KsqlException", - "message": "Error in WITH clause property 'VALUE_DELIMITER': Delimiter must be a single character.\nExample valid value: ';'" + "message": "Error in WITH clause property 'VALUE_DELIMITER': Delimiter must be a single character, 'TAB' or 'SPACE'.\nExample valid value: ';'" }, "inputs": [], "outputs": [] @@ -95,6 +95,56 @@ {"topic": "S2", "key": 100, "value": "100$100$500", "timestamp": 0}, {"topic": "S2", "key": 100, "value": "100$100$100", "timestamp": 0} ] + }, + { + "name": "select delimited value_format with space separated values using custom delimiter character", + "statements": [ + "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='SPACE');", + "CREATE STREAM S2 as SELECT id, name, value FROM test;" + ], + "topics": [ + { + "name": "test_topic", + "format": "DELIMITED" + } + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": "0 zero 0", "timestamp": 0}, + {"topic": "test_topic", "key": 0, "value": "0 100 100", "timestamp": 0}, + {"topic": "test_topic", "key": 100, "value": "100 100 500", "timestamp": 0}, + {"topic": "test_topic", "key": 100, "value": "100 100 100", "timestamp": 0} + ], + "outputs": [ + {"topic": "S2", "key": 0, "value": "0,zero,0", "timestamp": 0}, + {"topic": "S2", "key": 0, "value": "0,100,100", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100,100,500", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100,100,100", "timestamp": 0} + ] + }, + { + "name": "select delimited value_format with tab separated values using custom delimiter character", + "statements": [ + "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='TAB');", + "CREATE STREAM S2 as SELECT id, name, value FROM test;" + ], + "topics": [ + { + "name": "test_topic", + "format": "DELIMITED" + } + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": "0\tzero\t0", "timestamp": 0}, + {"topic": "test_topic", "key": 0, "value": "0\t100\t100", "timestamp": 0}, + {"topic": "test_topic", "key": 100, "value": "100\t100\t500", "timestamp": 0}, + {"topic": "test_topic", "key": 100, "value": "100\t100\t100", "timestamp": 0} + ], + "outputs": [ + {"topic": "S2", "key": 0, "value": "0,zero,0", "timestamp": 0}, + {"topic": "S2", "key": 0, "value": "0,100,100", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100,100,500", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100,100,100", "timestamp": 0} + ] } ] } \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/scratch.json b/ksql-functional-tests/src/test/resources/query-validation-tests/scratch.json deleted file mode 100644 index ab2585d9489b..000000000000 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/scratch.json +++ /dev/null @@ -1,100 +0,0 @@ -{ - "comments": [ - "When using value_format DELIMITED, we can define VALUE_DELIMITER as custom character." - ], - "tests": [ - { - "name": "select delimited value_format", - "statements": [ - "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED');", - "CREATE STREAM S2 as SELECT id, name, value FROM test;" - ], - "topics": [ - { - "name": "test_topic", - "format": "DELIMITED" - } - ], - "inputs": [ - {"topic": "test_topic", "key": 0, "value": "0,zero,0", "timestamp": 0}, - {"topic": "test_topic", "key": 0, "value": "0,100,100", "timestamp": 0}, - {"topic": "test_topic", "key": 100, "value": "100,100,500", "timestamp": 0}, - {"topic": "test_topic", "key": 100, "value": "100,100,100", "timestamp": 0} - ], - "outputs": [ - {"topic": "S2", "key": 0, "value": "0,zero,0", "timestamp": 0}, - {"topic": "S2", "key": 0, "value": "0,100,100", "timestamp": 0}, - {"topic": "S2", "key": 100, "value": "100,100,500", "timestamp": 0}, - {"topic": "S2", "key": 100, "value": "100,100,100", "timestamp": 0} - ] - }, - { - "name": "validate value_delimiter to be single character", - "statements": [ - "CREATE STREAM TEST WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='<~>');" - ], - "topics": [ - { - "name": "test_topic", - "format": "DELIMITED" - } - ], - "expectedException": { - "type": "io.confluent.ksql.util.KsqlException", - "message": "Error in WITH clause property 'VALUE_DELIMITER': Delimiter must be a single character.\nExample valid value: ';'" - }, - "inputs": [], - "outputs": [] - }, - { - "name": "select delimited value_format with pipe separated values using custom delimiter character", - "statements": [ - "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='|');", - "CREATE STREAM S2 as SELECT id, name, value FROM test;" - ], - "topics": [ - { - "name": "test_topic", - "format": "DELIMITED" - } - ], - "inputs": [ - {"topic": "test_topic", "key": 0, "value": "0|zero|0", "timestamp": 0}, - {"topic": "test_topic", "key": 0, "value": "0|100|100", "timestamp": 0}, - {"topic": "test_topic", "key": 100, "value": "100|100|500", "timestamp": 0}, - {"topic": "test_topic", "key": 100, "value": "100|100|100", "timestamp": 0} - ], - "outputs": [ - {"topic": "S2", "key": 0, "value": "0,zero,0", "timestamp": 0}, - {"topic": "S2", "key": 0, "value": "0,100,100", "timestamp": 0}, - {"topic": "S2", "key": 100, "value": "100,100,500", "timestamp": 0}, - {"topic": "S2", "key": 100, "value": "100,100,100", "timestamp": 0} - ] - }, - { - "name": "Serialize delimited value_format with $", - "statements": [ - "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='|');", - "CREATE STREAM S2 WITH(value_delimiter='$') AS SELECT * FROM test;" - ], - "topics": [ - { - "name": "test_topic", - "format": "DELIMITED" - } - ], - "inputs": [ - {"topic": "test_topic", "key": 0, "value": "0|zero|0", "timestamp": 0}, - {"topic": "test_topic", "key": 0, "value": "0|100|100", "timestamp": 0}, - {"topic": "test_topic", "key": 100, "value": "100|100|500", "timestamp": 0}, - {"topic": "test_topic", "key": 100, "value": "100|100|100", "timestamp": 0} - ], - "outputs": [ - {"topic": "S2", "key": 0, "value": "0$zero$0", "timestamp": 0}, - {"topic": "S2", "key": 0, "value": "0$100$100", "timestamp": 0}, - {"topic": "S2", "key": 100, "value": "100$100$500", "timestamp": 0}, - {"topic": "S2", "key": 100, "value": "100$100$100", "timestamp": 0} - ] - } - ] -} \ No newline at end of file diff --git a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java index d580554d747a..cfd32261bb87 100644 --- a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java +++ b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java @@ -22,6 +22,7 @@ import io.confluent.ksql.execution.expression.tree.Literal; import io.confluent.ksql.properties.with.CommonCreateConfigs; import io.confluent.ksql.properties.with.ConfigMetaData; +import io.confluent.ksql.schema.Operator; import io.confluent.ksql.util.KsqlException; import java.time.format.DateTimeFormatter; import java.util.HashMap; @@ -126,18 +127,38 @@ public Optional getValueDelimiter() { System.out.println("foo"); } if (providedValueDelimiter != null) { - if (providedValueDelimiter.length() != 1) { + if (providedValueDelimiter.isEmpty()) { throw new KsqlException("Error in WITH clause property '" + CommonCreateConfigs.VALUE_DELIMITER_PROPERTY - + "': Delimiter must be a single character." + + "': Delimiter cannot be empty or whitespace." + System.lineSeparator() - + "Example valid value: ';'" + + "For tab or space delimited use 'TAB' or 'SPACE' as the delimeter." ); + } else if (providedValueDelimiter.length() == 1) { + return Optional.of(providedValueDelimiter.charAt(0)); + } else { + Character delim = NAMED_DELIMITERS.get(providedValueDelimiter); + if (delim != null) { + return Optional.of(delim); + } else { + throw new KsqlException("Error in WITH clause property '" + + CommonCreateConfigs.VALUE_DELIMITER_PROPERTY + + "': Delimiter must be a single character, 'TAB' or 'SPACE'." + + System.lineSeparator() + + "Example valid value: ';'" + ); + } } - return Optional.of(providedValueDelimiter.charAt(0)); } else { return Optional.empty(); } } + private static final Map NAMED_DELIMITERS = ImmutableMap + .builder() + .put("TAB", '\t') + .put("SPACE", ' ') + .build(); + + } From 4992a91be668440f20dbb7223639aee92140c6c7 Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Sat, 21 Sep 2019 07:30:45 +0100 Subject: [PATCH 25/46] fix: nit and add more tests --- .../io/confluent/ksql/topic/TopicFactory.java | 4 +- .../delimited-value-format.json | 78 ++++++++++++++++++- .../properties/with/WithClauseProperties.java | 3 +- 3 files changed, 77 insertions(+), 8 deletions(-) diff --git a/ksql-engine/src/main/java/io/confluent/ksql/topic/TopicFactory.java b/ksql-engine/src/main/java/io/confluent/ksql/topic/TopicFactory.java index 90cf4ad5c6f1..fd4d81614ee0 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/topic/TopicFactory.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/topic/TopicFactory.java @@ -43,12 +43,10 @@ public static KsqlTopic create(final CreateSourceProperties properties) { .orElseGet(() -> KeyFormat .nonWindowed(FormatInfo.of(Format.KAFKA))); - Optional valueDelimiter = properties.getValueDelimiter(); - final ValueFormat valueFormat = ValueFormat.of(FormatInfo.of( properties.getValueFormat(), properties.getValueAvroSchemaName(), - valueDelimiter + properties.getValueDelimiter() )); return new KsqlTopic( diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json b/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json index 14619406270c..6d49c6b6dd5d 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json @@ -46,6 +46,60 @@ "inputs": [], "outputs": [] }, + { + "name": "validate delimeter is not empty", + "statements": [ + "CREATE STREAM TEST WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='');" + ], + "topics": [ + { + "name": "test_topic", + "format": "DELIMITED" + } + ], + "expectedException": { + "type": "io.confluent.ksql.util.KsqlException", + "message": "Error in WITH clause property 'VALUE_DELIMITER': Delimiter cannot be empty or whitespace." + }, + "inputs": [], + "outputs": [] + }, + { + "name": "validate delimeter is not a space", + "statements": [ + "CREATE STREAM TEST WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter=' ');" + ], + "topics": [ + { + "name": "test_topic", + "format": "DELIMITED" + } + ], + "expectedException": { + "type": "io.confluent.ksql.util.KsqlException", + "message": "Error in WITH clause property 'VALUE_DELIMITER': Delimiter cannot be empty or whitespace." + }, + "inputs": [], + "outputs": [] + }, + { + "name": "validate delimeter is not a tab character", + "statements": [ + "CREATE STREAM TEST WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='\t');" + ], + "topics": [ + { + "name": "test_topic", + "format": "DELIMITED" + } + ], + "expectedException": { + "type": "io.confluent.ksql.util.KsqlException", + "message": "Error in WITH clause property 'VALUE_DELIMITER': Delimiter cannot be empty or whitespace." + }, + "inputs": [], + "outputs": [] + }, { "name": "select delimited value_format with pipe separated values using custom delimiter character", "statements": [ @@ -72,7 +126,7 @@ ] }, { - "name": "Serialize delimited value_format with $", + "name": "select delimited value_format with $ separated values using custom delimiter character", "statements": [ "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='|');", "CREATE STREAM S2 WITH(value_delimiter='$') AS SELECT * FROM test;" @@ -97,7 +151,7 @@ ] }, { - "name": "select delimited value_format with space separated values using custom delimiter character", + "name": "select delimited value_format with SPACE separated values using custom delimiter character", "statements": [ "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='SPACE');", "CREATE STREAM S2 as SELECT id, name, value FROM test;" @@ -122,7 +176,7 @@ ] }, { - "name": "select delimited value_format with tab separated values using custom delimiter character", + "name": "select delimited value_format with TAB separated values using custom delimiter character", "statements": [ "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='TAB');", "CREATE STREAM S2 as SELECT id, name, value FROM test;" @@ -145,6 +199,24 @@ {"topic": "S2", "key": 100, "value": "100,100,500", "timestamp": 0}, {"topic": "S2", "key": 100, "value": "100,100,100", "timestamp": 0} ] + }, + { + "name": "validate cannot specify delimeter with json format", + "statements": [ + "CREATE STREAM TEST WITH (kafka_topic='test_topic', value_format='JSON', value_delimiter='|');" + ], + "topics": [ + { + "name": "test_topic", + "format": "JSON" + } + ], + "expectedException": { + "type": "io.confluent.ksql.util.KsqlException", + "message": "Delimeter only supported with DELIMITED format" + }, + "inputs": [], + "outputs": [] } ] } \ No newline at end of file diff --git a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java index cfd32261bb87..25a4304ba8ef 100644 --- a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java +++ b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java @@ -22,7 +22,6 @@ import io.confluent.ksql.execution.expression.tree.Literal; import io.confluent.ksql.properties.with.CommonCreateConfigs; import io.confluent.ksql.properties.with.ConfigMetaData; -import io.confluent.ksql.schema.Operator; import io.confluent.ksql.util.KsqlException; import java.time.format.DateTimeFormatter; import java.util.HashMap; @@ -137,7 +136,7 @@ public Optional getValueDelimiter() { } else if (providedValueDelimiter.length() == 1) { return Optional.of(providedValueDelimiter.charAt(0)); } else { - Character delim = NAMED_DELIMITERS.get(providedValueDelimiter); + final Character delim = NAMED_DELIMITERS.get(providedValueDelimiter); if (delim != null) { return Optional.of(delim); } else { From ad6a8d2e5c4052a768ca70e48ff23ebda71a8b42 Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Sat, 21 Sep 2019 07:58:48 +0100 Subject: [PATCH 26/46] fix: update docs, nits and make sure datagen supports TAB and SPACE --- docs/developer-guide/syntax-reference.rst | 14 ++++++++++++-- .../java/io/confluent/ksql/datagen/DataGen.java | 6 ++++-- .../io/confluent/ksql/datagen/DataGenTest.java | 2 -- .../delimited/KsqlDelimitedSerializerTest.java | 2 +- 4 files changed, 17 insertions(+), 7 deletions(-) diff --git a/docs/developer-guide/syntax-reference.rst b/docs/developer-guide/syntax-reference.rst index f1220663d062..2faf0da9f6ce 100644 --- a/docs/developer-guide/syntax-reference.rst +++ b/docs/developer-guide/syntax-reference.rst @@ -362,6 +362,8 @@ The WITH clause supports the following properties: +-------------------------+--------------------------------------------------------------------------------------------+ | VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supports single character to be a delimiter, | | | defaults to ','. | +| | For space and tab delimited values you mnust use the special values 'SPACE' or 'TAB', not | +| | an actual space or tab character. | +-------------------------+--------------------------------------------------------------------------------------------+ | KEY | Optimization hint: If the Kafka message key is also present as a field/column in the Kafka | | | message value, you may set this property to associate the corresponding field/column with | @@ -475,6 +477,8 @@ The WITH clause supports the following properties: +-------------------------+--------------------------------------------------------------------------------------------+ | VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supports single character to be a delimiter, | | | defaults to ','. | +| | For space and tab delimited values you mnust use the special values 'SPACE' or 'TAB', not | +| | an actual space or tab character. | +-------------------------+--------------------------------------------------------------------------------------------+ | KEY | Optimization hint: If the Kafka message key is also present as a field/column in the Kafka | | | message value, you may set this property to associate the corresponding field/column with | @@ -593,7 +597,10 @@ The WITH clause for the result supports the following properties: | | If this property is not set, then the format of the input stream/table is used. | | | For more information, see :ref:`ksql_formats`. | +-------------------------+------------------------------------------------------------------------------------------------------+ -| VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supports single character to be a delimiter, defaults to ','. | +| VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supports single character to be a delimiter, | +| | defaults to ','. | +| | For space and tab delimited values you mnust use the special values 'SPACE' or 'TAB', not | +| | an actual space or tab character. | +-------------------------+------------------------------------------------------------------------------------------------------+ | PARTITIONS | The number of partitions in the backing topic. If this property is not set, then the number | | | of partitions of the input stream/table will be used. In join queries, the property values are taken | @@ -702,7 +709,10 @@ The WITH clause supports the following properties: | | If this property is not set, then the format of the input stream/table is used. | | | For more information, see :ref:`ksql_formats`. | +-------------------------+------------------------------------------------------------------------------------------------------+ -| VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supports single character to be a delimiter, defaults to ','. | +| VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supports single character to be a delimiter, | +| | defaults to ','. | +| | For space and tab delimited values you mnust use the special values 'SPACE' or 'TAB', not | +| | an actual space or tab character. | +-------------------------+------------------------------------------------------------------------------------------------------+ | PARTITIONS | The number of partitions in the backing topic. If this property is not set, then the number | | | of partitions of the input stream/table will be used. In join queries, the property values are taken | diff --git a/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java b/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java index 19a3785cc32a..c6f22cdd0081 100644 --- a/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java +++ b/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java @@ -476,9 +476,11 @@ private static Character parseValueDelimiter(final String valueDelimiterString) if (valueDelimiterString == null) { return null; } else { - if (valueDelimiterString.length() != 1) { + if (valueDelimiterString.length() != 1 && + !(valueDelimiterString.equals("TAB") || valueDelimiterString.equals("SPACE"))) { throw new ArgumentParseException(String.format( - "Invalid value_delimiter; was expecting a single character, got '%s'", + "Invalid value_delimiter; was expecting a single character, 'TAB', or " + + "'SPACE', got '%s'", valueDelimiterString )); } diff --git a/ksql-examples/src/test/java/io/confluent/ksql/datagen/DataGenTest.java b/ksql-examples/src/test/java/io/confluent/ksql/datagen/DataGenTest.java index 944a5cef4b50..0d4cb911e550 100644 --- a/ksql-examples/src/test/java/io/confluent/ksql/datagen/DataGenTest.java +++ b/ksql-examples/src/test/java/io/confluent/ksql/datagen/DataGenTest.java @@ -19,13 +19,11 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsString; -import io.confluent.ksql.datagen.DataGen; import io.confluent.ksql.util.KsqlConfig; import java.util.Properties; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; -import scala.reflect.internal.Trees.Throw; public class DataGenTest { @Rule diff --git a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java index bab40f0851df..2c5900bc0c6b 100644 --- a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java +++ b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java @@ -51,7 +51,7 @@ public class KsqlDelimitedSerializerTest { @Before public void before() { - serializer = new KsqlDelimitedSerializer(CSVFormat.DEFAULT.withDelimiter(',')); + serializer = new KsqlDelimitedSerializer(CSVFormat.DEFAULT.withDelimiter(',')); } @Test From 28b433f8a787e54f5d481c0b31a1c27a54e70496 Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Sat, 21 Sep 2019 08:41:49 +0100 Subject: [PATCH 27/46] fix:checkstyle nit --- .../src/main/java/io/confluent/ksql/datagen/DataGen.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java b/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java index c6f22cdd0081..9ed683d04b06 100644 --- a/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java +++ b/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java @@ -476,8 +476,8 @@ private static Character parseValueDelimiter(final String valueDelimiterString) if (valueDelimiterString == null) { return null; } else { - if (valueDelimiterString.length() != 1 && - !(valueDelimiterString.equals("TAB") || valueDelimiterString.equals("SPACE"))) { + if (valueDelimiterString.length() != 1 + && !(valueDelimiterString.equals("TAB") || valueDelimiterString.equals("SPACE"))) { throw new ArgumentParseException(String.format( "Invalid value_delimiter; was expecting a single character, 'TAB', or " + "'SPACE', got '%s'", From e2a690a17404717cccac513a8dae13328f847729 Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Sat, 21 Sep 2019 09:06:05 +0100 Subject: [PATCH 28/46] fix: Removed some debug code --- .../ksql/parser/properties/with/WithClauseProperties.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java index 25a4304ba8ef..bbd293f11856 100644 --- a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java +++ b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java @@ -122,9 +122,6 @@ private static void throwOnUnknownProperty( public Optional getValueDelimiter() { final String providedValueDelimiter = getString(CommonCreateConfigs.VALUE_DELIMITER_PROPERTY); - if (providedValueDelimiter != null && providedValueDelimiter.equalsIgnoreCase("$")) { - System.out.println("foo"); - } if (providedValueDelimiter != null) { if (providedValueDelimiter.isEmpty()) { throw new KsqlException("Error in WITH clause property '" From 4a215d51bff94a4133bf10f1568c8ca24c92afc5 Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Sat, 21 Sep 2019 09:10:18 +0100 Subject: [PATCH 29/46] fix: nit --- .../src/main/java/io/confluent/ksql/datagen/DataGen.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java b/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java index 9ed683d04b06..563b213c8263 100644 --- a/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java +++ b/ksql-examples/src/main/java/io/confluent/ksql/datagen/DataGen.java @@ -476,8 +476,8 @@ private static Character parseValueDelimiter(final String valueDelimiterString) if (valueDelimiterString == null) { return null; } else { - if (valueDelimiterString.length() != 1 - && !(valueDelimiterString.equals("TAB") || valueDelimiterString.equals("SPACE"))) { + if (!(valueDelimiterString.length() == 1 || valueDelimiterString.equals("TAB") + || valueDelimiterString.equals("SPACE"))) { throw new ArgumentParseException(String.format( "Invalid value_delimiter; was expecting a single character, 'TAB', or " + "'SPACE', got '%s'", From 543fdeecc5b7d65dce3852a0e6ac0285b38c7be4 Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Tue, 24 Sep 2019 14:32:16 +0100 Subject: [PATCH 30/46] fix:nit --- .../properties/with/WithClauseProperties.java | 40 +++++++++---------- 1 file changed, 19 insertions(+), 21 deletions(-) diff --git a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java index bbd293f11856..c6f22450d7b0 100644 --- a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java +++ b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java @@ -122,31 +122,30 @@ private static void throwOnUnknownProperty( public Optional getValueDelimiter() { final String providedValueDelimiter = getString(CommonCreateConfigs.VALUE_DELIMITER_PROPERTY); - if (providedValueDelimiter != null) { - if (providedValueDelimiter.isEmpty()) { + if (providedValueDelimiter == null) { + return Optional.empty(); + } + if (providedValueDelimiter.isEmpty()) { + throw new KsqlException("Error in WITH clause property '" + + CommonCreateConfigs.VALUE_DELIMITER_PROPERTY + + "': Delimiter cannot be empty or whitespace." + + System.lineSeparator() + + "For tab or space delimited use 'TAB' or 'SPACE' as the delimeter." + ); + } else if (providedValueDelimiter.length() == 1) { + return Optional.of(providedValueDelimiter.charAt(0)); + } else { + final Character delim = NAMED_DELIMITERS.get(providedValueDelimiter); + if (delim != null) { + return Optional.of(delim); + } else { throw new KsqlException("Error in WITH clause property '" + CommonCreateConfigs.VALUE_DELIMITER_PROPERTY - + "': Delimiter cannot be empty or whitespace." + + "': Delimiter must be a single character, 'TAB' or 'SPACE'." + System.lineSeparator() - + "For tab or space delimited use 'TAB' or 'SPACE' as the delimeter." + + "Example valid value: ';'" ); - } else if (providedValueDelimiter.length() == 1) { - return Optional.of(providedValueDelimiter.charAt(0)); - } else { - final Character delim = NAMED_DELIMITERS.get(providedValueDelimiter); - if (delim != null) { - return Optional.of(delim); - } else { - throw new KsqlException("Error in WITH clause property '" - + CommonCreateConfigs.VALUE_DELIMITER_PROPERTY - + "': Delimiter must be a single character, 'TAB' or 'SPACE'." - + System.lineSeparator() - + "Example valid value: ';'" - ); - } } - } else { - return Optional.empty(); } } @@ -156,5 +155,4 @@ public Optional getValueDelimiter() { .put("SPACE", ' ') .build(); - } From d2699eff5b5e3da9686e175f23d6cde136b4cfc5 Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Tue, 24 Sep 2019 16:23:02 +0100 Subject: [PATCH 31/46] fix:nits --- docs/developer-guide/syntax-reference.rst | 8 +++---- .../KsqlDelimitedDeserializerTest.java | 2 +- .../KsqlDelimitedSerializerTest.java | 24 +++---------------- 3 files changed, 8 insertions(+), 26 deletions(-) diff --git a/docs/developer-guide/syntax-reference.rst b/docs/developer-guide/syntax-reference.rst index 2faf0da9f6ce..84b236b98ff8 100644 --- a/docs/developer-guide/syntax-reference.rst +++ b/docs/developer-guide/syntax-reference.rst @@ -362,7 +362,7 @@ The WITH clause supports the following properties: +-------------------------+--------------------------------------------------------------------------------------------+ | VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supports single character to be a delimiter, | | | defaults to ','. | -| | For space and tab delimited values you mnust use the special values 'SPACE' or 'TAB', not | +| | For space and tab delimited values you must use the special values 'SPACE' or 'TAB', not | | | an actual space or tab character. | +-------------------------+--------------------------------------------------------------------------------------------+ | KEY | Optimization hint: If the Kafka message key is also present as a field/column in the Kafka | @@ -477,7 +477,7 @@ The WITH clause supports the following properties: +-------------------------+--------------------------------------------------------------------------------------------+ | VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supports single character to be a delimiter, | | | defaults to ','. | -| | For space and tab delimited values you mnust use the special values 'SPACE' or 'TAB', not | +| | For space and tab delimited values you must use the special values 'SPACE' or 'TAB', not | | | an actual space or tab character. | +-------------------------+--------------------------------------------------------------------------------------------+ | KEY | Optimization hint: If the Kafka message key is also present as a field/column in the Kafka | @@ -599,7 +599,7 @@ The WITH clause for the result supports the following properties: +-------------------------+------------------------------------------------------------------------------------------------------+ | VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supports single character to be a delimiter, | | | defaults to ','. | -| | For space and tab delimited values you mnust use the special values 'SPACE' or 'TAB', not | +| | For space and tab delimited values you must use the special values 'SPACE' or 'TAB', not | | | an actual space or tab character. | +-------------------------+------------------------------------------------------------------------------------------------------+ | PARTITIONS | The number of partitions in the backing topic. If this property is not set, then the number | @@ -711,7 +711,7 @@ The WITH clause supports the following properties: +-------------------------+------------------------------------------------------------------------------------------------------+ | VALUE_DELIMITER | Used when VALUE_FORMAT='DELIMITED'. Supports single character to be a delimiter, | | | defaults to ','. | -| | For space and tab delimited values you mnust use the special values 'SPACE' or 'TAB', not | +| | For space and tab delimited values you must use the special values 'SPACE' or 'TAB', not | | | an actual space or tab character. | +-------------------------+------------------------------------------------------------------------------------------------------+ | PARTITIONS | The number of partitions in the backing topic. If this property is not set, then the number | diff --git a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java index ad380ab5cb20..e71fc30a12c8 100644 --- a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java +++ b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java @@ -59,7 +59,7 @@ public class KsqlDelimitedDeserializerTest { private KsqlDelimitedDeserializer deserializer; @Before - public void before() { + public void setUp() { deserializer = new KsqlDelimitedDeserializer(ORDER_SCHEMA, CSVFormat.DEFAULT); } diff --git a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java index 2c5900bc0c6b..1cff6ef577f5 100644 --- a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java +++ b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java @@ -50,7 +50,7 @@ public class KsqlDelimitedSerializerTest { private KsqlDelimitedSerializer serializer; @Before - public void before() { + public void setUp() { serializer = new KsqlDelimitedSerializer(CSVFormat.DEFAULT.withDelimiter(',')); } @@ -105,8 +105,6 @@ public void shouldSerializedTopLevelPrimitiveIfValueHasOneField() { .field("id", Schema.OPTIONAL_INT64_SCHEMA) .build(); - final Serializer serializer = createSerializer(); - final Struct value = new Struct(schema) .put("id", 10L); @@ -124,8 +122,6 @@ public void shouldSerializeDecimal() { .field("id", DecimalUtil.builder(4, 2).build()) .build(); - final Serializer serializer = createSerializer(); - final Struct value = new Struct(schema) .put("id", new BigDecimal("11.12")); @@ -143,8 +139,6 @@ public void shouldSerializeDecimalWithPaddedZeros() { .field("id", DecimalUtil.builder(4, 2).build()) .build(); - final Serializer serializer = createSerializer(); - final Struct value = new Struct(schema) .put("id", new BigDecimal("1.12")); @@ -162,8 +156,6 @@ public void shouldSerializeZeroDecimalWithPaddedZeros() { .field("id", DecimalUtil.builder(4, 2).build()) .build(); - final Serializer serializer = createSerializer(); - final Struct value = new Struct(schema) .put("id", BigDecimal.ZERO); @@ -181,8 +173,6 @@ public void shouldSerializeOneHalfDecimalWithPaddedZeros() { .field("id", DecimalUtil.builder(4, 2).build()) .build(); - final Serializer serializer = createSerializer(); - final Struct value = new Struct(schema) .put("id", new BigDecimal(0.5)); @@ -200,8 +190,6 @@ public void shouldSerializeNegativeOneHalfDecimalWithPaddedZeros() { .field("id", DecimalUtil.builder(4, 2).build()) .build(); - final Serializer serializer = createSerializer(); - final Struct value = new Struct(schema) .put("id", new BigDecimal(-0.5)); @@ -219,8 +207,6 @@ public void shouldSerializeNegativeDecimalWithPaddedZeros() { .field("id", DecimalUtil.builder(4, 2).build()) .build(); - final Serializer serializer = createSerializer(); - final Struct value = new Struct(schema) .put("id", new BigDecimal("-1.12")); @@ -241,7 +227,7 @@ public void shouldSerializeRowCorrectlyWithBarDelimeter() { shouldSerializeRowCorrectlyWithNonDefaultDelimeter('|'); } - private void shouldSerializeRowCorrectlyWithNonDefaultDelimeter(char delimiter) { + private void shouldSerializeRowCorrectlyWithNonDefaultDelimeter(final char delimiter) { // Given: final Struct data = new Struct(SCHEMA) .put("ORDERTIME", 1511897796092L) @@ -249,7 +235,7 @@ private void shouldSerializeRowCorrectlyWithNonDefaultDelimeter(char delimiter) .put("ITEMID", "item_1") .put("ORDERUNITS", 10.0); - KsqlDelimitedSerializer serializer = + final KsqlDelimitedSerializer serializer = new KsqlDelimitedSerializer(CSVFormat.DEFAULT.withDelimiter(delimiter)); // When: @@ -328,8 +314,4 @@ public void shouldThrowOnStructField() { serializer.serialize("t1", data); } - private KsqlDelimitedSerializer createSerializer() { - return new KsqlDelimitedSerializer(CSVFormat.DEFAULT.withDelimiter(',')); - } - } From 49426b8b027510ebe6d5380ad47d88bf345fe243 Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Tue, 24 Sep 2019 17:02:28 +0100 Subject: [PATCH 32/46] fix:nits --- .../java/io/confluent/ksql/serde/KeyFormat.java | 12 ------------ .../io/confluent/ksql/serde/ValueFormatTest.java | 6 ++++-- .../ksql/test/tools/TestCaseBuilderUtil.java | 8 +++++--- .../delimited-value-format.json | 16 ++++++++-------- 4 files changed, 17 insertions(+), 25 deletions(-) diff --git a/ksql-common/src/main/java/io/confluent/ksql/serde/KeyFormat.java b/ksql-common/src/main/java/io/confluent/ksql/serde/KeyFormat.java index 8e7dc92db60d..ad9cc5de821f 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/serde/KeyFormat.java +++ b/ksql-common/src/main/java/io/confluent/ksql/serde/KeyFormat.java @@ -34,18 +34,6 @@ public static KeyFormat nonWindowed(final FormatInfo format) { return new KeyFormat(format, Optional.empty()); } - public static KeyFormat windowed( - final Format format, - final Optional avroSchemaName, - final Optional delimiter, - final WindowInfo windowInfo - ) { - return new KeyFormat( - FormatInfo.of(format, avroSchemaName, delimiter), - Optional.of(windowInfo) - ); - } - public static KeyFormat windowed( final FormatInfo format, final WindowInfo windowInfo diff --git a/ksql-common/src/test/java/io/confluent/ksql/serde/ValueFormatTest.java b/ksql-common/src/test/java/io/confluent/ksql/serde/ValueFormatTest.java index 21adf12b0e18..3e78a9f1ff2f 100644 --- a/ksql-common/src/test/java/io/confluent/ksql/serde/ValueFormatTest.java +++ b/ksql-common/src/test/java/io/confluent/ksql/serde/ValueFormatTest.java @@ -30,8 +30,10 @@ public class ValueFormatTest { private static final FormatInfo FORMAT_INFO = FormatInfo.of( - AVRO, Optional.of("something"), - Optional.empty()); + AVRO, + Optional.of("something"), + Optional.empty() + ); @Test public void shouldThrowNPEs() { diff --git a/ksql-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestCaseBuilderUtil.java b/ksql-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestCaseBuilderUtil.java index e26c1daef9fa..51821a54f0e8 100644 --- a/ksql-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestCaseBuilderUtil.java +++ b/ksql-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestCaseBuilderUtil.java @@ -229,9 +229,11 @@ private static SerdeSupplier getKeySerdeSupplier(final Optional w ? WindowType.SESSION.name() : WindowType.TUMBLING.name()); final KeyFormat windowKeyFormat = KeyFormat.windowed( - Format.KAFKA, - Optional.empty(), - Optional.empty(), + FormatInfo.of( + Format.KAFKA, + Optional.empty(), + Optional.empty() + ), WindowInfo.of( windowType, windowType == WindowType.SESSION diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json b/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json index 6d49c6b6dd5d..152852cdd2f0 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json @@ -40,7 +40,7 @@ } ], "expectedException": { - "type": "io.confluent.ksql.util.KsqlException", + "type": "io.confluent.ksql.util.KsqlStatementException", "message": "Error in WITH clause property 'VALUE_DELIMITER': Delimiter must be a single character, 'TAB' or 'SPACE'.\nExample valid value: ';'" }, "inputs": [], @@ -58,8 +58,8 @@ } ], "expectedException": { - "type": "io.confluent.ksql.util.KsqlException", - "message": "Error in WITH clause property 'VALUE_DELIMITER': Delimiter cannot be empty or whitespace." + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Error in WITH clause property 'VALUE_DELIMITER': Delimiter cannot be empty or whitespace.\nFor tab or space delimited use 'TAB' or 'SPACE' as the delimeter." }, "inputs": [], "outputs": [] @@ -76,8 +76,8 @@ } ], "expectedException": { - "type": "io.confluent.ksql.util.KsqlException", - "message": "Error in WITH clause property 'VALUE_DELIMITER': Delimiter cannot be empty or whitespace." + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Error in WITH clause property 'VALUE_DELIMITER': Delimiter cannot be empty or whitespace.\nFor tab or space delimited use 'TAB' or 'SPACE' as the delimeter." }, "inputs": [], "outputs": [] @@ -94,8 +94,8 @@ } ], "expectedException": { - "type": "io.confluent.ksql.util.KsqlException", - "message": "Error in WITH clause property 'VALUE_DELIMITER': Delimiter cannot be empty or whitespace." + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Error in WITH clause property 'VALUE_DELIMITER': Delimiter cannot be empty or whitespace.\nFor tab or space delimited use 'TAB' or 'SPACE' as the delimeter." }, "inputs": [], "outputs": [] @@ -212,7 +212,7 @@ } ], "expectedException": { - "type": "io.confluent.ksql.util.KsqlException", + "type": "io.confluent.ksql.util.KsqlStatementException", "message": "Delimeter only supported with DELIMITED format" }, "inputs": [], From 53cdf5f0178438ede4b17ae6dad27c2b83795fa7 Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Tue, 24 Sep 2019 17:14:20 +0100 Subject: [PATCH 33/46] fix:nits --- .../properties/with/WithClauseProperties.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java index c6f22450d7b0..b27770e665aa 100644 --- a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java +++ b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java @@ -132,20 +132,20 @@ public Optional getValueDelimiter() { + System.lineSeparator() + "For tab or space delimited use 'TAB' or 'SPACE' as the delimeter." ); - } else if (providedValueDelimiter.length() == 1) { + } + if (providedValueDelimiter.length() == 1) { return Optional.of(providedValueDelimiter.charAt(0)); } else { final Character delim = NAMED_DELIMITERS.get(providedValueDelimiter); if (delim != null) { return Optional.of(delim); - } else { - throw new KsqlException("Error in WITH clause property '" - + CommonCreateConfigs.VALUE_DELIMITER_PROPERTY - + "': Delimiter must be a single character, 'TAB' or 'SPACE'." - + System.lineSeparator() - + "Example valid value: ';'" - ); } + throw new KsqlException("Error in WITH clause property '" + + CommonCreateConfigs.VALUE_DELIMITER_PROPERTY + + "': Delimiter must be a single character, 'TAB' or 'SPACE'." + + System.lineSeparator() + + "Example valid value: ';'" + ); } } From 1e421eca6712dac2daa6951b5fbdc98488b9a2e4 Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Tue, 24 Sep 2019 19:48:28 +0100 Subject: [PATCH 34/46] Improved CommonCreateConfigs description for value_delimiter --- .../confluent/ksql/properties/with/CommonCreateConfigs.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java b/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java index 37563ce19c3e..c9ab45bce31e 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java +++ b/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java @@ -126,7 +126,10 @@ static void addToConfigDef( ConfigDef.Type.STRING, null, Importance.LOW, - "The delimiter to use when VALUE_FORMAT='DELIMITED'."); + "The delimiter to use when VALUE_FORMAT='DELIMITED'. Supports single " + + "character to be a delimiter, defaults to ','. For space and tab delimited values " + + "you must use the special values 'SPACE' or 'TAB', not an actual space or tab " + + "character."); } private CommonCreateConfigs() { From 2f05ad3d2fdf963acaef4705fa3b5c3bb86a8b49 Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Tue, 24 Sep 2019 20:24:47 +0100 Subject: [PATCH 35/46] Use validator to check value_delimiter is not empty --- .../ksql/properties/with/CommonCreateConfigs.java | 1 + .../query-validation-tests/delimited-value-format.json | 6 +++--- .../ksql/parser/properties/with/WithClauseProperties.java | 8 -------- 3 files changed, 4 insertions(+), 11 deletions(-) diff --git a/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java b/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java index c9ab45bce31e..92a47a97b8cd 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java +++ b/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java @@ -125,6 +125,7 @@ static void addToConfigDef( VALUE_DELIMITER_PROPERTY, ConfigDef.Type.STRING, null, + new NonEmptyString(), Importance.LOW, "The delimiter to use when VALUE_FORMAT='DELIMITED'. Supports single " + "character to be a delimiter, defaults to ','. For space and tab delimited values " diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json b/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json index 152852cdd2f0..116ec63fa0a5 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json @@ -59,7 +59,7 @@ ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "Error in WITH clause property 'VALUE_DELIMITER': Delimiter cannot be empty or whitespace.\nFor tab or space delimited use 'TAB' or 'SPACE' as the delimeter." + "message": "Failed to prepare statement: Invalid value for property VALUE_DELIMITER: String must be non-empty\nStatement: CREATE STREAM TEST WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='');" }, "inputs": [], "outputs": [] @@ -77,7 +77,7 @@ ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "Error in WITH clause property 'VALUE_DELIMITER': Delimiter cannot be empty or whitespace.\nFor tab or space delimited use 'TAB' or 'SPACE' as the delimeter." + "message": "Failed to prepare statement: Invalid value for property VALUE_DELIMITER: String must be non-empty\nStatement: CREATE STREAM TEST WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter=' ');" }, "inputs": [], "outputs": [] @@ -95,7 +95,7 @@ ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "Error in WITH clause property 'VALUE_DELIMITER': Delimiter cannot be empty or whitespace.\nFor tab or space delimited use 'TAB' or 'SPACE' as the delimeter." + "message": "Failed to prepare statement: Invalid value for property VALUE_DELIMITER: String must be non-empty\nStatement: CREATE STREAM TEST WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='\t');" }, "inputs": [], "outputs": [] diff --git a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java index b27770e665aa..507e75215ad4 100644 --- a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java +++ b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java @@ -125,14 +125,6 @@ public Optional getValueDelimiter() { if (providedValueDelimiter == null) { return Optional.empty(); } - if (providedValueDelimiter.isEmpty()) { - throw new KsqlException("Error in WITH clause property '" - + CommonCreateConfigs.VALUE_DELIMITER_PROPERTY - + "': Delimiter cannot be empty or whitespace." - + System.lineSeparator() - + "For tab or space delimited use 'TAB' or 'SPACE' as the delimeter." - ); - } if (providedValueDelimiter.length() == 1) { return Optional.of(providedValueDelimiter.charAt(0)); } else { From 8cf99d3113496338368041813ca03a3db8aaa3bd Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Tue, 24 Sep 2019 20:32:35 +0100 Subject: [PATCH 36/46] Use concatenated delimiter named values in error message --- .../delimited-value-format.json | 2 +- .../properties/with/WithClauseProperties.java | 18 +++++++++++++++++- 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json b/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json index 116ec63fa0a5..a85dfff115ca 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json @@ -41,7 +41,7 @@ ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "Error in WITH clause property 'VALUE_DELIMITER': Delimiter must be a single character, 'TAB' or 'SPACE'.\nExample valid value: ';'" + "message": "Error in WITH clause property 'VALUE_DELIMITER': Delimiter must be a single character or TAB, SPACE\nExample valid value: ';'" }, "inputs": [], "outputs": [] diff --git a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java index 507e75215ad4..88979c993956 100644 --- a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java +++ b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java @@ -25,6 +25,7 @@ import io.confluent.ksql.util.KsqlException; import java.time.format.DateTimeFormatter; import java.util.HashMap; +import java.util.Iterator; import java.util.Map; import java.util.Objects; import java.util.Optional; @@ -134,7 +135,7 @@ public Optional getValueDelimiter() { } throw new KsqlException("Error in WITH clause property '" + CommonCreateConfigs.VALUE_DELIMITER_PROPERTY - + "': Delimiter must be a single character, 'TAB' or 'SPACE'." + + "': Delimiter must be a single character or " + NAMED_DELIMITERS_STRING + System.lineSeparator() + "Example valid value: ';'" ); @@ -147,4 +148,19 @@ public Optional getValueDelimiter() { .put("SPACE", ' ') .build(); + private static final String NAMED_DELIMITERS_STRING = getNamedDelimitersString(); + + private static String getNamedDelimitersString() { + final StringBuilder sb = new StringBuilder(); + final Iterator iter = NAMED_DELIMITERS.keySet().iterator(); + while (iter.hasNext()) { + sb.append(iter.next()); + if (iter.hasNext()) { + sb.append(", "); + } + } + return sb.toString(); + } + + } From 1d7a223f61275be803288fd51f806f008ffed526 Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Tue, 24 Sep 2019 21:18:20 +0100 Subject: [PATCH 37/46] Use value delimiter from source if sink doesn't specify one --- .../io/confluent/ksql/analyzer/Analyzer.java | 17 +++++++++- .../delimited-value-format.json | 26 +++++++-------- .../resources/query-validation-tests/foo.json | 32 +++++++++++++++++++ 3 files changed, 61 insertions(+), 14 deletions(-) create mode 100644 ksql-functional-tests/src/test/resources/query-validation-tests/foo.json diff --git a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java index dcfcc8f95b2c..53a08d869019 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java @@ -182,7 +182,7 @@ private void analyzeNonStdOutSink(final Sink sink) { final ValueFormat valueFormat = ValueFormat.of(FormatInfo.of( getValueFormat(sink), sink.getProperties().getValueAvroSchemaName(), - sink.getProperties().getValueDelimiter() + getValueDelimiter(sink) )); final KsqlTopic intoKsqlTopic = new KsqlTopic( @@ -279,6 +279,21 @@ private Format getValueFormat(final Sink sink) { .getFormat()); } + private Optional getValueDelimiter(final Sink sink) { + if (sink.getProperties().getValueDelimiter().isPresent()) { + return sink.getProperties().getValueDelimiter(); + } else { + return analysis + .getFromDataSources() + .get(0) + .getDataSource() + .getKsqlTopic() + .getValueFormat() + .getFormatInfo() + .getDelimiter(); + } + } + @Override protected AstNode visitQuery( diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json b/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json index a85dfff115ca..ffe00884b908 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json @@ -101,7 +101,7 @@ "outputs": [] }, { - "name": "select delimited value_format with pipe separated values using custom delimiter character", + "name": "select delimited value_format with pipe separated values - should take source delimiter for sink", "statements": [ "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='|');", "CREATE STREAM S2 as SELECT id, name, value FROM test;" @@ -119,10 +119,10 @@ {"topic": "test_topic", "key": 100, "value": "100|100|100", "timestamp": 0} ], "outputs": [ - {"topic": "S2", "key": 0, "value": "0,zero,0", "timestamp": 0}, - {"topic": "S2", "key": 0, "value": "0,100,100", "timestamp": 0}, - {"topic": "S2", "key": 100, "value": "100,100,500", "timestamp": 0}, - {"topic": "S2", "key": 100, "value": "100,100,100", "timestamp": 0} + {"topic": "S2", "key": 0, "value": "0|zero|0", "timestamp": 0}, + {"topic": "S2", "key": 0, "value": "0|100|100", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100|100|500", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100|100|100", "timestamp": 0} ] }, { @@ -169,10 +169,10 @@ {"topic": "test_topic", "key": 100, "value": "100 100 100", "timestamp": 0} ], "outputs": [ - {"topic": "S2", "key": 0, "value": "0,zero,0", "timestamp": 0}, - {"topic": "S2", "key": 0, "value": "0,100,100", "timestamp": 0}, - {"topic": "S2", "key": 100, "value": "100,100,500", "timestamp": 0}, - {"topic": "S2", "key": 100, "value": "100,100,100", "timestamp": 0} + {"topic": "S2", "key": 0, "value": "0 zero 0", "timestamp": 0}, + {"topic": "S2", "key": 0, "value": "0 100 100", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100 100 500", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100 100 100", "timestamp": 0} ] }, { @@ -194,10 +194,10 @@ {"topic": "test_topic", "key": 100, "value": "100\t100\t100", "timestamp": 0} ], "outputs": [ - {"topic": "S2", "key": 0, "value": "0,zero,0", "timestamp": 0}, - {"topic": "S2", "key": 0, "value": "0,100,100", "timestamp": 0}, - {"topic": "S2", "key": 100, "value": "100,100,500", "timestamp": 0}, - {"topic": "S2", "key": 100, "value": "100,100,100", "timestamp": 0} + {"topic": "S2", "key": 0, "value": "0\tzero\t0", "timestamp": 0}, + {"topic": "S2", "key": 0, "value": "0\t100\t100", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100\t100\t500", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100\t100\t100", "timestamp": 0} ] }, { diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/foo.json b/ksql-functional-tests/src/test/resources/query-validation-tests/foo.json new file mode 100644 index 000000000000..7a7139e032ad --- /dev/null +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/foo.json @@ -0,0 +1,32 @@ +{ + "comments": [ + "When using value_format DELIMITED, we can define VALUE_DELIMITER as custom character." + ], + "tests": [ + { + "name": "select delimited value_format with pipe separated values using custom delimiter character", + "statements": [ + "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='|');", + "CREATE STREAM S2 as SELECT id, name, value FROM test;" + ], + "topics": [ + { + "name": "test_topic", + "format": "DELIMITED" + } + ], + "inputs": [ + {"topic": "test_topic", "key": 0, "value": "0|zero|0", "timestamp": 0}, + {"topic": "test_topic", "key": 0, "value": "0|100|100", "timestamp": 0}, + {"topic": "test_topic", "key": 100, "value": "100|100|500", "timestamp": 0}, + {"topic": "test_topic", "key": 100, "value": "100|100|100", "timestamp": 0} + ], + "outputs": [ + {"topic": "S2", "key": 0, "value": "0,zero,0", "timestamp": 0}, + {"topic": "S2", "key": 0, "value": "0,100,100", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100,100,500", "timestamp": 0}, + {"topic": "S2", "key": 100, "value": "100,100,100", "timestamp": 0} + ] + } + ] +} \ No newline at end of file From 33e7358f9bd84472ed56ffcd3b4f4015331da43e Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Tue, 24 Sep 2019 22:30:57 +0100 Subject: [PATCH 38/46] Remove file that shouldn't be there --- .../resources/query-validation-tests/foo.json | 32 ------------------- 1 file changed, 32 deletions(-) delete mode 100644 ksql-functional-tests/src/test/resources/query-validation-tests/foo.json diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/foo.json b/ksql-functional-tests/src/test/resources/query-validation-tests/foo.json deleted file mode 100644 index 7a7139e032ad..000000000000 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/foo.json +++ /dev/null @@ -1,32 +0,0 @@ -{ - "comments": [ - "When using value_format DELIMITED, we can define VALUE_DELIMITER as custom character." - ], - "tests": [ - { - "name": "select delimited value_format with pipe separated values using custom delimiter character", - "statements": [ - "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='|');", - "CREATE STREAM S2 as SELECT id, name, value FROM test;" - ], - "topics": [ - { - "name": "test_topic", - "format": "DELIMITED" - } - ], - "inputs": [ - {"topic": "test_topic", "key": 0, "value": "0|zero|0", "timestamp": 0}, - {"topic": "test_topic", "key": 0, "value": "0|100|100", "timestamp": 0}, - {"topic": "test_topic", "key": 100, "value": "100|100|500", "timestamp": 0}, - {"topic": "test_topic", "key": 100, "value": "100|100|100", "timestamp": 0} - ], - "outputs": [ - {"topic": "S2", "key": 0, "value": "0,zero,0", "timestamp": 0}, - {"topic": "S2", "key": 0, "value": "0,100,100", "timestamp": 0}, - {"topic": "S2", "key": 100, "value": "100,100,500", "timestamp": 0}, - {"topic": "S2", "key": 100, "value": "100,100,100", "timestamp": 0} - ] - } - ] -} \ No newline at end of file From 0bbbb37a3a48b7cffd8d50a8baf4cb6f97b6479e Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Wed, 25 Sep 2019 15:42:41 +0100 Subject: [PATCH 39/46] Created delimiter class --- .../ksql/configdef/ConfigValidators.java | 18 +++ .../properties/with/CommonCreateConfigs.java | 3 +- .../io/confluent/ksql/serde/Delimiter.java | 109 ++++++++++++++++++ .../io/confluent/ksql/serde/FormatInfo.java | 8 +- .../confluent/ksql/serde/FormatInfoTest.java | 10 +- .../io/confluent/ksql/analyzer/Analyzer.java | 3 +- .../delimited-value-format.json | 8 +- .../with/CreateSourceAsProperties.java | 6 + .../with/CreateSourceProperties.java | 6 + .../properties/with/WithClauseProperties.java | 45 -------- .../delimited/KsqlDelimitedSerdeFactory.java | 7 +- .../KsqlDelimitedSerdeFactoryTest.java | 3 +- 12 files changed, 162 insertions(+), 64 deletions(-) create mode 100644 ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java diff --git a/ksql-common/src/main/java/io/confluent/ksql/configdef/ConfigValidators.java b/ksql-common/src/main/java/io/confluent/ksql/configdef/ConfigValidators.java index 5aecc1c53b16..02132480e372 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/configdef/ConfigValidators.java +++ b/ksql-common/src/main/java/io/confluent/ksql/configdef/ConfigValidators.java @@ -20,6 +20,7 @@ import java.util.Arrays; import java.util.EnumSet; import java.util.List; +import java.util.function.Function; import java.util.stream.Collectors; import org.apache.kafka.common.config.ConfigDef.Validator; import org.apache.kafka.common.config.ConfigException; @@ -32,6 +33,23 @@ public final class ConfigValidators { private ConfigValidators() { } + public static void parses( + final String name, + final Object val, + final Function parser) { + try { + if (val == null) { + return; + } + if (!(val instanceof String)) { + throw new ConfigException(name, val, "Must be String"); + } + parser.apply((String)val); + } catch (Exception e) { + throw new ConfigException(name, val, e.getMessage()); + } + } + public static > Validator enumValues(final Class enumClass) { final String[] enumValues = EnumSet.allOf(enumClass) .stream() diff --git a/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java b/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java index 92a47a97b8cd..a1d506600872 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java +++ b/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java @@ -16,6 +16,7 @@ package io.confluent.ksql.properties.with; import io.confluent.ksql.configdef.ConfigValidators; +import io.confluent.ksql.serde.Delimiter; import io.confluent.ksql.serde.Format; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; @@ -125,7 +126,7 @@ static void addToConfigDef( VALUE_DELIMITER_PROPERTY, ConfigDef.Type.STRING, null, - new NonEmptyString(), + (name, val) -> ConfigValidators.parses(name, val, Delimiter::parse), Importance.LOW, "The delimiter to use when VALUE_FORMAT='DELIMITED'. Supports single " + "character to be a delimiter, defaults to ','. For space and tab delimited values " diff --git a/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java b/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java new file mode 100644 index 000000000000..4cc4f91fe21c --- /dev/null +++ b/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java @@ -0,0 +1,109 @@ +/* + * Copyright 2019 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.serde; + +import com.google.common.collect.ImmutableMap; +import com.google.errorprone.annotations.Immutable; +import java.util.Iterator; +import java.util.Map; +import java.util.Objects; + +@Immutable +public final class Delimiter { + + public final char delimiter; + + public static Delimiter of(final char ch) { + return new Delimiter(ch); + } + + public static Delimiter of(final String str) { + return new Delimiter(parse(str)); + } + + private Delimiter(final char delimiter) { + this.delimiter = delimiter; + } + + public static Character parse(final String providedValueDelimiter) { + if (providedValueDelimiter == null) { + return null; + } + if (providedValueDelimiter.trim().isEmpty()) { + throw new IllegalArgumentException( + "Delimiter cannot be empty" + + System.lineSeparator() + + "Example valid value: ';'" + ); + } + if (providedValueDelimiter.length() == 1) { + return providedValueDelimiter.charAt(0); + } else { + final Character delim = NAMED_DELIMITERS.get(providedValueDelimiter); + if (delim != null) { + return delim; + } + throw new IllegalArgumentException( + "Delimiter must be a single character or " + + NAMED_DELIMITERS_STRING + + System.lineSeparator() + + "Example valid value: ';'" + ); + } + } + + private static final Map NAMED_DELIMITERS = ImmutableMap + .builder() + .put("TAB", '\t') + .put("SPACE", ' ') + .build(); + + private static final String NAMED_DELIMITERS_STRING = getNamedDelimitersString(); + + private static String getNamedDelimitersString() { + final StringBuilder sb = new StringBuilder(); + final Iterator iter = NAMED_DELIMITERS.keySet().iterator(); + while (iter.hasNext()) { + sb.append(iter.next()); + if (iter.hasNext()) { + sb.append(", "); + } + } + return sb.toString(); + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final Delimiter delimiter1 = (Delimiter) o; + return delimiter == delimiter1.delimiter; + } + + @Override + public int hashCode() { + return Objects.hash(delimiter); + } + + @Override + public String toString() { + return String.valueOf(delimiter); + } +} diff --git a/ksql-common/src/main/java/io/confluent/ksql/serde/FormatInfo.java b/ksql-common/src/main/java/io/confluent/ksql/serde/FormatInfo.java index 460fe879acc4..8127b05e8772 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/serde/FormatInfo.java +++ b/ksql-common/src/main/java/io/confluent/ksql/serde/FormatInfo.java @@ -28,7 +28,7 @@ public final class FormatInfo { private final Format format; private final Optional avroFullSchemaName; - private final Optional delimiter; + private final Optional delimiter; public static FormatInfo of(final Format format) { return FormatInfo.of(format, Optional.empty(), Optional.empty()); @@ -37,14 +37,14 @@ public static FormatInfo of(final Format format) { public static FormatInfo of( final Format format, final Optional avroFullSchemaName, - final Optional valueDelimiter) { + final Optional valueDelimiter) { return new FormatInfo(format, avroFullSchemaName, valueDelimiter); } private FormatInfo( final Format format, final Optional avroFullSchemaName, - final Optional delimiter + final Optional delimiter ) { this.format = Objects.requireNonNull(format, "format"); this.avroFullSchemaName = Objects.requireNonNull(avroFullSchemaName, "avroFullSchemaName"); @@ -74,7 +74,7 @@ public Optional getAvroFullSchemaName() { return avroFullSchemaName; } - public Optional getDelimiter() { + public Optional getDelimiter() { return delimiter; } diff --git a/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java b/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java index 6b2bce8cdfb5..9ac5807bf502 100644 --- a/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java +++ b/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java @@ -45,8 +45,8 @@ public void shouldThrowNPEs() { public void shouldImplementEquals() { new EqualsTester() .addEqualityGroup( - FormatInfo.of(Format.DELIMITED, Optional.empty(), Optional.of('x')), - FormatInfo.of(Format.DELIMITED, Optional.empty(), Optional.of('x')) + FormatInfo.of(Format.DELIMITED, Optional.empty(), Optional.of(Delimiter.of('x'))), + FormatInfo.of(Format.DELIMITED, Optional.empty(), Optional.of(Delimiter.of('x'))) ) .addEqualityGroup( FormatInfo.of(Format.AVRO, Optional.of("something"), Optional.empty()), @@ -79,7 +79,7 @@ public void shouldImplementToStringAvro() { @Test public void shouldImplementToStringDelimited() { // Given: - final FormatInfo info = FormatInfo.of(DELIMITED, Optional.empty(), Optional.of('~')); + final FormatInfo info = FormatInfo.of(DELIMITED, Optional.empty(), Optional.of(Delimiter.of("~"))); // When: final String result = info.toString(); @@ -130,7 +130,7 @@ public void shouldThrowWhenAttemptingToUseValueDelimeterWithAvroFormat() { expectedException.expectMessage("Delimeter only supported with DELIMITED format"); // When: - FormatInfo.of(Format.AVRO, Optional.of("something"), Optional.of('x')); + FormatInfo.of(Format.AVRO, Optional.of("something"), Optional.of(Delimiter.of('x'))); } @Test @@ -140,6 +140,6 @@ public void shouldThrowWhenAttemptingToUseValueDelimeterWithJsonFormat() { expectedException.expectMessage("Delimeter only supported with DELIMITED format"); // When: - FormatInfo.of(Format.JSON, Optional.empty(), Optional.of('x')); + FormatInfo.of(Format.JSON, Optional.empty(), Optional.of(Delimiter.of('x'))); } } \ No newline at end of file diff --git a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java index 53a08d869019..8e16852505f6 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java @@ -49,6 +49,7 @@ import io.confluent.ksql.planner.plan.JoinNode; import io.confluent.ksql.schema.ksql.Column; import io.confluent.ksql.schema.ksql.LogicalSchema; +import io.confluent.ksql.serde.Delimiter; import io.confluent.ksql.serde.Format; import io.confluent.ksql.serde.FormatInfo; import io.confluent.ksql.serde.KeyFormat; @@ -279,7 +280,7 @@ private Format getValueFormat(final Sink sink) { .getFormat()); } - private Optional getValueDelimiter(final Sink sink) { + private Optional getValueDelimiter(final Sink sink) { if (sink.getProperties().getValueDelimiter().isPresent()) { return sink.getProperties().getValueDelimiter(); } else { diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json b/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json index ffe00884b908..d782c7c66a05 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json @@ -41,7 +41,7 @@ ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "Error in WITH clause property 'VALUE_DELIMITER': Delimiter must be a single character or TAB, SPACE\nExample valid value: ';'" + "message": "Failed to prepare statement: Invalid value <~> for property VALUE_DELIMITER: Delimiter must be a single character or TAB, SPACE\nExample valid value: ';'" }, "inputs": [], "outputs": [] @@ -59,7 +59,7 @@ ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "Failed to prepare statement: Invalid value for property VALUE_DELIMITER: String must be non-empty\nStatement: CREATE STREAM TEST WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='');" + "message": "Failed to prepare statement: Invalid value for property VALUE_DELIMITER: Delimiter cannot be empty\nExample valid value: ';'" }, "inputs": [], "outputs": [] @@ -77,7 +77,7 @@ ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "Failed to prepare statement: Invalid value for property VALUE_DELIMITER: String must be non-empty\nStatement: CREATE STREAM TEST WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter=' ');" + "message": "Failed to prepare statement: Invalid value for property VALUE_DELIMITER: Delimiter cannot be empty\nExample valid value: ';'" }, "inputs": [], "outputs": [] @@ -95,7 +95,7 @@ ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "Failed to prepare statement: Invalid value for property VALUE_DELIMITER: String must be non-empty\nStatement: CREATE STREAM TEST WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='\t');" + "message": "Failed to prepare statement: Invalid value for property VALUE_DELIMITER: Delimiter cannot be empty\nExample valid value: ';'" }, "inputs": [], "outputs": [] diff --git a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceAsProperties.java b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceAsProperties.java index 389c9b7ca8bb..cb0572ca8edb 100644 --- a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceAsProperties.java +++ b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceAsProperties.java @@ -22,6 +22,7 @@ import io.confluent.ksql.execution.expression.tree.StringLiteral; import io.confluent.ksql.properties.with.CommonCreateConfigs; import io.confluent.ksql.properties.with.CreateAsConfigs; +import io.confluent.ksql.serde.Delimiter; import io.confluent.ksql.serde.Format; import io.confluent.ksql.util.KsqlException; import java.util.Map; @@ -92,6 +93,11 @@ public Optional getWrapSingleValues() { return Optional.ofNullable(getBoolean(CommonCreateConfigs.WRAP_SINGLE_VALUE)); } + public Optional getValueDelimiter() { + final String val = getString(CommonCreateConfigs.VALUE_DELIMITER_PROPERTY); + return val == null ? Optional.empty() : Optional.of(Delimiter.of(val)); + } + public CreateSourceAsProperties withTopic( final String name, final int partitions, diff --git a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceProperties.java b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceProperties.java index 0a8f68bbf283..67fcf9615aa0 100644 --- a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceProperties.java +++ b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceProperties.java @@ -23,6 +23,7 @@ import io.confluent.ksql.parser.DurationParser; import io.confluent.ksql.properties.with.CommonCreateConfigs; import io.confluent.ksql.properties.with.CreateConfigs; +import io.confluent.ksql.serde.Delimiter; import io.confluent.ksql.serde.Format; import io.confluent.ksql.util.KsqlException; import java.time.Duration; @@ -129,6 +130,11 @@ public Optional getWrapSingleValues() { return Optional.ofNullable(getBoolean(CommonCreateConfigs.WRAP_SINGLE_VALUE)); } + public Optional getValueDelimiter() { + final String val = getString(CommonCreateConfigs.VALUE_DELIMITER_PROPERTY); + return val == null ? Optional.empty() : Optional.of(Delimiter.of(val)); + } + public CreateSourceProperties withSchemaId(final int id) { final Map originals = copyOfOriginalLiterals(); originals.put(CreateConfigs.AVRO_SCHEMA_ID, new IntegerLiteral(id)); diff --git a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java index 88979c993956..d9469852931f 100644 --- a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java +++ b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/WithClauseProperties.java @@ -20,15 +20,12 @@ import com.google.common.collect.Sets.SetView; import com.google.errorprone.annotations.Immutable; import io.confluent.ksql.execution.expression.tree.Literal; -import io.confluent.ksql.properties.with.CommonCreateConfigs; import io.confluent.ksql.properties.with.ConfigMetaData; import io.confluent.ksql.util.KsqlException; import java.time.format.DateTimeFormatter; import java.util.HashMap; -import java.util.Iterator; import java.util.Map; import java.util.Objects; -import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; import org.apache.kafka.common.config.AbstractConfig; @@ -121,46 +118,4 @@ private static void throwOnUnknownProperty( } } - public Optional getValueDelimiter() { - final String providedValueDelimiter = getString(CommonCreateConfigs.VALUE_DELIMITER_PROPERTY); - if (providedValueDelimiter == null) { - return Optional.empty(); - } - if (providedValueDelimiter.length() == 1) { - return Optional.of(providedValueDelimiter.charAt(0)); - } else { - final Character delim = NAMED_DELIMITERS.get(providedValueDelimiter); - if (delim != null) { - return Optional.of(delim); - } - throw new KsqlException("Error in WITH clause property '" - + CommonCreateConfigs.VALUE_DELIMITER_PROPERTY - + "': Delimiter must be a single character or " + NAMED_DELIMITERS_STRING - + System.lineSeparator() - + "Example valid value: ';'" - ); - } - } - - private static final Map NAMED_DELIMITERS = ImmutableMap - .builder() - .put("TAB", '\t') - .put("SPACE", ' ') - .build(); - - private static final String NAMED_DELIMITERS_STRING = getNamedDelimitersString(); - - private static String getNamedDelimitersString() { - final StringBuilder sb = new StringBuilder(); - final Iterator iter = NAMED_DELIMITERS.keySet().iterator(); - while (iter.hasNext()) { - sb.append(iter.next()); - if (iter.hasNext()) { - sb.append(", "); - } - } - return sb.toString(); - } - - } diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java index 6b09857edf7f..23d7045c8b54 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java @@ -19,6 +19,7 @@ import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import io.confluent.ksql.schema.connect.SchemaWalker; import io.confluent.ksql.schema.ksql.PersistenceSchema; +import io.confluent.ksql.serde.Delimiter; import io.confluent.ksql.serde.Format; import io.confluent.ksql.serde.KsqlSerdeFactory; import io.confluent.ksql.util.DecimalUtil; @@ -37,12 +38,12 @@ @Immutable public class KsqlDelimitedSerdeFactory implements KsqlSerdeFactory { - private static final char DEFAULT_DELIMITER = ','; + private static final Delimiter DEFAULT_DELIMITER = Delimiter.of(','); private final CSVFormat csvFormat; - public KsqlDelimitedSerdeFactory(final Optional delimiter) { - this.csvFormat = CSVFormat.DEFAULT.withDelimiter(delimiter.orElse(DEFAULT_DELIMITER)); + public KsqlDelimitedSerdeFactory(final Optional delimiter) { + this.csvFormat = CSVFormat.DEFAULT.withDelimiter(delimiter.orElse(DEFAULT_DELIMITER).delimiter); } @Override diff --git a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactoryTest.java b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactoryTest.java index 5563f9c8ead0..fca636f87e69 100644 --- a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactoryTest.java +++ b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactoryTest.java @@ -20,6 +20,7 @@ import io.confluent.ksql.schema.ksql.PhysicalSchema; import io.confluent.ksql.schema.ksql.types.SqlType; import io.confluent.ksql.schema.ksql.types.SqlTypes; +import io.confluent.ksql.serde.Delimiter; import io.confluent.ksql.serde.SerdeOption; import io.confluent.ksql.util.KsqlException; import java.util.Optional; @@ -37,7 +38,7 @@ public class KsqlDelimitedSerdeFactoryTest { @Before public void setUp() { - factory = new KsqlDelimitedSerdeFactory(Optional.of(',')); + factory = new KsqlDelimitedSerdeFactory(Optional.of(Delimiter.of(','))); } @Test From 854b6ec756cf1f190820e516c4b075009adc748b Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Wed, 25 Sep 2019 18:07:57 +0100 Subject: [PATCH 40/46] Review comments --- .../ksql/configdef/ConfigValidators.java | 45 +++++++++++----- .../properties/with/CommonCreateConfigs.java | 2 +- .../io/confluent/ksql/serde/Delimiter.java | 44 +++++++-------- .../confluent/ksql/serde/FormatInfoTest.java | 10 ++-- .../io/confluent/ksql/analyzer/Analyzer.java | 17 +++--- ...mited-value-format.json => delimited.json} | 54 ++----------------- .../with/CreateSourceAsProperties.java | 2 +- .../with/CreateSourceProperties.java | 2 +- .../delimited/KsqlDelimitedSerdeFactory.java | 2 +- .../KsqlDelimitedSerdeFactoryTest.java | 2 +- 10 files changed, 73 insertions(+), 107 deletions(-) rename ksql-functional-tests/src/test/resources/query-validation-tests/{delimited-value-format.json => delimited.json} (87%) diff --git a/ksql-common/src/main/java/io/confluent/ksql/configdef/ConfigValidators.java b/ksql-common/src/main/java/io/confluent/ksql/configdef/ConfigValidators.java index 02132480e372..838ea4c4532b 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/configdef/ConfigValidators.java +++ b/ksql-common/src/main/java/io/confluent/ksql/configdef/ConfigValidators.java @@ -33,21 +33,40 @@ public final class ConfigValidators { private ConfigValidators() { } - public static void parses( - final String name, - final Object val, - final Function parser) { - try { - if (val == null) { - return; + /** + * Validator that tests the STRING property can be parsed by the supplied {@code parser}. + * @param parser the parser. + * @return the validator + */ + public static Validator parses(final Function parser) { + return (name, val) -> { + try { + if (val == null) { + return; + } + if (!(val instanceof String)) { + throw new ConfigException(name, val, "Must be String"); + } + parser.apply((String)val); + } catch (Exception e) { + throw new ConfigException(name, val, e.getMessage()); } - if (!(val instanceof String)) { - throw new ConfigException(name, val, "Must be String"); + }; + } + + /** + * Validator that allows null values and calls the {@code delegate} for any non-null values. + * @param delegate the delegate to call for non-null values. + * @return the validator. + */ + public static Validator nullsAllowed(final Validator delegate) { + return (name, value) -> { + if (value == null) { + return; } - parser.apply((String)val); - } catch (Exception e) { - throw new ConfigException(name, val, e.getMessage()); - } + + delegate.ensureValid(name, value); + }; } public static > Validator enumValues(final Class enumClass) { diff --git a/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java b/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java index a1d506600872..7cf624e42e56 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java +++ b/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java @@ -126,7 +126,7 @@ static void addToConfigDef( VALUE_DELIMITER_PROPERTY, ConfigDef.Type.STRING, null, - (name, val) -> ConfigValidators.parses(name, val, Delimiter::parse), + ConfigValidators.nullsAllowed(ConfigValidators.parses(Delimiter::parse)), Importance.LOW, "The delimiter to use when VALUE_FORMAT='DELIMITED'. Supports single " + "character to be a delimiter, defaults to ','. For space and tab delimited values " diff --git a/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java b/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java index 4cc4f91fe21c..e694e3395ee1 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java +++ b/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java @@ -26,43 +26,39 @@ public final class Delimiter { public final char delimiter; - public static Delimiter of(final char ch) { + public static Delimiter parse(final char ch) { return new Delimiter(ch); } - public static Delimiter of(final String str) { - return new Delimiter(parse(str)); - } - private Delimiter(final char delimiter) { this.delimiter = delimiter; } - public static Character parse(final String providedValueDelimiter) { - if (providedValueDelimiter == null) { - return null; - } - if (providedValueDelimiter.trim().isEmpty()) { - throw new IllegalArgumentException( - "Delimiter cannot be empty" - + System.lineSeparator() - + "Example valid value: ';'" - ); + public static Delimiter parse(final String str) { + if (str == null) { + throw new NullPointerException(); } - if (providedValueDelimiter.length() == 1) { - return providedValueDelimiter.charAt(0); - } else { - final Character delim = NAMED_DELIMITERS.get(providedValueDelimiter); - if (delim != null) { - return delim; - } + if (str.trim().isEmpty()) { throw new IllegalArgumentException( - "Delimiter must be a single character or " - + NAMED_DELIMITERS_STRING + "Delimiter cannot be empty, if you meant to have a tab or space for delimiter, please " + + "use the special values 'TAB' or 'SPACE'" + System.lineSeparator() + "Example valid value: ';'" ); } + if (str.length() == 1) { + return new Delimiter(str.charAt(0)); + } + final Character delim = NAMED_DELIMITERS.get(str); + if (delim != null) { + return new Delimiter(delim); + } + throw new IllegalArgumentException( + "Delimiter must be a single character or " + + NAMED_DELIMITERS_STRING + + System.lineSeparator() + + "Example valid value: ';'" + ); } private static final Map NAMED_DELIMITERS = ImmutableMap diff --git a/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java b/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java index 9ac5807bf502..15a8586e0f53 100644 --- a/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java +++ b/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java @@ -45,8 +45,8 @@ public void shouldThrowNPEs() { public void shouldImplementEquals() { new EqualsTester() .addEqualityGroup( - FormatInfo.of(Format.DELIMITED, Optional.empty(), Optional.of(Delimiter.of('x'))), - FormatInfo.of(Format.DELIMITED, Optional.empty(), Optional.of(Delimiter.of('x'))) + FormatInfo.of(Format.DELIMITED, Optional.empty(), Optional.of(Delimiter.parse('x'))), + FormatInfo.of(Format.DELIMITED, Optional.empty(), Optional.of(Delimiter.parse('x'))) ) .addEqualityGroup( FormatInfo.of(Format.AVRO, Optional.of("something"), Optional.empty()), @@ -79,7 +79,7 @@ public void shouldImplementToStringAvro() { @Test public void shouldImplementToStringDelimited() { // Given: - final FormatInfo info = FormatInfo.of(DELIMITED, Optional.empty(), Optional.of(Delimiter.of("~"))); + final FormatInfo info = FormatInfo.of(DELIMITED, Optional.empty(), Optional.of(Delimiter.parse("~"))); // When: final String result = info.toString(); @@ -130,7 +130,7 @@ public void shouldThrowWhenAttemptingToUseValueDelimeterWithAvroFormat() { expectedException.expectMessage("Delimeter only supported with DELIMITED format"); // When: - FormatInfo.of(Format.AVRO, Optional.of("something"), Optional.of(Delimiter.of('x'))); + FormatInfo.of(Format.AVRO, Optional.of("something"), Optional.of(Delimiter.parse('x'))); } @Test @@ -140,6 +140,6 @@ public void shouldThrowWhenAttemptingToUseValueDelimeterWithJsonFormat() { expectedException.expectMessage("Delimeter only supported with DELIMITED format"); // When: - FormatInfo.of(Format.JSON, Optional.empty(), Optional.of(Delimiter.of('x'))); + FormatInfo.of(Format.JSON, Optional.empty(), Optional.of(Delimiter.parse('x'))); } } \ No newline at end of file diff --git a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java index 8e16852505f6..81e8700db2a1 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java @@ -283,16 +283,15 @@ private Format getValueFormat(final Sink sink) { private Optional getValueDelimiter(final Sink sink) { if (sink.getProperties().getValueDelimiter().isPresent()) { return sink.getProperties().getValueDelimiter(); - } else { - return analysis - .getFromDataSources() - .get(0) - .getDataSource() - .getKsqlTopic() - .getValueFormat() - .getFormatInfo() - .getDelimiter(); } + return analysis + .getFromDataSources() + .get(0) + .getDataSource() + .getKsqlTopic() + .getValueFormat() + .getFormatInfo() + .getDelimiter(); } diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json b/ksql-functional-tests/src/test/resources/query-validation-tests/delimited.json similarity index 87% rename from ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json rename to ksql-functional-tests/src/test/resources/query-validation-tests/delimited.json index d782c7c66a05..ce7fe1298d76 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/delimited-value-format.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/delimited.json @@ -9,12 +9,6 @@ "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED');", "CREATE STREAM S2 as SELECT id, name, value FROM test;" ], - "topics": [ - { - "name": "test_topic", - "format": "DELIMITED" - } - ], "inputs": [ {"topic": "test_topic", "key": 0, "value": "0,zero,0", "timestamp": 0}, {"topic": "test_topic", "key": 0, "value": "0,100,100", "timestamp": 0}, @@ -33,12 +27,6 @@ "statements": [ "CREATE STREAM TEST WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='<~>');" ], - "topics": [ - { - "name": "test_topic", - "format": "DELIMITED" - } - ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", "message": "Failed to prepare statement: Invalid value <~> for property VALUE_DELIMITER: Delimiter must be a single character or TAB, SPACE\nExample valid value: ';'" @@ -59,7 +47,7 @@ ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "Failed to prepare statement: Invalid value for property VALUE_DELIMITER: Delimiter cannot be empty\nExample valid value: ';'" + "message": "Failed to prepare statement: Invalid value for property VALUE_DELIMITER: Delimiter cannot be empty, if you meant to have a tab or space for delimiter, please use the special values 'TAB' or 'SPACE'\nExample valid value: ';'" }, "inputs": [], "outputs": [] @@ -69,15 +57,9 @@ "statements": [ "CREATE STREAM TEST WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter=' ');" ], - "topics": [ - { - "name": "test_topic", - "format": "DELIMITED" - } - ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "Failed to prepare statement: Invalid value for property VALUE_DELIMITER: Delimiter cannot be empty\nExample valid value: ';'" + "message": "Failed to prepare statement: Invalid value for property VALUE_DELIMITER: Delimiter cannot be empty, if you meant to have a tab or space for delimiter, please use the special values 'TAB' or 'SPACE'\nExample valid value: ';'" }, "inputs": [], "outputs": [] @@ -87,15 +69,9 @@ "statements": [ "CREATE STREAM TEST WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='\t');" ], - "topics": [ - { - "name": "test_topic", - "format": "DELIMITED" - } - ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "Failed to prepare statement: Invalid value for property VALUE_DELIMITER: Delimiter cannot be empty\nExample valid value: ';'" + "message": "Failed to prepare statement: Invalid value for property VALUE_DELIMITER: Delimiter cannot be empty, if you meant to have a tab or space for delimiter, please use the special values 'TAB' or 'SPACE'\nExample valid value: ';'" }, "inputs": [], "outputs": [] @@ -106,12 +82,6 @@ "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='|');", "CREATE STREAM S2 as SELECT id, name, value FROM test;" ], - "topics": [ - { - "name": "test_topic", - "format": "DELIMITED" - } - ], "inputs": [ {"topic": "test_topic", "key": 0, "value": "0|zero|0", "timestamp": 0}, {"topic": "test_topic", "key": 0, "value": "0|100|100", "timestamp": 0}, @@ -131,12 +101,6 @@ "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='|');", "CREATE STREAM S2 WITH(value_delimiter='$') AS SELECT * FROM test;" ], - "topics": [ - { - "name": "test_topic", - "format": "DELIMITED" - } - ], "inputs": [ {"topic": "test_topic", "key": 0, "value": "0|zero|0", "timestamp": 0}, {"topic": "test_topic", "key": 0, "value": "0|100|100", "timestamp": 0}, @@ -156,12 +120,6 @@ "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='SPACE');", "CREATE STREAM S2 as SELECT id, name, value FROM test;" ], - "topics": [ - { - "name": "test_topic", - "format": "DELIMITED" - } - ], "inputs": [ {"topic": "test_topic", "key": 0, "value": "0 zero 0", "timestamp": 0}, {"topic": "test_topic", "key": 0, "value": "0 100 100", "timestamp": 0}, @@ -181,12 +139,6 @@ "CREATE STREAM TEST (ID bigint, NAME varchar, VALUE integer) WITH (kafka_topic='test_topic', value_format='DELIMITED', value_delimiter='TAB');", "CREATE STREAM S2 as SELECT id, name, value FROM test;" ], - "topics": [ - { - "name": "test_topic", - "format": "DELIMITED" - } - ], "inputs": [ {"topic": "test_topic", "key": 0, "value": "0\tzero\t0", "timestamp": 0}, {"topic": "test_topic", "key": 0, "value": "0\t100\t100", "timestamp": 0}, diff --git a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceAsProperties.java b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceAsProperties.java index cb0572ca8edb..99659f9ab947 100644 --- a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceAsProperties.java +++ b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceAsProperties.java @@ -95,7 +95,7 @@ public Optional getWrapSingleValues() { public Optional getValueDelimiter() { final String val = getString(CommonCreateConfigs.VALUE_DELIMITER_PROPERTY); - return val == null ? Optional.empty() : Optional.of(Delimiter.of(val)); + return val == null ? Optional.empty() : Optional.of(Delimiter.parse(val)); } public CreateSourceAsProperties withTopic( diff --git a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceProperties.java b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceProperties.java index 67fcf9615aa0..a985755aa54f 100644 --- a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceProperties.java +++ b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceProperties.java @@ -132,7 +132,7 @@ public Optional getWrapSingleValues() { public Optional getValueDelimiter() { final String val = getString(CommonCreateConfigs.VALUE_DELIMITER_PROPERTY); - return val == null ? Optional.empty() : Optional.of(Delimiter.of(val)); + return val == null ? Optional.empty() : Optional.of(Delimiter.parse(val)); } public CreateSourceProperties withSchemaId(final int id) { diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java index 23d7045c8b54..15d87af797a5 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java @@ -38,7 +38,7 @@ @Immutable public class KsqlDelimitedSerdeFactory implements KsqlSerdeFactory { - private static final Delimiter DEFAULT_DELIMITER = Delimiter.of(','); + private static final Delimiter DEFAULT_DELIMITER = Delimiter.parse(','); private final CSVFormat csvFormat; diff --git a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactoryTest.java b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactoryTest.java index fca636f87e69..03f8db7c49e2 100644 --- a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactoryTest.java +++ b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactoryTest.java @@ -38,7 +38,7 @@ public class KsqlDelimitedSerdeFactoryTest { @Before public void setUp() { - factory = new KsqlDelimitedSerdeFactory(Optional.of(Delimiter.of(','))); + factory = new KsqlDelimitedSerdeFactory(Optional.of(Delimiter.parse(','))); } @Test From 4e7b0a7d44b855d84340e6db41892bc259433281 Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Wed, 25 Sep 2019 19:10:23 +0100 Subject: [PATCH 41/46] Added getter for delimiter --- .../src/main/java/io/confluent/ksql/serde/Delimiter.java | 6 +++++- .../ksql/serde/delimited/KsqlDelimitedSerdeFactory.java | 3 ++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java b/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java index e694e3395ee1..5b403b8d65ca 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java +++ b/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java @@ -24,7 +24,7 @@ @Immutable public final class Delimiter { - public final char delimiter; + private final char delimiter; public static Delimiter parse(final char ch) { return new Delimiter(ch); @@ -102,4 +102,8 @@ public int hashCode() { public String toString() { return String.valueOf(delimiter); } + + public char getDelimiter() { + return delimiter; + } } diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java index 15d87af797a5..45250d9ff712 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java @@ -43,7 +43,8 @@ public class KsqlDelimitedSerdeFactory implements KsqlSerdeFactory { private final CSVFormat csvFormat; public KsqlDelimitedSerdeFactory(final Optional delimiter) { - this.csvFormat = CSVFormat.DEFAULT.withDelimiter(delimiter.orElse(DEFAULT_DELIMITER).delimiter); + this.csvFormat = + CSVFormat.DEFAULT.withDelimiter(delimiter.orElse(DEFAULT_DELIMITER).getDelimiter()); } @Override From ea3371117fffb8265d4f8070d5dbc823dd89724b Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Wed, 25 Sep 2019 20:09:07 +0100 Subject: [PATCH 42/46] more nits --- .../io/confluent/ksql/serde/Delimiter.java | 30 +++++++------------ 1 file changed, 10 insertions(+), 20 deletions(-) diff --git a/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java b/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java index 5b403b8d65ca..9ab6e15224e4 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java +++ b/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java @@ -20,10 +20,20 @@ import java.util.Iterator; import java.util.Map; import java.util.Objects; +import org.apache.commons.lang3.StringUtils; @Immutable public final class Delimiter { + private static final Map NAMED_DELIMITERS = ImmutableMap + .builder() + .put("TAB", '\t') + .put("SPACE", ' ') + .build(); + + private static final String NAMED_DELIMITERS_STRING = + StringUtils.join(NAMED_DELIMITERS, ","); + private final char delimiter; public static Delimiter parse(final char ch) { @@ -61,26 +71,6 @@ public static Delimiter parse(final String str) { ); } - private static final Map NAMED_DELIMITERS = ImmutableMap - .builder() - .put("TAB", '\t') - .put("SPACE", ' ') - .build(); - - private static final String NAMED_DELIMITERS_STRING = getNamedDelimitersString(); - - private static String getNamedDelimitersString() { - final StringBuilder sb = new StringBuilder(); - final Iterator iter = NAMED_DELIMITERS.keySet().iterator(); - while (iter.hasNext()) { - sb.append(iter.next()); - if (iter.hasNext()) { - sb.append(", "); - } - } - return sb.toString(); - } - @Override public boolean equals(final Object o) { if (this == o) { From fa12f03dfd7f4b8d8307479a064d4c314fae70fd Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Wed, 25 Sep 2019 20:21:33 +0100 Subject: [PATCH 43/46] Add invalid value to exception message for Delimiter --- .../io/confluent/ksql/configdef/ConfigValidators.java | 2 +- .../src/main/java/io/confluent/ksql/serde/Delimiter.java | 6 +++--- .../test/resources/query-validation-tests/delimited.json | 8 ++++---- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/ksql-common/src/main/java/io/confluent/ksql/configdef/ConfigValidators.java b/ksql-common/src/main/java/io/confluent/ksql/configdef/ConfigValidators.java index 838ea4c4532b..a7c0d3310b58 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/configdef/ConfigValidators.java +++ b/ksql-common/src/main/java/io/confluent/ksql/configdef/ConfigValidators.java @@ -49,7 +49,7 @@ public static Validator parses(final Function parser) { } parser.apply((String)val); } catch (Exception e) { - throw new ConfigException(name, val, e.getMessage()); + throw new ConfigException("Configuration " + name + " is invalid: " + e.getMessage()); } }; } diff --git a/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java b/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java index 9ab6e15224e4..565624ba7b23 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java +++ b/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java @@ -17,7 +17,6 @@ import com.google.common.collect.ImmutableMap; import com.google.errorprone.annotations.Immutable; -import java.util.Iterator; import java.util.Map; import java.util.Objects; import org.apache.commons.lang3.StringUtils; @@ -32,7 +31,7 @@ public final class Delimiter { .build(); private static final String NAMED_DELIMITERS_STRING = - StringUtils.join(NAMED_DELIMITERS, ","); + StringUtils.join(NAMED_DELIMITERS.keySet(), ","); private final char delimiter; @@ -64,7 +63,8 @@ public static Delimiter parse(final String str) { return new Delimiter(delim); } throw new IllegalArgumentException( - "Delimiter must be a single character or " + "Invalid delimiter value: '" + str + + "'. Delimiter must be a single character or " + NAMED_DELIMITERS_STRING + System.lineSeparator() + "Example valid value: ';'" diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/delimited.json b/ksql-functional-tests/src/test/resources/query-validation-tests/delimited.json index ce7fe1298d76..06571d645a4a 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/delimited.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/delimited.json @@ -29,7 +29,7 @@ ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "Failed to prepare statement: Invalid value <~> for property VALUE_DELIMITER: Delimiter must be a single character or TAB, SPACE\nExample valid value: ';'" + "message": "Failed to prepare statement: Configuration VALUE_DELIMITER is invalid: Invalid delimiter value: '<~>'. Delimiter must be a single character or TAB,SPACE\nExample valid value: ';'" }, "inputs": [], "outputs": [] @@ -47,7 +47,7 @@ ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "Failed to prepare statement: Invalid value for property VALUE_DELIMITER: Delimiter cannot be empty, if you meant to have a tab or space for delimiter, please use the special values 'TAB' or 'SPACE'\nExample valid value: ';'" + "message": "Failed to prepare statement: Configuration VALUE_DELIMITER is invalid: Delimiter cannot be empty, if you meant to have a tab or space for delimiter, please use the special values 'TAB' or 'SPACE'\nExample valid value: ';'" }, "inputs": [], "outputs": [] @@ -59,7 +59,7 @@ ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "Failed to prepare statement: Invalid value for property VALUE_DELIMITER: Delimiter cannot be empty, if you meant to have a tab or space for delimiter, please use the special values 'TAB' or 'SPACE'\nExample valid value: ';'" + "message": "Failed to prepare statement: Configuration VALUE_DELIMITER is invalid: Delimiter cannot be empty, if you meant to have a tab or space for delimiter, please use the special values 'TAB' or 'SPACE'\nExample valid value: ';'" }, "inputs": [], "outputs": [] @@ -71,7 +71,7 @@ ], "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", - "message": "Failed to prepare statement: Invalid value for property VALUE_DELIMITER: Delimiter cannot be empty, if you meant to have a tab or space for delimiter, please use the special values 'TAB' or 'SPACE'\nExample valid value: ';'" + "message": "Failed to prepare statement: Configuration VALUE_DELIMITER is invalid: Delimiter cannot be empty, if you meant to have a tab or space for delimiter, please use the special values 'TAB' or 'SPACE'\nExample valid value: ';'" }, "inputs": [], "outputs": [] From a261c35a5bfdb7429036e79444362f2cb1f79351 Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Wed, 25 Sep 2019 22:19:45 +0100 Subject: [PATCH 44/46] review --- .../ksql/configdef/ConfigValidators.java | 9 +-- .../properties/with/CommonCreateConfigs.java | 2 +- .../io/confluent/ksql/serde/Delimiter.java | 4 +- .../ksql/configdef/ConfigValidatorsTest.java | 61 +++++++++++++++++++ .../confluent/ksql/serde/FormatInfoTest.java | 10 +-- .../with/CreateSourceAsProperties.java | 2 +- .../with/CreateSourceProperties.java | 2 +- .../delimited/KsqlDelimitedSerdeFactory.java | 2 +- .../KsqlDelimitedSerdeFactoryTest.java | 2 +- 9 files changed, 76 insertions(+), 18 deletions(-) diff --git a/ksql-common/src/main/java/io/confluent/ksql/configdef/ConfigValidators.java b/ksql-common/src/main/java/io/confluent/ksql/configdef/ConfigValidators.java index a7c0d3310b58..db8075ef6f1a 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/configdef/ConfigValidators.java +++ b/ksql-common/src/main/java/io/confluent/ksql/configdef/ConfigValidators.java @@ -40,13 +40,10 @@ private ConfigValidators() { */ public static Validator parses(final Function parser) { return (name, val) -> { + if (val != null && !(val instanceof String)) { + throw new IllegalArgumentException("validator should only be used with STRING defs"); + } try { - if (val == null) { - return; - } - if (!(val instanceof String)) { - throw new ConfigException(name, val, "Must be String"); - } parser.apply((String)val); } catch (Exception e) { throw new ConfigException("Configuration " + name + " is invalid: " + e.getMessage()); diff --git a/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java b/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java index 7cf624e42e56..da4c161a465a 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java +++ b/ksql-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java @@ -126,7 +126,7 @@ static void addToConfigDef( VALUE_DELIMITER_PROPERTY, ConfigDef.Type.STRING, null, - ConfigValidators.nullsAllowed(ConfigValidators.parses(Delimiter::parse)), + ConfigValidators.nullsAllowed(ConfigValidators.parses(Delimiter::of)), Importance.LOW, "The delimiter to use when VALUE_FORMAT='DELIMITED'. Supports single " + "character to be a delimiter, defaults to ','. For space and tab delimited values " diff --git a/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java b/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java index 565624ba7b23..f5f7190c863d 100644 --- a/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java +++ b/ksql-common/src/main/java/io/confluent/ksql/serde/Delimiter.java @@ -35,7 +35,7 @@ public final class Delimiter { private final char delimiter; - public static Delimiter parse(final char ch) { + public static Delimiter of(final char ch) { return new Delimiter(ch); } @@ -43,7 +43,7 @@ private Delimiter(final char delimiter) { this.delimiter = delimiter; } - public static Delimiter parse(final String str) { + public static Delimiter of(final String str) { if (str == null) { throw new NullPointerException(); } diff --git a/ksql-common/src/test/java/io/confluent/ksql/configdef/ConfigValidatorsTest.java b/ksql-common/src/test/java/io/confluent/ksql/configdef/ConfigValidatorsTest.java index 9d94fa29d9e6..73a12e184fca 100644 --- a/ksql-common/src/test/java/io/confluent/ksql/configdef/ConfigValidatorsTest.java +++ b/ksql-common/src/test/java/io/confluent/ksql/configdef/ConfigValidatorsTest.java @@ -15,14 +15,27 @@ package io.confluent.ksql.configdef; +import static org.mockito.ArgumentMatchers.any; + +import java.util.function.Function; import org.apache.kafka.common.config.ConfigDef.Validator; import org.apache.kafka.common.config.ConfigException; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +@RunWith(MockitoJUnitRunner.class) public class ConfigValidatorsTest { + @Mock + private Function parser; + @Rule public final ExpectedException expectedException = ExpectedException.none(); @@ -122,6 +135,54 @@ public void shouldThrowIfNoMatchForCaseInsensitiveStringNull() { validator.ensureValid("propName", null); } + @Test(expected = IllegalArgumentException.class) + public void shouldThrowOnNonStringFromParse() { + // Given: + final Validator validator = ConfigValidators.parses(parser); + + // When: + validator.ensureValid("propName", 10); + } + + @Test + public void shouldPassNullsToParser() { + // Given: + final Validator validator = ConfigValidators.parses(parser); + + // When: + validator.ensureValid("propName", null); + + // Then: + verify(parser).apply(null); + } + + @Test + public void shouldPassStringsToParser() { + // Given: + final Validator validator = ConfigValidators.parses(parser); + + // When: + validator.ensureValid("propName", "value"); + + // Then: + verify(parser).apply("value"); + } + + @Test + public void shouldThrowIfParserThrows() { + // Given: + final Validator validator = ConfigValidators.parses(parser); + when(parser.apply(any())).thenThrow(new IllegalArgumentException("some error")); + + // Then: + expectedException.expect(ConfigException.class); + expectedException + .expectMessage("Invalid value value for configuration propName: Invalid value: some error"); + + // When: + validator.ensureValid("propName", "value"); + } + private enum TestEnum { FOO, BAR } diff --git a/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java b/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java index 15a8586e0f53..9ac5807bf502 100644 --- a/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java +++ b/ksql-common/src/test/java/io/confluent/ksql/serde/FormatInfoTest.java @@ -45,8 +45,8 @@ public void shouldThrowNPEs() { public void shouldImplementEquals() { new EqualsTester() .addEqualityGroup( - FormatInfo.of(Format.DELIMITED, Optional.empty(), Optional.of(Delimiter.parse('x'))), - FormatInfo.of(Format.DELIMITED, Optional.empty(), Optional.of(Delimiter.parse('x'))) + FormatInfo.of(Format.DELIMITED, Optional.empty(), Optional.of(Delimiter.of('x'))), + FormatInfo.of(Format.DELIMITED, Optional.empty(), Optional.of(Delimiter.of('x'))) ) .addEqualityGroup( FormatInfo.of(Format.AVRO, Optional.of("something"), Optional.empty()), @@ -79,7 +79,7 @@ public void shouldImplementToStringAvro() { @Test public void shouldImplementToStringDelimited() { // Given: - final FormatInfo info = FormatInfo.of(DELIMITED, Optional.empty(), Optional.of(Delimiter.parse("~"))); + final FormatInfo info = FormatInfo.of(DELIMITED, Optional.empty(), Optional.of(Delimiter.of("~"))); // When: final String result = info.toString(); @@ -130,7 +130,7 @@ public void shouldThrowWhenAttemptingToUseValueDelimeterWithAvroFormat() { expectedException.expectMessage("Delimeter only supported with DELIMITED format"); // When: - FormatInfo.of(Format.AVRO, Optional.of("something"), Optional.of(Delimiter.parse('x'))); + FormatInfo.of(Format.AVRO, Optional.of("something"), Optional.of(Delimiter.of('x'))); } @Test @@ -140,6 +140,6 @@ public void shouldThrowWhenAttemptingToUseValueDelimeterWithJsonFormat() { expectedException.expectMessage("Delimeter only supported with DELIMITED format"); // When: - FormatInfo.of(Format.JSON, Optional.empty(), Optional.of(Delimiter.parse('x'))); + FormatInfo.of(Format.JSON, Optional.empty(), Optional.of(Delimiter.of('x'))); } } \ No newline at end of file diff --git a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceAsProperties.java b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceAsProperties.java index 99659f9ab947..cb0572ca8edb 100644 --- a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceAsProperties.java +++ b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceAsProperties.java @@ -95,7 +95,7 @@ public Optional getWrapSingleValues() { public Optional getValueDelimiter() { final String val = getString(CommonCreateConfigs.VALUE_DELIMITER_PROPERTY); - return val == null ? Optional.empty() : Optional.of(Delimiter.parse(val)); + return val == null ? Optional.empty() : Optional.of(Delimiter.of(val)); } public CreateSourceAsProperties withTopic( diff --git a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceProperties.java b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceProperties.java index a985755aa54f..67fcf9615aa0 100644 --- a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceProperties.java +++ b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceProperties.java @@ -132,7 +132,7 @@ public Optional getWrapSingleValues() { public Optional getValueDelimiter() { final String val = getString(CommonCreateConfigs.VALUE_DELIMITER_PROPERTY); - return val == null ? Optional.empty() : Optional.of(Delimiter.parse(val)); + return val == null ? Optional.empty() : Optional.of(Delimiter.of(val)); } public CreateSourceProperties withSchemaId(final int id) { diff --git a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java index 45250d9ff712..5310081e129b 100644 --- a/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java +++ b/ksql-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java @@ -38,7 +38,7 @@ @Immutable public class KsqlDelimitedSerdeFactory implements KsqlSerdeFactory { - private static final Delimiter DEFAULT_DELIMITER = Delimiter.parse(','); + private static final Delimiter DEFAULT_DELIMITER = Delimiter.of(','); private final CSVFormat csvFormat; diff --git a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactoryTest.java b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactoryTest.java index 03f8db7c49e2..fca636f87e69 100644 --- a/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactoryTest.java +++ b/ksql-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactoryTest.java @@ -38,7 +38,7 @@ public class KsqlDelimitedSerdeFactoryTest { @Before public void setUp() { - factory = new KsqlDelimitedSerdeFactory(Optional.of(Delimiter.parse(','))); + factory = new KsqlDelimitedSerdeFactory(Optional.of(Delimiter.of(','))); } @Test From 0552373dc40500bb93d84b55cb76dd6e78f47648 Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Wed, 25 Sep 2019 23:26:23 +0100 Subject: [PATCH 45/46] fixed test --- .../java/io/confluent/ksql/configdef/ConfigValidatorsTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ksql-common/src/test/java/io/confluent/ksql/configdef/ConfigValidatorsTest.java b/ksql-common/src/test/java/io/confluent/ksql/configdef/ConfigValidatorsTest.java index 73a12e184fca..2cf0d7d7cfc0 100644 --- a/ksql-common/src/test/java/io/confluent/ksql/configdef/ConfigValidatorsTest.java +++ b/ksql-common/src/test/java/io/confluent/ksql/configdef/ConfigValidatorsTest.java @@ -177,7 +177,7 @@ public void shouldThrowIfParserThrows() { // Then: expectedException.expect(ConfigException.class); expectedException - .expectMessage("Invalid value value for configuration propName: Invalid value: some error"); + .expectMessage("Configuration propName is invalid: some error"); // When: validator.ensureValid("propName", "value"); From 6c27d26a71d3e54b298884113b6c31b3f2ca53f4 Mon Sep 17 00:00:00 2001 From: Tim Fox Date: Thu, 26 Sep 2019 07:02:14 +0100 Subject: [PATCH 46/46] fixed get of delimiter --- .../ksql/parser/properties/with/CreateSourceAsProperties.java | 2 +- .../ksql/parser/properties/with/CreateSourceProperties.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceAsProperties.java b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceAsProperties.java index 9bfce99efd38..2691d4a79b16 100644 --- a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceAsProperties.java +++ b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceAsProperties.java @@ -97,7 +97,7 @@ public Optional getWrapSingleValues() { } public Optional getValueDelimiter() { - final String val = getString(CommonCreateConfigs.VALUE_DELIMITER_PROPERTY); + final String val = props.getString(CommonCreateConfigs.VALUE_DELIMITER_PROPERTY); return val == null ? Optional.empty() : Optional.of(Delimiter.of(val)); } diff --git a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceProperties.java b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceProperties.java index 1df625f5a18d..d074f62cd0fb 100644 --- a/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceProperties.java +++ b/ksql-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceProperties.java @@ -132,7 +132,7 @@ public Optional getWrapSingleValues() { } public Optional getValueDelimiter() { - final String val = getString(CommonCreateConfigs.VALUE_DELIMITER_PROPERTY); + final String val = props.getString(CommonCreateConfigs.VALUE_DELIMITER_PROPERTY); return val == null ? Optional.empty() : Optional.of(Delimiter.of(val)); }