diff --git a/config/ksql-production-server.properties b/config/ksql-production-server.properties index 1fb062606a43..62b88ff2e405 100644 --- a/config/ksql-production-server.properties +++ b/config/ksql-production-server.properties @@ -61,11 +61,6 @@ bootstrap.servers=localhost:9092 # Enable snappy compression for the Kafka producers compression.type=snappy -# By default, wait up to 500 millis to allow a second poll to Kafka if one side of a join is -# exhausted. This improves join predictability. -# Increasing this setting improves join predictability at the cost of increased end-to-end latency. -# ksql.streams.max.task.idle.ms=? - # uncomment the below to start an embedded Connect worker # ksql.connect.worker.config=config/connect.properties # ksql.connect.configs.topic=ksql-connect-configs diff --git a/config/ksql-server.properties b/config/ksql-server.properties index c238389cd1a7..834904fa2251 100644 --- a/config/ksql-server.properties +++ b/config/ksql-server.properties @@ -68,11 +68,6 @@ bootstrap.servers=localhost:9092 # Enable snappy compression for the Kafka producers compression.type=snappy -# By default, wait up to 500 millis to allow a second poll to Kafka if one side of a join is -# exhausted. This improves join predictability. -# Increasing this setting improves join predictability at the cost of increased end-to-end latency. -# ksql.streams.max.task.idle.ms=? - #------ Connect ------- # uncomment the below to start an embedded Connect worker diff --git a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/integration/ClientIntegrationTest.java b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/integration/ClientIntegrationTest.java index efde79d8855c..ee588642dc47 100644 --- a/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/integration/ClientIntegrationTest.java +++ b/ksqldb-api-client/src/test/java/io/confluent/ksql/api/client/integration/ClientIntegrationTest.java @@ -45,16 +45,16 @@ import io.confluent.ksql.api.client.InsertAck; import io.confluent.ksql.api.client.InsertsPublisher; import io.confluent.ksql.api.client.KsqlArray; -import io.confluent.ksql.api.client.KsqlObject; import io.confluent.ksql.api.client.QueryInfo; import io.confluent.ksql.api.client.QueryInfo.QueryType; +import io.confluent.ksql.api.client.exception.KsqlClientException; +import io.confluent.ksql.api.client.KsqlObject; import io.confluent.ksql.api.client.Row; import io.confluent.ksql.api.client.SourceDescription; import io.confluent.ksql.api.client.StreamInfo; import io.confluent.ksql.api.client.StreamedQueryResult; import io.confluent.ksql.api.client.TableInfo; import io.confluent.ksql.api.client.TopicInfo; -import io.confluent.ksql.api.client.exception.KsqlClientException; import io.confluent.ksql.api.client.util.ClientTestUtil.TestSubscriber; import io.confluent.ksql.api.client.util.RowUtil; import io.confluent.ksql.engine.KsqlEngine; @@ -908,14 +908,13 @@ public void shouldDescribeSource() throws Exception { assertThat(description.timestampColumn(), is(Optional.empty())); assertThat(description.windowType(), is(Optional.empty())); assertThat(description.sqlStatement(), is( - "CREATE STREAM " + TEST_STREAM + " (`STR` STRING KEY, `LONG` BIGINT, " - + "`DEC` DECIMAL(4, 2), `ARRAY` ARRAY, `MAP` MAP, " - + "`STRUCT` STRUCT<`F1` INTEGER>, `COMPLEX` STRUCT<`DECIMAL` DECIMAL(2, 1), " - + "`STRUCT` STRUCT<`F1` STRING, `F2` INTEGER>, `ARRAY_ARRAY` ARRAY>, " - + "`ARRAY_STRUCT` ARRAY>, `ARRAY_MAP` ARRAY>, " - + "`MAP_ARRAY` MAP>, `MAP_MAP` MAP>, " - + "`MAP_STRUCT` MAP>>) WITH " - + "(kafka_topic='" + TEST_TOPIC + "', value_format='json');")); + "CREATE STREAM " + TEST_STREAM + " (STR STRING KEY, LONG BIGINT, DEC DECIMAL(4, 2), " + + "ARRAY ARRAY, MAP MAP, STRUCT STRUCT, " + + "COMPLEX STRUCT<`DECIMAL` DECIMAL(2, 1), STRUCT STRUCT, " + + "ARRAY_ARRAY ARRAY>, ARRAY_STRUCT ARRAY>, " + + "ARRAY_MAP ARRAY>, MAP_ARRAY MAP>, " + + "MAP_MAP MAP>, MAP_STRUCT MAP>>) " + + "WITH (KAFKA_TOPIC='STRUCTURED_TYPES_TOPIC', KEY_FORMAT='KAFKA', VALUE_FORMAT='JSON');")); } @Test diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/config/SessionConfig.java b/ksqldb-common/src/main/java/io/confluent/ksql/config/SessionConfig.java new file mode 100644 index 000000000000..71441a0ce358 --- /dev/null +++ b/ksqldb-common/src/main/java/io/confluent/ksql/config/SessionConfig.java @@ -0,0 +1,103 @@ +/* + * Copyright 2020 Confluent Inc. + * + * Licensed under the Confluent Community 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.config; + +import com.google.common.collect.ImmutableMap; +import com.google.errorprone.annotations.Immutable; +import io.confluent.ksql.testing.EffectivelyImmutable; +import io.confluent.ksql.util.KsqlConfig; +import java.util.Map; +import java.util.Objects; + +/** + * Combination of the system config + session overrides provided by the user. + */ +@Immutable +public final class SessionConfig { + + @EffectivelyImmutable // The map stores primitive types only. + private final ImmutableMap overrides; + private final KsqlConfig systemConfig; + + /** + * Create an instance. + * + * @param systemConfig the system config the server started up with. + * @param overrides the overrides that came as part of the users request. + * @return the session config + */ + public static SessionConfig of(final KsqlConfig systemConfig, final Map overrides) { + return new SessionConfig(systemConfig, overrides); + } + + private SessionConfig(final KsqlConfig systemConfig, final Map overrides) { + this.systemConfig = Objects.requireNonNull(systemConfig, "systemConfig"); + this.overrides = ImmutableMap.copyOf(Objects.requireNonNull(overrides, "overrides")); + } + + /** + * Get the `KsqlConfig` instance, either with or without the overrides applied. + * + *

Calling with {@code withOverridesApplied} of {@code true} will return the system config + * with the user's overrides applied. This is the more common case. This can be useful when the + * user should be able to influence how code executes by using `SET` to provide config overrides. + * + *

Calling with {@code withOverridesApplied} of {@code false} will return the system config + * without any overrides. This can be useful if the users shouldn't be able to supply overrides to + * change the behaviour of the code you're passing the config to. + * + * @param withOverridesApplied flag to indicate if the user supplied overrides should be applied. + * @return the config. + * + */ + public KsqlConfig getConfig(final boolean withOverridesApplied) { + return withOverridesApplied + ? systemConfig.cloneWithPropertyOverwrite(overrides) + : systemConfig; + } + + /** + * @return User supplied overrides, i.e. those provided in the request to the server. + */ + public Map getOverrides() { + return overrides; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final SessionConfig that = (SessionConfig) o; + return Objects.equals(overrides, that.overrides) + && Objects.equals(systemConfig, that.systemConfig); + } + + @Override + public int hashCode() { + return Objects.hash(overrides, systemConfig); + } + + @Override + public String toString() { + return "SessionConfig{" + + "overrides=" + overrides + + ", systemConfig=" + systemConfig + + '}'; + } +} diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java b/ksqldb-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java index d9b2817b3d24..e2d5f5f3e91b 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/properties/with/CommonCreateConfigs.java @@ -17,6 +17,8 @@ import io.confluent.ksql.configdef.ConfigValidators; import io.confluent.ksql.serde.Delimiter; +import io.confluent.ksql.util.KsqlException; +import java.util.Map; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.NonEmptyString; @@ -38,14 +40,15 @@ public final class CommonCreateConfigs { // Persistence Props: public static final String VALUE_AVRO_SCHEMA_FULL_NAME = "VALUE_AVRO_SCHEMA_FULL_NAME"; public static final String VALUE_FORMAT_PROPERTY = "VALUE_FORMAT"; + public static final String KEY_FORMAT_PROPERTY = "KEY_FORMAT"; + public static final String FORMAT_PROPERTY = "FORMAT"; public static final String WRAP_SINGLE_VALUE = "WRAP_SINGLE_VALUE"; public static final String VALUE_DELIMITER_PROPERTY = "VALUE_DELIMITER"; public static void addToConfigDef( final ConfigDef configDef, - final boolean topicNameRequired, - final boolean valueFormatRequired + final boolean topicNameRequired ) { configDef .define( @@ -79,7 +82,7 @@ public static void addToConfigDef( .define( VALUE_FORMAT_PROPERTY, ConfigDef.Type.STRING, - valueFormatRequired ? ConfigDef.NO_DEFAULT_VALUE : null, + null, Importance.HIGH, "The format of the serialized value" ) @@ -129,7 +132,40 @@ public static void addToConfigDef( "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."); + + "character.") + .define( + KEY_FORMAT_PROPERTY, + ConfigDef.Type.STRING, + null, + Importance.HIGH, + "The format of the serialized key" + ) + .define( + FORMAT_PROPERTY, + ConfigDef.Type.STRING, + null, + Importance.HIGH, + "The format of the serialized key and value"); + } + + public static void validateKeyValueFormats(final Map configs) { + final Object value = configs.get(FORMAT_PROPERTY); + if (value == null) { + return; + } + + if (configs.get(KEY_FORMAT_PROPERTY) != null) { + throw new KsqlException("Cannot supply both '" + KEY_FORMAT_PROPERTY + "' and '" + + FORMAT_PROPERTY + "' properties, as '" + FORMAT_PROPERTY + "' sets both key and value " + + "formats. Either use just '" + FORMAT_PROPERTY + "', or use '" + KEY_FORMAT_PROPERTY + + "' and '" + VALUE_FORMAT_PROPERTY + "'."); + } + if (configs.get(VALUE_FORMAT_PROPERTY) != null) { + throw new KsqlException("Cannot supply both '" + VALUE_FORMAT_PROPERTY + "' and '" + + FORMAT_PROPERTY + "' properties, as '" + FORMAT_PROPERTY + "' sets both key and value " + + "formats. Either use just '" + FORMAT_PROPERTY + "', or use '" + KEY_FORMAT_PROPERTY + + "' and '" + VALUE_FORMAT_PROPERTY + "'."); + } } private CommonCreateConfigs() { diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/properties/with/CreateAsConfigs.java b/ksqldb-common/src/main/java/io/confluent/ksql/properties/with/CreateAsConfigs.java index 253a41765c4a..f9ca4f1e1e69 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/properties/with/CreateAsConfigs.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/properties/with/CreateAsConfigs.java @@ -25,7 +25,7 @@ public final class CreateAsConfigs { private static final ConfigDef CONFIG_DEF = new ConfigDef(); static { - CommonCreateConfigs.addToConfigDef(CONFIG_DEF, false, false); + CommonCreateConfigs.addToConfigDef(CONFIG_DEF, false); } public static final ConfigMetaData CONFIG_METADATA = ConfigMetaData.of(CONFIG_DEF); diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/properties/with/CreateConfigs.java b/ksqldb-common/src/main/java/io/confluent/ksql/properties/with/CreateConfigs.java index 3f057c1713f7..f04e9ca81d30 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/properties/with/CreateConfigs.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/properties/with/CreateConfigs.java @@ -71,7 +71,7 @@ public final class CreateConfigs { ); static { - CommonCreateConfigs.addToConfigDef(CONFIG_DEF, true, true); + CommonCreateConfigs.addToConfigDef(CONFIG_DEF, true); } public static final ConfigMetaData CONFIG_METADATA = ConfigMetaData.of(CONFIG_DEF); diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/serde/RefinementInfo.java b/ksqldb-common/src/main/java/io/confluent/ksql/serde/RefinementInfo.java index 02a01e8c0b5b..e65fa12e0ed9 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/serde/RefinementInfo.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/serde/RefinementInfo.java @@ -19,6 +19,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.errorprone.annotations.Immutable; import io.confluent.ksql.parser.OutputRefinement; + import java.util.Objects; /** diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/serde/SerdeOption.java b/ksqldb-common/src/main/java/io/confluent/ksql/serde/SerdeOption.java index c9c8acbcaf4d..037fbd422a47 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/serde/SerdeOption.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/serde/SerdeOption.java @@ -28,7 +28,7 @@ public enum SerdeOption { * * @see SerdeOption#UNWRAP_SINGLE_VALUES */ - WRAP_SINGLE_VALUES(SerdeFeature.WRAP_SINGLES), + WRAP_SINGLE_VALUES(SerdeFeature.WRAP_SINGLES, false), /** * If the value schema contains only a single field, persist it as an anonymous value. @@ -38,19 +38,25 @@ public enum SerdeOption { * * @see SerdeOption#WRAP_SINGLE_VALUES */ - UNWRAP_SINGLE_VALUES(SerdeFeature.UNWRAP_SINGLES), + UNWRAP_SINGLE_VALUES(SerdeFeature.UNWRAP_SINGLES, false), /** * Key version of {@link #UNWRAP_SINGLE_VALUES}. */ - UNWRAP_SINGLE_KEYS(SerdeFeature.UNWRAP_SINGLES); + UNWRAP_SINGLE_KEYS(SerdeFeature.UNWRAP_SINGLES, true); + private final boolean isKey; private final SerdeFeature requiredFeature; - SerdeOption(final SerdeFeature requiredFeature) { + SerdeOption(final SerdeFeature requiredFeature, final boolean isKey) { + this.isKey = isKey; this.requiredFeature = Objects.requireNonNull(requiredFeature, "requiredFeature"); } + public boolean isKeyOption() { + return isKey; + } + public SerdeFeature requiredFeature() { return requiredFeature; } diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/serde/SerdeOptions.java b/ksqldb-common/src/main/java/io/confluent/ksql/serde/SerdeOptions.java index 899365b4aa3e..c7e67a605554 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/serde/SerdeOptions.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/serde/SerdeOptions.java @@ -16,7 +16,6 @@ package io.confluent.ksql.serde; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; import com.google.common.collect.Sets; import com.google.errorprone.annotations.Immutable; import java.util.EnumSet; @@ -33,11 +32,11 @@ @Immutable public final class SerdeOptions { - private static final ImmutableSet KEY_WRAPPING_OPTIONS = ImmutableSet.of( + public static final ImmutableSet KEY_WRAPPING_OPTIONS = ImmutableSet.of( SerdeOption.UNWRAP_SINGLE_KEYS ); - private static final ImmutableSet VALUE_WRAPPING_OPTIONS = ImmutableSet.of( + public static final ImmutableSet VALUE_WRAPPING_OPTIONS = ImmutableSet.of( SerdeOption.WRAP_SINGLE_VALUES, SerdeOption.UNWRAP_SINGLE_VALUES ); @@ -60,33 +59,22 @@ private SerdeOptions(final ImmutableSet options) { validate(this.options); } - @SuppressWarnings("MethodMayBeStatic") public EnabledSerdeFeatures keyFeatures() { - // Currently there are no key features: - return EnabledSerdeFeatures.of(); + return features(true); } public EnabledSerdeFeatures valueFeatures() { - // Currently there are no key features, so all are value features: - return EnabledSerdeFeatures.from(options.stream() - .map(SerdeOption::requiredFeature) - .collect(Collectors.toSet())); + return features(false); } public Set all() { return options; } - public Optional keyWrapping() { - return Optional.ofNullable( - Iterables.getFirst(Sets.intersection(options, KEY_WRAPPING_OPTIONS), null) - ); - } - - public Optional valueWrapping() { - return Optional.ofNullable( - Iterables.getFirst(Sets.intersection(options, VALUE_WRAPPING_OPTIONS), null) - ); + public Optional findAny(final Set anyOf) { + return anyOf.stream() + .filter(options::contains) + .findAny(); } @Override @@ -111,6 +99,13 @@ public String toString() { return options.toString(); } + private EnabledSerdeFeatures features(final boolean key) { + return EnabledSerdeFeatures.from(options.stream() + .filter(option -> option.isKeyOption() == key) + .map(SerdeOption::requiredFeature) + .collect(Collectors.toSet())); + } + private static void validate(final Set options) { final Set wrappingOptions = Sets.intersection(options, VALUE_WRAPPING_OPTIONS); if (wrappingOptions.size() > 1) { diff --git a/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java b/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java index 6787f45bd3f6..12d2114a708d 100644 --- a/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java +++ b/ksqldb-common/src/main/java/io/confluent/ksql/util/KsqlConfig.java @@ -135,6 +135,23 @@ public class KsqlConfig extends AbstractConfig { + "in interactive mode. Once this limit is reached, any further persistent queries will not " + "be accepted."; + public static final String KSQL_KEY_FORMAT_ENABLED = "ksql.key.format.enabled"; + public static final Boolean KSQL_KEY_FORMAT_ENABLED_DEFAULT = false; + public static final String KSQL_KEY_FORMAT_ENABLED_DOC = + "Feature flag for non-Kafka key formats"; + + public static final String KSQL_DEFAULT_KEY_FORMAT_CONFIG = "ksql.persistence.default.format.key"; + private static final String KSQL_DEFAULT_KEY_FORMAT_DEFAULT = "KAFKA"; + private static final String KSQL_DEFAULT_KEY_FORMAT_DOC = + "Key format that will be used by default if none is specified in the WITH properties of " + + "CREATE STREAM/TABLE statements."; + + public static final String KSQL_DEFAULT_VALUE_FORMAT_CONFIG = + "ksql.persistence.default.format.value"; + private static final String KSQL_DEFAULT_VALUE_FORMAT_DOC = + "Value format that will be used by default if none is specified in the WITH properties of " + + "CREATE STREAM/TABLE statements."; + public static final String KSQL_WRAP_SINGLE_VALUES = "ksql.persistence.wrap.single.values"; @@ -309,14 +326,6 @@ public class KsqlConfig extends AbstractConfig { + " buffer will be unbounded. If the maximum capacity is exceeded, the query will be" + " terminated"; - // Defaults for config NOT defined by this class's ConfigDef: - static final ImmutableMap NON_KSQL_DEFAULTS = ImmutableMap - .builder() - // Improve join predictability by generally allowing a second poll to ensure both sizes - // of a join have data. See https://github.com/confluentinc/ksql/issues/5537. - .put(KSQL_STREAMS_PREFIX + StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 500L) - .build(); - public static final String KSQL_QUERY_RETRY_BACKOFF_INITIAL_MS = "ksql.query.retry.backoff.initial.ms"; public static final Long KSQL_QUERY_RETRY_BACKOFF_INITIAL_MS_DEFAULT = 15000L; @@ -586,6 +595,24 @@ private static ConfigDef buildConfigDef(final ConfigGeneration generation) { KSQL_SECURITY_EXTENSION_DEFAULT, ConfigDef.Importance.LOW, KSQL_SECURITY_EXTENSION_DOC + ).define( + KSQL_KEY_FORMAT_ENABLED, + Type.BOOLEAN, + KSQL_KEY_FORMAT_ENABLED_DEFAULT, + ConfigDef.Importance.LOW, + KSQL_KEY_FORMAT_ENABLED_DOC + ).define( + KSQL_DEFAULT_KEY_FORMAT_CONFIG, + Type.STRING, + KSQL_DEFAULT_KEY_FORMAT_DEFAULT, + ConfigDef.Importance.LOW, + KSQL_DEFAULT_KEY_FORMAT_DOC + ).define( + KSQL_DEFAULT_VALUE_FORMAT_CONFIG, + Type.STRING, + null, + ConfigDef.Importance.LOW, + KSQL_DEFAULT_VALUE_FORMAT_DOC ).define( KSQL_WRAP_SINGLE_VALUES, ConfigDef.Type.BOOLEAN, @@ -865,7 +892,7 @@ public KsqlConfig(final Map props) { } private KsqlConfig(final ConfigGeneration generation, final Map props) { - super(configDef(generation), addNonKsqlDefaults(props)); + super(configDef(generation), props); final Map streamsConfigDefaults = new HashMap<>(); streamsConfigDefaults.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, KsqlConstants @@ -894,12 +921,6 @@ private KsqlConfig(final ConfigGeneration generation, final Map props) { this.ksqlStreamConfigProps = buildStreamingConfig(streamsConfigDefaults, originals()); } - private static Map addNonKsqlDefaults(final Map props) { - final Map withDefaults = new HashMap<>(props); - NON_KSQL_DEFAULTS.forEach(withDefaults::putIfAbsent); - return withDefaults; - } - private boolean getBooleanConfig(final String config, final boolean defaultValue) { final Object value = originals().get(config); if (value == null) { diff --git a/ksqldb-common/src/test/java/io/confluent/ksql/config/SessionConfigTest.java b/ksqldb-common/src/test/java/io/confluent/ksql/config/SessionConfigTest.java new file mode 100644 index 000000000000..efc7426b10e9 --- /dev/null +++ b/ksqldb-common/src/test/java/io/confluent/ksql/config/SessionConfigTest.java @@ -0,0 +1,113 @@ +package io.confluent.ksql.config; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.sameInstance; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.common.collect.ImmutableMap; +import com.google.common.testing.EqualsTester; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import io.confluent.ksql.util.KsqlConfig; +import java.util.HashMap; +import java.util.Map; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + + +@SuppressFBWarnings("RV_RETURN_VALUE_IGNORED_INFERRED") +@RunWith(MockitoJUnitRunner.class) +public class SessionConfigTest { + + private static final Map OVERRIDES = ImmutableMap.of("Something", 1); + + @Mock + private KsqlConfig systemConfig; + @Mock + private KsqlConfig configWithOverrides; + private SessionConfig config; + + @Before + public void setUp() { + when(systemConfig.cloneWithPropertyOverwrite(any())).thenReturn(configWithOverrides); + + config = SessionConfig.of(systemConfig, OVERRIDES); + } + + @SuppressWarnings("UnstableApiUsage") + @Test + public void shouldImplementHashCodeAndEquals() { + new EqualsTester() + .addEqualityGroup( + SessionConfig.of(systemConfig, OVERRIDES), + SessionConfig.of(systemConfig, OVERRIDES) + ) + .addEqualityGroup( + SessionConfig.of(new KsqlConfig(ImmutableMap.of()), OVERRIDES) + ) + .addEqualityGroup( + SessionConfig.of(systemConfig, ImmutableMap.of("different", "overrides")) + ) + .testEquals(); + } + + @Test + public void shouldHaveSensibleToString() { + // When: + final String result = SessionConfig.of(systemConfig, OVERRIDES).toString(); + + // Then: + assertThat(result, containsString(systemConfig.toString())); + assertThat(result, containsString(OVERRIDES.toString())); + } + + @Test + public void shouldGetConfigWithoutOverrides() { + // When: + final KsqlConfig result = config.getConfig(false); + + // Then: + assertThat(result, is(sameInstance(systemConfig))); + } + + @Test + public void shouldGetConfigWithOverrides() { + // When: + final KsqlConfig result = config.getConfig(true); + + // Then: + assertThat(result, is(sameInstance(configWithOverrides))); + verify(systemConfig).cloneWithPropertyOverwrite(OVERRIDES); + } + + @Test + public void shouldTakeDefensiveCopyOfProperties() { + // Given: + final Map props = new HashMap<>(); + props.put("this", "that"); + + final SessionConfig config = SessionConfig.of(systemConfig, props); + + // When: + props.put("other", "thing"); + + // Then: + assertThat(config.getOverrides(), is(ImmutableMap.of("this", "that"))); + } + + @Test + public void shouldReturnImmutableProperties() { + // Given: + final SessionConfig config = SessionConfig.of(systemConfig, new HashMap<>()); + + // Then: + assertThat(config.getOverrides(), is(instanceOf(ImmutableMap.class))); + } +} \ No newline at end of file diff --git a/ksqldb-common/src/test/java/io/confluent/ksql/serde/EnabledSerdeFeaturesTest.java b/ksqldb-common/src/test/java/io/confluent/ksql/serde/EnabledSerdeFeaturesTest.java index 62dff6a8e8bb..6d180a118da6 100644 --- a/ksqldb-common/src/test/java/io/confluent/ksql/serde/EnabledSerdeFeaturesTest.java +++ b/ksqldb-common/src/test/java/io/confluent/ksql/serde/EnabledSerdeFeaturesTest.java @@ -33,7 +33,7 @@ public void shouldThrowOnIncompatibleFeatures() { @Test public void shouldHaveSensibleToString() { // Given: - final EnabledSerdeFeatures features = EnabledSerdeFeatures.of(SerdeFeature.WRAP_SINGLES); + final EnabledSerdeFeatures features = EnabledSerdeFeatures.of(WRAP_SINGLES); // Then: assertThat(features.toString(), is("[WRAP_SINGLES]")); diff --git a/ksqldb-common/src/test/java/io/confluent/ksql/serde/RefinementInfoTest.java b/ksqldb-common/src/test/java/io/confluent/ksql/serde/RefinementInfoTest.java index 07799a0bd7ba..0d4c81533300 100644 --- a/ksqldb-common/src/test/java/io/confluent/ksql/serde/RefinementInfoTest.java +++ b/ksqldb-common/src/test/java/io/confluent/ksql/serde/RefinementInfoTest.java @@ -18,9 +18,12 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.is; import com.google.common.testing.EqualsTester; import com.google.common.testing.NullPointerTester; +import java.util.Optional; + import io.confluent.ksql.parser.OutputRefinement; import org.junit.Test; diff --git a/ksqldb-common/src/test/java/io/confluent/ksql/serde/SerdeOptionsTest.java b/ksqldb-common/src/test/java/io/confluent/ksql/serde/SerdeOptionsTest.java index 649aca69fa8e..e491276b0296 100644 --- a/ksqldb-common/src/test/java/io/confluent/ksql/serde/SerdeOptionsTest.java +++ b/ksqldb-common/src/test/java/io/confluent/ksql/serde/SerdeOptionsTest.java @@ -3,6 +3,8 @@ import static io.confluent.ksql.serde.SerdeOption.UNWRAP_SINGLE_KEYS; import static io.confluent.ksql.serde.SerdeOption.UNWRAP_SINGLE_VALUES; import static io.confluent.ksql.serde.SerdeOption.WRAP_SINGLE_VALUES; +import static io.confluent.ksql.serde.SerdeOptions.KEY_WRAPPING_OPTIONS; +import static io.confluent.ksql.serde.SerdeOptions.VALUE_WRAPPING_OPTIONS; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.is; @@ -12,6 +14,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.testing.EqualsTester; import io.confluent.ksql.serde.SerdeOptions.Builder; +import java.util.EnumSet; import java.util.Optional; import org.hamcrest.MatcherAssert; import org.junit.Test; @@ -66,27 +69,49 @@ public void shouldThrowOnBuildOnClashingOptions() { // Then: final SerdeOptions result = builder.build(); - assertThat("should be in valid state", result.all(), not(contains(WRAP_SINGLE_VALUES))); + assertThat("should be in invalid state", result.all(), not(contains(WRAP_SINGLE_VALUES))); } @Test - public void shouldReturnKeyWrapping() { - assertThat(SerdeOptions.of().keyWrapping(), is(Optional.empty())); - assertThat(SerdeOptions.of(UNWRAP_SINGLE_KEYS).keyWrapping(), - is(Optional.of(UNWRAP_SINGLE_KEYS))); + public void shouldReturnKeyFeatures() { + assertThat(SerdeOptions.of().keyFeatures(), is(EnabledSerdeFeatures.of())); + assertThat(SerdeOptions.of(WRAP_SINGLE_VALUES, UNWRAP_SINGLE_KEYS).keyFeatures(), + is(EnabledSerdeFeatures.of(SerdeFeature.UNWRAP_SINGLES))); } @Test - public void shouldReturnValueWrapping() { - assertThat(SerdeOptions.of().valueWrapping(), is(Optional.empty())); - assertThat(SerdeOptions.of(WRAP_SINGLE_VALUES).valueWrapping(), - is(Optional.of(WRAP_SINGLE_VALUES))); - assertThat(SerdeOptions.of(UNWRAP_SINGLE_VALUES).valueWrapping(), - is(Optional.of(UNWRAP_SINGLE_VALUES))); + public void shouldReturnValueFeatures() { + assertThat(SerdeOptions.of().valueFeatures(), is(EnabledSerdeFeatures.of())); + assertThat(SerdeOptions.of(WRAP_SINGLE_VALUES, UNWRAP_SINGLE_KEYS).valueFeatures(), + is(EnabledSerdeFeatures.of(SerdeFeature.WRAP_SINGLES))); } @Test - public void shouldImplmentToString() { + public void shouldReturnEmptyFromFindAnyOnNoMatch() { + assertThat(SerdeOptions.of() + .findAny(VALUE_WRAPPING_OPTIONS), is(Optional.empty())); + + assertThat(SerdeOptions.of(UNWRAP_SINGLE_KEYS) + .findAny(VALUE_WRAPPING_OPTIONS), is(Optional.empty())); + } + + @Test + public void shouldReturnAnOptionFromFindAnyOnAMatch() { + assertThat(SerdeOptions.of(WRAP_SINGLE_VALUES, UNWRAP_SINGLE_KEYS) + .findAny(VALUE_WRAPPING_OPTIONS), is(Optional.of(WRAP_SINGLE_VALUES))); + + assertThat(SerdeOptions.of(UNWRAP_SINGLE_VALUES, UNWRAP_SINGLE_KEYS) + .findAny(KEY_WRAPPING_OPTIONS), is(Optional.of(UNWRAP_SINGLE_KEYS))); + } + + @Test + public void shouldHandleIncompatibleOptionsInFindAnyParam() { + assertThat(SerdeOptions.of(WRAP_SINGLE_VALUES, UNWRAP_SINGLE_KEYS) + .findAny(EnumSet.allOf(SerdeOption.class)), is(not(Optional.empty()))); + } + + @Test + public void shouldImplementToString() { MatcherAssert.assertThat( SerdeOptions.of(SerdeOption.UNWRAP_SINGLE_VALUES).toString(), is("[UNWRAP_SINGLE_VALUES]")); diff --git a/ksqldb-common/src/test/java/io/confluent/ksql/util/KsqlConfigTest.java b/ksqldb-common/src/test/java/io/confluent/ksql/util/KsqlConfigTest.java index dc6b7dc09b2a..15001d0689e4 100644 --- a/ksqldb-common/src/test/java/io/confluent/ksql/util/KsqlConfigTest.java +++ b/ksqldb-common/src/test/java/io/confluent/ksql/util/KsqlConfigTest.java @@ -316,13 +316,11 @@ public void shouldHaveCorrectOriginalsAfterCloneWithOverwrite() { )); // Then: - assertThat(cloned.originals(), is(ImmutableMap.builder() - .put(KsqlConfig.KSQL_SERVICE_ID_CONFIG, "overridden-id") - .put(KsqlConfig.KSQL_WRAP_SINGLE_VALUES, "true") - .put(KsqlConfig.KSQL_PERSISTENT_QUERY_NAME_PREFIX_CONFIG, "bob") - .putAll(KsqlConfig.NON_KSQL_DEFAULTS) - .build() - )); + assertThat(cloned.originals(), is(ImmutableMap.of( + KsqlConfig.KSQL_SERVICE_ID_CONFIG, "overridden-id", + KsqlConfig.KSQL_WRAP_SINGLE_VALUES, "true", + KsqlConfig.KSQL_PERSISTENT_QUERY_NAME_PREFIX_CONFIG, "bob" + ))); } @Test @@ -577,50 +575,4 @@ public void shouldFilterProducerConfigs() { assertThat(ksqlConfig.getProducerClientConfigProps(), hasEntry(ProducerConfig.CLIENT_ID_CONFIG, null)); assertThat(ksqlConfig.getProducerClientConfigProps(), not(hasKey("not.a.config"))); } - - @Test - public void shouldDefaultStreamsMinTaskIdleConfig() { - // When: - final KsqlConfig ksqlConfig = new KsqlConfig(ImmutableMap.of()); - - // Then: - assertThat( - ksqlConfig.getKsqlStreamConfigProps(), - hasEntry(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 500L) - ); - } - - @Test - public void shouldUsePrefixedStreamsMinTaskIdleConfig() { - // Given: - final ImmutableMap props = ImmutableMap.of( - KsqlConfig.KSQL_STREAMS_PREFIX + StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 100L - ); - - // When: - final KsqlConfig ksqlConfig = new KsqlConfig(props); - - // Then: - assertThat( - ksqlConfig.getKsqlStreamConfigProps(), - hasEntry(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 100L) - ); - } - - @Test - public void shouldUseNonPrefixedStreamsMinTaskIdleConfig() { - // Given: - final ImmutableMap props = ImmutableMap.of( - StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 1000L - ); - - // When: - final KsqlConfig ksqlConfig = new KsqlConfig(props); - - // Then: - assertThat( - ksqlConfig.getKsqlStreamConfigProps(), - hasEntry(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 1000L) - ); - } } diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java b/ksqldb-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java index 24e7d0bd7e88..fb4f114eafe2 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/analyzer/Analyzer.java @@ -176,7 +176,7 @@ private void analyzeNonStdOutSink(final Sink sink) { // overwrite any inheritable properties if they were explicitly // specified in the statement - sourceProperties.putAll(sink.getProperties().getFormatProperties()); + sourceProperties.putAll(sink.getProperties().getValueFormatProperties()); final ValueFormat valueFormat = ValueFormat.of(FormatInfo.of( format.name(), @@ -211,8 +211,8 @@ private KeyFormat buildKeyFormat() { } private Format getValueFormat(final Sink sink) { - return sink.getProperties().getValueFormat() - .orElseGet(() -> FormatFactory.of(getSourceInfo())); + return FormatFactory.of( + sink.getProperties().getValueFormat().orElseGet(this::getSourceInfo)); } private FormatInfo getSourceInfo() { diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/analyzer/RewrittenAnalysis.java b/ksqldb-engine/src/main/java/io/confluent/ksql/analyzer/RewrittenAnalysis.java index 47dd4d7feedb..c68269478445 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/analyzer/RewrittenAnalysis.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/analyzer/RewrittenAnalysis.java @@ -40,6 +40,7 @@ import io.confluent.ksql.parser.tree.WindowExpression; import io.confluent.ksql.serde.RefinementInfo; import io.confluent.ksql.util.KsqlException; + import java.util.List; import java.util.Objects; import java.util.Optional; diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/ddl/commands/CommandFactories.java b/ksqldb-engine/src/main/java/io/confluent/ksql/ddl/commands/CommandFactories.java index aee919d6bcb2..910a8ec12a52 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/ddl/commands/CommandFactories.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/ddl/commands/CommandFactories.java @@ -16,6 +16,7 @@ package io.confluent.ksql.ddl.commands; import com.google.common.annotations.VisibleForTesting; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.execution.ddl.commands.CreateStreamCommand; import io.confluent.ksql.execution.ddl.commands.CreateTableCommand; import io.confluent.ksql.execution.ddl.commands.DdlCommand; @@ -33,9 +34,7 @@ import io.confluent.ksql.services.ServiceContext; import io.confluent.ksql.util.HandlerMaps; import io.confluent.ksql.util.HandlerMaps.ClassHandlerMapR2; -import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.KsqlException; -import java.util.Map; import java.util.Objects; // CHECKSTYLE_RULES.OFF: ClassDataAbstractionCoupling @@ -88,8 +87,7 @@ public CommandFactories(final ServiceContext serviceContext, final MetaStore met public DdlCommand create( final String sqlExpression, final DdlStatement ddlStatement, - final KsqlConfig ksqlConfig, - final Map properties + final SessionConfig config ) { return FACTORIES .getOrDefault(ddlStatement.getClass(), (statement, cf, ci) -> { @@ -102,7 +100,7 @@ public DdlCommand create( }) .handle( this, - new CallInfo(sqlExpression, ksqlConfig, properties), + new CallInfo(sqlExpression, config), ddlStatement); } @@ -112,7 +110,7 @@ private CreateStreamCommand handleCreateStream( ) { return createSourceFactory.createStreamCommand( statement, - callInfo.ksqlConfig + callInfo.config.getConfig(true) ); } @@ -122,7 +120,7 @@ private CreateTableCommand handleCreateTable( ) { return createSourceFactory.createTableCommand( statement, - callInfo.ksqlConfig + callInfo.config.getConfig(true) ); } @@ -134,12 +132,10 @@ private DropSourceCommand handleDropTable(final DropTable statement) { return dropSourceFactory.create(statement); } - @SuppressWarnings("MethodMayBeStatic") private RegisterTypeCommand handleRegisterType(final RegisterType statement) { return registerTypeFactory.create(statement); } - @SuppressWarnings("MethodMayBeStatic") private DropTypeCommand handleDropType(final DropType statement) { return dropTypeFactory.create(statement); } @@ -147,18 +143,14 @@ private DropTypeCommand handleDropType(final DropType statement) { private static final class CallInfo { final String sqlExpression; - final KsqlConfig ksqlConfig; - final Map properties; + final SessionConfig config; private CallInfo( final String sqlExpression, - final KsqlConfig ksqlConfig, - final Map properties + final SessionConfig config ) { this.sqlExpression = Objects.requireNonNull(sqlExpression, "sqlExpression"); - this.properties = Objects.requireNonNull(properties, "properties"); - this.ksqlConfig = Objects.requireNonNull(ksqlConfig, "ksqlConfig") - .cloneWithPropertyOverwrite(properties); + this.config = Objects.requireNonNull(config, "config"); } } } diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/ddl/commands/DdlCommandFactory.java b/ksqldb-engine/src/main/java/io/confluent/ksql/ddl/commands/DdlCommandFactory.java index 18d8402616c6..60df72bbe8b0 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/ddl/commands/DdlCommandFactory.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/ddl/commands/DdlCommandFactory.java @@ -15,16 +15,14 @@ package io.confluent.ksql.ddl.commands; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.execution.ddl.commands.DdlCommand; import io.confluent.ksql.parser.tree.DdlStatement; -import io.confluent.ksql.util.KsqlConfig; -import java.util.Map; public interface DdlCommandFactory { DdlCommand create( String sqlExpression, DdlStatement ddlStatement, - KsqlConfig ksqlConfig, - Map properties + SessionConfig config ); } diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/embedded/KsqlContext.java b/ksqldb-engine/src/main/java/io/confluent/ksql/embedded/KsqlContext.java index 6626fa411a1c..cb00958543dd 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/embedded/KsqlContext.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/embedded/KsqlContext.java @@ -20,6 +20,7 @@ import io.confluent.ksql.KsqlExecutionContext; import io.confluent.ksql.KsqlExecutionContext.ExecuteResult; import io.confluent.ksql.ServiceInfo; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.function.InternalFunctionRegistry; import io.confluent.ksql.function.MutableFunctionRegistry; @@ -191,8 +192,7 @@ private static ExecuteResult execute( final ConfiguredStatement configured = injector.inject(ConfiguredStatement.of( prepared, - mutableSessionPropertyOverrides, - ksqlConfig + SessionConfig.of(ksqlConfig, mutableSessionPropertyOverrides) )); final CustomExecutor executor = diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/EngineContext.java b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/EngineContext.java index c4298fe81670..204cf6ea719c 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/EngineContext.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/EngineContext.java @@ -19,6 +19,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.ddl.commands.CommandFactories; import io.confluent.ksql.ddl.commands.DdlCommandExec; import io.confluent.ksql.engine.rewrite.AstSanitizer; @@ -37,7 +38,6 @@ import io.confluent.ksql.query.id.QueryIdGenerator; import io.confluent.ksql.services.SandboxedServiceContext; import io.confluent.ksql.services.ServiceContext; -import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.KsqlStatementException; import io.confluent.ksql.util.PersistentQueryMetadata; import io.confluent.ksql.util.QueryMetadata; @@ -181,12 +181,11 @@ QueryEngine createQueryEngine(final ServiceContext serviceContext) { } QueryExecutor createQueryExecutor( - final KsqlConfig ksqlConfig, - final Map overriddenProperties, - final ServiceContext serviceContext) { + final SessionConfig config, + final ServiceContext serviceContext + ) { return new QueryExecutor( - ksqlConfig.cloneWithPropertyOverwrite(overriddenProperties), - overriddenProperties, + config, processingLogContext, serviceContext, metaStore, @@ -197,14 +196,12 @@ QueryExecutor createQueryExecutor( DdlCommand createDdlCommand( final String sqlExpression, final ExecutableDdlStatement statement, - final KsqlConfig ksqlConfig, - final Map overriddenProperties + final SessionConfig config ) { return ddlCommandFactory.create( sqlExpression, statement, - ksqlConfig, - overriddenProperties + config ); } diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/EngineExecutor.java b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/EngineExecutor.java index 35c7d1dab7bc..538c12437a23 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/EngineExecutor.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/EngineExecutor.java @@ -19,6 +19,7 @@ import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import io.confluent.ksql.KsqlExecutionContext.ExecuteResult; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.execution.ddl.commands.CreateSourceCommand; import io.confluent.ksql.execution.ddl.commands.CreateStreamCommand; import io.confluent.ksql.execution.ddl.commands.CreateTableCommand; @@ -27,7 +28,9 @@ import io.confluent.ksql.execution.plan.Formats; import io.confluent.ksql.metastore.model.DataSource; import io.confluent.ksql.name.SourceName; +import io.confluent.ksql.parser.properties.with.SourcePropertiesUtil; import io.confluent.ksql.parser.tree.CreateAsSelect; +import io.confluent.ksql.parser.tree.CreateSource; import io.confluent.ksql.parser.tree.CreateStreamAsSelect; import io.confluent.ksql.parser.tree.CreateTableAsSelect; import io.confluent.ksql.parser.tree.ExecutableDdlStatement; @@ -44,6 +47,10 @@ import io.confluent.ksql.query.QueryExecutor; import io.confluent.ksql.query.QueryId; import io.confluent.ksql.schema.ksql.LogicalSchema; +import io.confluent.ksql.serde.Format; +import io.confluent.ksql.serde.FormatFactory; +import io.confluent.ksql.serde.FormatInfo; +import io.confluent.ksql.serde.KeyFormatUtils; import io.confluent.ksql.services.ServiceContext; import io.confluent.ksql.statement.ConfiguredStatement; import io.confluent.ksql.util.AvroUtil; @@ -53,7 +60,6 @@ import io.confluent.ksql.util.PersistentQueryMetadata; import io.confluent.ksql.util.PlanSummary; import io.confluent.ksql.util.TransientQueryMetadata; -import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -75,31 +81,26 @@ final class EngineExecutor { private final EngineContext engineContext; private final ServiceContext serviceContext; - private final KsqlConfig ksqlConfig; - private final Map overriddenProperties; + private final SessionConfig config; private EngineExecutor( final EngineContext engineContext, final ServiceContext serviceContext, - final KsqlConfig ksqlConfig, - final Map overriddenProperties + final SessionConfig config ) { this.engineContext = Objects.requireNonNull(engineContext, "engineContext"); this.serviceContext = Objects.requireNonNull(serviceContext, "serviceContext"); - this.ksqlConfig = Objects.requireNonNull(ksqlConfig, "ksqlConfig"); - this.overriddenProperties = - Objects.requireNonNull(overriddenProperties, "overriddenProperties"); + this.config = Objects.requireNonNull(config, "config"); - KsqlEngineProps.throwOnImmutableOverride(overriddenProperties); + KsqlEngineProps.throwOnImmutableOverride(config.getOverrides()); } static EngineExecutor create( final EngineContext engineContext, final ServiceContext serviceContext, - final KsqlConfig ksqlConfig, - final Map overriddenProperties + final SessionConfig config ) { - return new EngineExecutor(engineContext, serviceContext, ksqlConfig, overriddenProperties); + return new EngineExecutor(engineContext, serviceContext, config); } ExecuteResult execute(final KsqlPlan plan) { @@ -122,11 +123,8 @@ ExecuteResult execute(final KsqlPlan plan) { TransientQueryMetadata executeQuery(final ConfiguredStatement statement) { final ExecutorPlans plans = planQuery(statement, statement.getStatement(), Optional.empty()); final OutputNode outputNode = plans.logicalPlan.getNode().get(); - final QueryExecutor executor = engineContext.createQueryExecutor( - ksqlConfig, - overriddenProperties, - serviceContext - ); + final QueryExecutor executor = engineContext.createQueryExecutor(config, serviceContext); + return executor.buildTransientQuery( statement.getStatementText(), plans.physicalPlan.getQueryId(), @@ -145,13 +143,13 @@ TransientQueryMetadata executeQuery(final ConfiguredStatement statement) KsqlPlan plan(final ConfiguredStatement statement) { try { throwOnNonExecutableStatement(statement); + throwOnUnsupportedKeyFormat(statement); if (statement.getStatement() instanceof ExecutableDdlStatement) { final DdlCommand ddlCommand = engineContext.createDdlCommand( statement.getStatementText(), (ExecutableDdlStatement) statement.getStatement(), - ksqlConfig, - overriddenProperties + config ); return KsqlPlan.ddlPlanCurrent(statement.getStatementText(), ddlCommand); @@ -198,12 +196,12 @@ private ExecutorPlans planQuery( final Query query, final Optional sink) { final QueryEngine queryEngine = engineContext.createQueryEngine(serviceContext); - final KsqlConfig config = this.ksqlConfig.cloneWithPropertyOverwrite(overriddenProperties); + final KsqlConfig ksqlConfig = config.getConfig(true); final OutputNode outputNode = QueryEngine.buildQueryLogicalPlan( query, sink, engineContext.getMetaStore(), - config + ksqlConfig ); final LogicalPlanNode logicalPlan = new LogicalPlanNode( statement.getStatementText(), @@ -213,12 +211,11 @@ private ExecutorPlans planQuery( engineContext.getMetaStore(), engineContext.idGenerator(), outputNode, - config.getBoolean(KsqlConfig.KSQL_CREATE_OR_REPLACE_ENABLED) + ksqlConfig.getBoolean(KsqlConfig.KSQL_CREATE_OR_REPLACE_ENABLED) ); final PhysicalPlan physicalPlan = queryEngine.buildPhysicalPlan( logicalPlan, - ksqlConfig, - overriddenProperties, + config, engineContext.getMetaStore(), queryId ); @@ -315,6 +312,28 @@ private void validateExistingSink( } } + private void throwOnUnsupportedKeyFormat(final ConfiguredStatement statement) { + if (statement.getStatement() instanceof CreateSource) { + final CreateSource createSource = (CreateSource) statement.getStatement(); + throwOnUnsupportedKeyFormat( + SourcePropertiesUtil.getKeyFormat(createSource.getProperties())); + } + + if (statement.getStatement() instanceof CreateAsSelect) { + final CreateAsSelect createAsSelect = (CreateAsSelect) statement.getStatement(); + createAsSelect.getProperties().getKeyFormat() + .ifPresent(this::throwOnUnsupportedKeyFormat); + } + } + + private void throwOnUnsupportedKeyFormat(final FormatInfo formatInfo) { + final KsqlConfig ksqlConfig = config.getConfig(true); + final Format format = FormatFactory.of(formatInfo); + if (!KeyFormatUtils.isSupportedKeyFormat(ksqlConfig, format)) { + throw new KsqlException("The key format '" + format.name() + "' is not currently supported."); + } + } + private static void validateQuery( final DataSourceType dataSourceType, final ConfiguredStatement statement @@ -368,8 +387,7 @@ private PersistentQueryMetadata executePersistentQuery( final String statementText ) { final QueryExecutor executor = engineContext.createQueryExecutor( - ksqlConfig, - overriddenProperties, + config, serviceContext ); @@ -387,6 +405,7 @@ private PersistentQueryMetadata executePersistentQuery( } private String buildPlanSummary(final QueryId queryId, final ExecutionStep plan) { - return new PlanSummary(queryId, ksqlConfig, engineContext.getMetaStore()).summarize(plan); + return new PlanSummary(queryId, config.getConfig(false), engineContext.getMetaStore()) + .summarize(plan); } } diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java index f8ce3b8d5474..a7f54a7e3d1a 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/InsertValuesExecutor.java @@ -129,8 +129,7 @@ public void execute( ) { final InsertValues insertValues = statement.getStatement(); final MetaStore metaStore = executionContext.getMetaStore(); - final KsqlConfig config = statement.getConfig() - .cloneWithPropertyOverwrite(statement.getConfigOverrides()); + final KsqlConfig config = statement.getSessionConfig().getConfig(true); final DataSource dataSource = getDataSource(config, metaStore, insertValues); @@ -184,11 +183,10 @@ private ProducerRecord buildRecord( final DataSource dataSource, final ServiceContext serviceContext ) { - throwIfDisabled(statement.getConfig()); + throwIfDisabled(statement.getSessionConfig().getConfig(false)); final InsertValues insertValues = statement.getStatement(); - final KsqlConfig config = statement.getConfig() - .cloneWithPropertyOverwrite(statement.getConfigOverrides()); + final KsqlConfig config = statement.getSessionConfig().getConfig(true); try { final KsqlGenericRecord row = new GenericRecordFactory(config, metaStore, clock).build( diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/KsqlEngine.java b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/KsqlEngine.java index 8f6bfbea984f..6a2341c3dbfc 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/KsqlEngine.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/KsqlEngine.java @@ -187,8 +187,7 @@ public KsqlPlan plan( final ConfiguredStatement statement ) { return EngineExecutor - .create( - primaryContext, serviceContext, statement.getConfig(), statement.getConfigOverrides()) + .create(primaryContext, serviceContext, statement.getSessionConfig()) .plan(statement); } @@ -196,7 +195,7 @@ public KsqlPlan plan( public ExecuteResult execute(final ServiceContext serviceContext, final ConfiguredKsqlPlan plan) { try { final ExecuteResult result = EngineExecutor - .create(primaryContext, serviceContext, plan.getConfig(), plan.getOverrides()) + .create(primaryContext, serviceContext, plan.getConfig()) .execute(plan.getPlan()); result.getQuery().ifPresent(this::registerQuery); return result; @@ -221,8 +220,7 @@ public ExecuteResult execute( serviceContext, ConfiguredKsqlPlan.of( plan(serviceContext, statement), - statement.getConfigOverrides(), - statement.getConfig() + statement.getSessionConfig() ) ); } @@ -234,12 +232,9 @@ public TransientQueryMetadata executeQuery( ) { try { final TransientQueryMetadata query = EngineExecutor - .create( - primaryContext, - serviceContext, - statement.getConfig(), - statement.getConfigOverrides()) + .create(primaryContext, serviceContext, statement.getSessionConfig()) .executeQuery(statement); + registerQuery(query); primaryContext.registerQuery(query); return query; diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/KsqlPlanV1.java b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/KsqlPlanV1.java index 001b0cf01b59..5036d26a9ee7 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/KsqlPlanV1.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/KsqlPlanV1.java @@ -21,6 +21,16 @@ import java.util.Optional; final class KsqlPlanV1 implements KsqlPlan { + /** + * This text should NEVER be parsed nor used when executing statements. Rather, it is only here to + * display back to the user in the case of error messages; {@code EXPLAIN }, + * {@code DESCRIBE }, and other similar commands; and in case-insensitive string + * comparison for {@code TERMINATE CLUSTER;} commands. + * + *

In light of the above, this field is NOT included in the {@code equals()} or + * {@code hashCode()} methods of this class, and is also not validated to be unchanging by + * query translation test historic plans. + */ private final String statementText; private final Optional ddlCommand; private final Optional queryPlan; @@ -35,7 +45,7 @@ final class KsqlPlanV1 implements KsqlPlan { this.queryPlan = Objects.requireNonNull(queryPlan, "queryPlan"); if (!ddlCommand.isPresent() && !queryPlan.isPresent()) { - throw new IllegalArgumentException("Plan requires at least a DDL command or queyr plan."); + throw new IllegalArgumentException("Plan requires at least a DDL command or query plan."); } } @@ -65,13 +75,12 @@ public boolean equals(final Object o) { return false; } final KsqlPlanV1 that = (KsqlPlanV1) o; - return Objects.equals(statementText, that.statementText) - && Objects.equals(ddlCommand, that.ddlCommand) + return Objects.equals(ddlCommand, that.ddlCommand) && Objects.equals(queryPlan, that.queryPlan); } @Override public int hashCode() { - return Objects.hash(statementText, ddlCommand, queryPlan); + return Objects.hash(ddlCommand, queryPlan); } } diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/QueryEngine.java b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/QueryEngine.java index 8b3c6a4d303e..4a3d3c2d9670 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/QueryEngine.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/QueryEngine.java @@ -17,6 +17,7 @@ import io.confluent.ksql.analyzer.Analysis; import io.confluent.ksql.analyzer.QueryAnalyzer; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.logging.processing.ProcessingLogContext; import io.confluent.ksql.metastore.MetaStore; import io.confluent.ksql.metastore.MutableMetaStore; @@ -30,7 +31,6 @@ import io.confluent.ksql.query.QueryId; import io.confluent.ksql.services.ServiceContext; import io.confluent.ksql.util.KsqlConfig; -import java.util.Map; import java.util.Objects; import java.util.Optional; import org.apache.kafka.streams.StreamsBuilder; @@ -71,8 +71,7 @@ static OutputNode buildQueryLogicalPlan( PhysicalPlan buildPhysicalPlan( final LogicalPlanNode logicalPlanNode, - final KsqlConfig ksqlConfig, - final Map overriddenProperties, + final SessionConfig config, final MutableMetaStore metaStore, final QueryId queryId ) { @@ -82,7 +81,7 @@ PhysicalPlan buildPhysicalPlan( // Build a physical plan, in this case a Kafka Streams DSL final PhysicalPlanBuilder physicalPlanBuilder = new PhysicalPlanBuilder( builder, - ksqlConfig.cloneWithPropertyOverwrite(overriddenProperties), + config.getConfig(true), serviceContext, processingLogContext, metaStore diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/SandboxedExecutionContext.java b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/SandboxedExecutionContext.java index 5f2c5210abba..8dcbc0e1b01a 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/engine/SandboxedExecutionContext.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/engine/SandboxedExecutionContext.java @@ -103,8 +103,7 @@ public KsqlPlan plan( return EngineExecutor.create( engineContext, serviceContext, - statement.getConfig(), - statement.getConfigOverrides() + statement.getSessionConfig() ).plan(statement); } @@ -116,8 +115,7 @@ public ExecuteResult execute( return EngineExecutor.create( engineContext, serviceContext, - ksqlPlan.getConfig(), - ksqlPlan.getOverrides() + ksqlPlan.getConfig() ).execute(ksqlPlan.getPlan()); } @@ -128,11 +126,7 @@ public ExecuteResult execute( ) { return execute( serviceContext, - ConfiguredKsqlPlan.of( - plan(serviceContext, statement), - statement.getConfigOverrides(), - statement.getConfig() - ) + ConfiguredKsqlPlan.of(plan(serviceContext, statement), statement.getSessionConfig()) ); } @@ -144,8 +138,7 @@ public TransientQueryMetadata executeQuery( return EngineExecutor.create( engineContext, serviceContext, - statement.getConfig(), - statement.getConfigOverrides() + statement.getSessionConfig() ).executeQuery(statement); } } diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/format/DefaultFormatInjector.java b/ksqldb-engine/src/main/java/io/confluent/ksql/format/DefaultFormatInjector.java new file mode 100644 index 000000000000..3cde38f4b280 --- /dev/null +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/format/DefaultFormatInjector.java @@ -0,0 +1,169 @@ +/* + * Copyright 2020 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.format; + +import io.confluent.ksql.parser.KsqlParser.PreparedStatement; +import io.confluent.ksql.parser.SqlFormatter; +import io.confluent.ksql.parser.exception.ParseFailedException; +import io.confluent.ksql.parser.properties.with.CreateSourceProperties; +import io.confluent.ksql.parser.tree.CreateSource; +import io.confluent.ksql.parser.tree.Statement; +import io.confluent.ksql.properties.with.CommonCreateConfigs; +import io.confluent.ksql.serde.FormatInfo; +import io.confluent.ksql.statement.ConfiguredStatement; +import io.confluent.ksql.statement.Injector; +import io.confluent.ksql.util.KsqlConfig; +import io.confluent.ksql.util.KsqlException; +import io.confluent.ksql.util.KsqlStatementException; +import java.util.Optional; + +/** + * An injector which injects the key and value formats into the supplied {@code statement}. + * + *

The key format is only injected if: + *

    + *
  • The statement is a CT/CS.
  • + *
  • The statement does not specify the FORMAT property in its WITH clause.
  • + *
  • The statement does not specify the KEY_FORMAT property in its WITH clause.
  • + *
+ * + *

Similarly, the value format is only injected if the above conditions are met, + * where the KEY_FORMAT property is replaced with the VALUE_FORMAT property accordingly. + * + *

If any of the above are not true then the {@code statement} is returned unchanged. + */ +public class DefaultFormatInjector implements Injector { + + public DefaultFormatInjector() { + } + + @SuppressWarnings("unchecked") + @Override + public ConfiguredStatement inject( + final ConfiguredStatement statement + ) { + if (featureFlagNotEnabled(statement)) { + validateLegacyFormatProperties(statement); + } + + if (statement.getStatement() instanceof CreateSource) { + return handleCreateSource((ConfiguredStatement) statement); + } + + return statement; + } + + @SuppressWarnings("unchecked") + private ConfiguredStatement handleCreateSource( + final ConfiguredStatement statement + ) { + try { + // Safe to cast as we know `T` is `CreateSource` + return (ConfiguredStatement) + injectForCreateStatement(statement).orElse(statement); + } catch (final KsqlStatementException e) { + throw e; + } catch (final KsqlException e) { + throw new KsqlStatementException( + e.getMessage(), + statement.getStatementText(), + e.getCause()); + } + } + + private Optional> injectForCreateStatement( + final ConfiguredStatement original + ) { + + final CreateSource statement = original.getStatement(); + final CreateSourceProperties properties = statement.getProperties(); + + final Optional keyFormat = properties.getKeyFormat(); + final Optional valueFormat = properties.getValueFormat(); + + if (keyFormat.isPresent() && valueFormat.isPresent()) { + return Optional.empty(); + } + + final KsqlConfig config = getConfig(original); + + final CreateSourceProperties injectedProps = properties.withFormats( + keyFormat.map(FormatInfo::getFormat) + .orElseGet(() -> getDefaultKeyFormat(config)), + valueFormat.map(FormatInfo::getFormat) + .orElseGet(() -> getDefaultValueFormat(config)) + ); + final CreateSource withFormats = statement.copyWith( + original.getStatement().getElements(), + injectedProps + ); + + final PreparedStatement prepared = buildPreparedStatement(withFormats); + final ConfiguredStatement configured = ConfiguredStatement + .of(prepared, original.getSessionConfig()); + + return Optional.of(configured); + } + + private static KsqlConfig getConfig(final ConfiguredStatement statement) { + return statement.getSessionConfig().getConfig(true); + } + + private static String getDefaultKeyFormat(final KsqlConfig config) { + final String format = config.getString(KsqlConfig.KSQL_DEFAULT_KEY_FORMAT_CONFIG); + if (format == null) { + throw new KsqlException("Statement is missing the '" + + CommonCreateConfigs.KEY_FORMAT_PROPERTY + "' property from the WITH clause. " + + "Either provide one or set a default via the '" + + KsqlConfig.KSQL_DEFAULT_KEY_FORMAT_CONFIG + "' config."); + } + + return format; + } + + private static String getDefaultValueFormat(final KsqlConfig config) { + final String format = config.getString(KsqlConfig.KSQL_DEFAULT_VALUE_FORMAT_CONFIG); + if (format == null) { + throw new KsqlException("Statement is missing the '" + + CommonCreateConfigs.VALUE_FORMAT_PROPERTY + "' property from the WITH clause. " + + "Either provide one or set a default via the '" + + KsqlConfig.KSQL_DEFAULT_VALUE_FORMAT_CONFIG + "' config."); + } + + return format; + } + + private static void validateLegacyFormatProperties(final ConfiguredStatement statement) { + if (statement.getStatement() instanceof CreateSource) { + final CreateSource createStatement = (CreateSource) statement.getStatement(); + + if (!createStatement.getProperties().getValueFormat().isPresent()) { + throw new ParseFailedException("Failed to prepare statement: Missing required property " + + "\"VALUE_FORMAT\" which has no default value."); + } + } + } + + private static boolean featureFlagNotEnabled(final ConfiguredStatement statement) { + return !getConfig(statement).getBoolean(KsqlConfig.KSQL_KEY_FORMAT_ENABLED); + } + + private static PreparedStatement buildPreparedStatement( + final CreateSource stmt + ) { + return PreparedStatement.of(SqlFormatter.formatSql(stmt), stmt); + } +} diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/ConfiguredKsqlPlan.java b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/ConfiguredKsqlPlan.java index 182aa02419a6..2e8cc7466827 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/ConfiguredKsqlPlan.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/planner/plan/ConfiguredKsqlPlan.java @@ -15,32 +15,27 @@ package io.confluent.ksql.planner.plan; -import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.KsqlPlan; -import io.confluent.ksql.util.KsqlConfig; -import java.util.Map; import java.util.Objects; public final class ConfiguredKsqlPlan { + private final KsqlPlan plan; - private final Map overrides; - private final KsqlConfig config; + private final SessionConfig config; public static ConfiguredKsqlPlan of( final KsqlPlan plan, - final Map overrides, - final KsqlConfig config + final SessionConfig config ) { - return new ConfiguredKsqlPlan(plan, overrides, config); + return new ConfiguredKsqlPlan(plan, config); } private ConfiguredKsqlPlan( final KsqlPlan plan, - final Map overrides, - final KsqlConfig config + final SessionConfig config ) { this.plan = Objects.requireNonNull(plan, "plan"); - this.overrides = ImmutableMap.copyOf(Objects.requireNonNull(overrides, "overrides")); this.config = Objects.requireNonNull(config, "config"); } @@ -48,11 +43,7 @@ public KsqlPlan getPlan() { return plan; } - public Map getOverrides() { - return overrides; - } - - public KsqlConfig getConfig() { + public SessionConfig getConfig() { return config; } @@ -66,12 +57,11 @@ public boolean equals(final Object o) { } final ConfiguredKsqlPlan that = (ConfiguredKsqlPlan) o; return Objects.equals(plan, that.plan) - && Objects.equals(overrides, that.overrides) && Objects.equals(config, that.config); } @Override public int hashCode() { - return Objects.hash(plan, overrides, config); + return Objects.hash(plan, config); } } diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/properties/PropertyOverrider.java b/ksqldb-engine/src/main/java/io/confluent/ksql/properties/PropertyOverrider.java index d586d55b9773..a27b1325510a 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/properties/PropertyOverrider.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/properties/PropertyOverrider.java @@ -26,7 +26,8 @@ public final class PropertyOverrider { - private PropertyOverrider() { } + private PropertyOverrider() { + } public static void set( final ConfiguredStatement statement, @@ -52,10 +53,13 @@ private static void throwIfInvalidPropertyValues( final SetProperty setProperty, final ConfiguredStatement statement) { try { - statement.getConfig().cloneWithPropertyOverwrite(ImmutableMap.of( - setProperty.getPropertyName(), - setProperty.getPropertyValue() - )); + statement + .getSessionConfig() + .getConfig(false) + .cloneWithPropertyOverwrite(ImmutableMap.of( + setProperty.getPropertyName(), + setProperty.getPropertyValue() + )); } catch (final Exception e) { throw new KsqlStatementException( e.getMessage(), statement.getStatementText(), e.getCause()); diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/query/QueryExecutor.java b/ksqldb-engine/src/main/java/io/confluent/ksql/query/QueryExecutor.java index 88ed6c9ca8d5..90e87a690c83 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/query/QueryExecutor.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/query/QueryExecutor.java @@ -17,9 +17,11 @@ import static io.confluent.ksql.util.KsqlConfig.KSQL_SHUTDOWN_TIMEOUT_MS_CONFIG; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import io.confluent.ksql.GenericRow; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.errors.ProductionExceptionHandlerUtil; import io.confluent.ksql.execution.builder.KsqlQueryBuilder; import io.confluent.ksql.execution.materialization.MaterializationInfo; @@ -69,8 +71,7 @@ // CHECKSTYLE_RULES.OFF: ClassDataAbstractionCoupling public final class QueryExecutor { // CHECKSTYLE_RULES.ON: ClassDataAbstractionCoupling - private final KsqlConfig ksqlConfig; - private final Map overrides; + private final SessionConfig config; private final ProcessingLogContext processingLogContext; private final ServiceContext serviceContext; private final FunctionRegistry functionRegistry; @@ -80,15 +81,13 @@ public final class QueryExecutor { private final MaterializationProviderBuilderFactory materializationProviderBuilderFactory; public QueryExecutor( - final KsqlConfig ksqlConfig, - final Map overrides, + final SessionConfig config, final ProcessingLogContext processingLogContext, final ServiceContext serviceContext, final FunctionRegistry functionRegistry, final Consumer queryCloseCallback) { this( - ksqlConfig, - overrides, + config, processingLogContext, serviceContext, functionRegistry, @@ -97,17 +96,16 @@ public QueryExecutor( Objects.requireNonNull(serviceContext, "serviceContext").getKafkaClientSupplier()), new StreamsBuilder(), new MaterializationProviderBuilderFactory( - ksqlConfig, + config.getConfig(true), serviceContext, new KsMaterializationFactory(), new KsqlMaterializationFactory(processingLogContext) - ) - ); + )); } + @VisibleForTesting QueryExecutor( - final KsqlConfig ksqlConfig, - final Map overrides, + final SessionConfig config, final ProcessingLogContext processingLogContext, final ServiceContext serviceContext, final FunctionRegistry functionRegistry, @@ -116,8 +114,7 @@ public QueryExecutor( final StreamsBuilder streamsBuilder, final MaterializationProviderBuilderFactory materializationProviderBuilderFactory ) { - this.ksqlConfig = Objects.requireNonNull(ksqlConfig, "ksqlConfig"); - this.overrides = Objects.requireNonNull(overrides, "overrides"); + this.config = Objects.requireNonNull(config, "config"); this.processingLogContext = Objects.requireNonNull( processingLogContext, "processingLogContext" @@ -147,6 +144,8 @@ public TransientQueryMetadata buildTransientQuery( ) { final BlockingRowQueue queue = buildTransientQueryQueue(queryId, physicalPlan, limit); + final KsqlConfig ksqlConfig = config.getConfig(true); + final String applicationId = QueryApplicationId.build(ksqlConfig, false, queryId); final Map streamsProperties = buildStreamsProperties(applicationId, queryId); @@ -162,7 +161,7 @@ public TransientQueryMetadata buildTransientQuery( topology, kafkaStreamsBuilder, streamsProperties, - overrides, + config.getOverrides(), queryCloseCallback, ksqlConfig.getLong(KSQL_SHUTDOWN_TIMEOUT_MS_CONFIG), ksqlConfig.getInt(KsqlConfig.KSQL_QUERY_ERROR_MAX_QUEUE_SIZE) @@ -187,6 +186,9 @@ public PersistentQueryMetadata buildPersistentQuery( final KsqlQueryBuilder ksqlQueryBuilder = queryBuilder(queryId); final PlanBuilder planBuilder = new KSPlanBuilder(ksqlQueryBuilder); final Object result = physicalPlan.build(planBuilder); + + final KsqlConfig ksqlConfig = config.getConfig(true); + final String applicationId = QueryApplicationId.build(ksqlConfig, true, queryId); final Map streamsProperties = buildStreamsProperties(applicationId, queryId); final Topology topology = streamsBuilder.build(PropertiesUtil.asProperties(streamsProperties)); @@ -224,7 +226,7 @@ public PersistentQueryMetadata buildPersistentQuery( kafkaStreamsBuilder, ksqlQueryBuilder.getSchemas(), streamsProperties, - overrides, + config.getOverrides(), queryCloseCallback, ksqlConfig.getLong(KSQL_SHUTDOWN_TIMEOUT_MS_CONFIG), classifier, @@ -236,7 +238,8 @@ public PersistentQueryMetadata buildPersistentQuery( private TransientQueryQueue buildTransientQueryQueue( final QueryId queryId, final ExecutionStep physicalPlan, - final OptionalInt limit) { + final OptionalInt limit + ) { final KsqlQueryBuilder ksqlQueryBuilder = queryBuilder(queryId); final PlanBuilder planBuilder = new KSPlanBuilder(ksqlQueryBuilder); final Object buildResult = physicalPlan.build(planBuilder); @@ -257,7 +260,7 @@ private TransientQueryQueue buildTransientQueryQueue( private KsqlQueryBuilder queryBuilder(final QueryId queryId) { return KsqlQueryBuilder.of( streamsBuilder, - ksqlConfig, + config.getConfig(true), serviceContext, processingLogContext, functionRegistry, @@ -270,7 +273,7 @@ private Map buildStreamsProperties( final QueryId queryId ) { final Map newStreamsProperties - = new HashMap<>(ksqlConfig.getKsqlStreamConfigProps(applicationId)); + = new HashMap<>(config.getConfig(true).getKsqlStreamConfigProps(applicationId)); newStreamsProperties.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); final ProcessingLogger logger = processingLogContext.getLoggerFactory().getLogger(queryId.toString()); diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/schema/ksql/inference/DefaultSchemaInjector.java b/ksqldb-engine/src/main/java/io/confluent/ksql/schema/ksql/inference/DefaultSchemaInjector.java index 1749cd81eb96..69a854e140cd 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/schema/ksql/inference/DefaultSchemaInjector.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/schema/ksql/inference/DefaultSchemaInjector.java @@ -19,6 +19,7 @@ import io.confluent.ksql.parser.KsqlParser.PreparedStatement; import io.confluent.ksql.parser.SqlFormatter; import io.confluent.ksql.parser.properties.with.CreateSourceProperties; +import io.confluent.ksql.parser.properties.with.SourcePropertiesUtil; import io.confluent.ksql.parser.tree.CreateSource; import io.confluent.ksql.parser.tree.Statement; import io.confluent.ksql.parser.tree.TableElement; @@ -27,6 +28,8 @@ import io.confluent.ksql.schema.ksql.SimpleColumn; import io.confluent.ksql.schema.ksql.inference.TopicSchemaSupplier.SchemaAndId; import io.confluent.ksql.schema.ksql.inference.TopicSchemaSupplier.SchemaResult; +import io.confluent.ksql.serde.FormatFactory; +import io.confluent.ksql.serde.FormatInfo; import io.confluent.ksql.statement.ConfiguredStatement; import io.confluent.ksql.statement.Injector; import io.confluent.ksql.util.ErrorMessageUtil; @@ -88,7 +91,7 @@ private Optional> forCreateStatement( final ConfiguredStatement statement ) { if (hasValueElements(statement) - || !statement.getStatement().getProperties().getValueFormat().supportsSchemaInference()) { + || !supportsSchemaInference(statement)) { return Optional.empty(); } @@ -96,7 +99,7 @@ private Optional> forCreateStatement( final CreateSource withSchema = addSchemaFields(statement, valueSchema); final PreparedStatement prepared = buildPreparedStatement(withSchema); final ConfiguredStatement configured = ConfiguredStatement - .of(prepared, statement.getConfigOverrides(), statement.getConfig()); + .of(prepared, statement.getSessionConfig()); return Optional.of(configured); } @@ -128,6 +131,14 @@ private static boolean hasValueElements( .anyMatch(e -> e.getNamespace().equals(Namespace.VALUE)); } + private static boolean supportsSchemaInference( + final ConfiguredStatement statement + ) { + final FormatInfo valueFormat = + SourcePropertiesUtil.getValueFormat(statement.getStatement().getProperties()); + return FormatFactory.of(valueFormat).supportsSchemaInference(); + } + private static CreateSource addSchemaFields( final ConfiguredStatement preparedStatement, final SchemaAndId schema diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/schema/ksql/inference/SchemaRegisterInjector.java b/ksqldb-engine/src/main/java/io/confluent/ksql/schema/ksql/inference/SchemaRegisterInjector.java index 4db35947aa78..ebc1e5b668b0 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/schema/ksql/inference/SchemaRegisterInjector.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/schema/ksql/inference/SchemaRegisterInjector.java @@ -20,6 +20,7 @@ import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; import io.confluent.ksql.KsqlExecutionContext; import io.confluent.ksql.KsqlExecutionContext.ExecuteResult; +import io.confluent.ksql.parser.properties.with.SourcePropertiesUtil; import io.confluent.ksql.parser.tree.CreateAsSelect; import io.confluent.ksql.parser.tree.CreateSource; import io.confluent.ksql.parser.tree.Statement; @@ -76,20 +77,22 @@ private void registerForCreateSource(final ConfiguredStatement getValueWrapping( final SerdeOptions explicitOptions, final KsqlConfig ksqlConfig ) { - final Optional valueWrapping = explicitOptions.valueWrapping(); + final Optional valueWrapping = explicitOptions + .findAny(SerdeOptions.VALUE_WRAPPING_OPTIONS); + if (valueWrapping.isPresent()) { validateExplicitValueWrapping(singleColumn, valueFormat, valueWrapping.get()); return valueWrapping; diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/statement/ConfiguredStatement.java b/ksqldb-engine/src/main/java/io/confluent/ksql/statement/ConfiguredStatement.java index cf113e5b257a..18e3e3989ce2 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/statement/ConfiguredStatement.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/statement/ConfiguredStatement.java @@ -17,11 +17,10 @@ import static java.util.Objects.requireNonNull; -import com.google.common.collect.ImmutableMap; import com.google.errorprone.annotations.Immutable; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.parser.KsqlParser.PreparedStatement; import io.confluent.ksql.parser.tree.Statement; -import io.confluent.ksql.testing.EffectivelyImmutable; import io.confluent.ksql.util.KsqlConfig; import java.util.Map; import java.util.Objects; @@ -34,50 +33,20 @@ public final class ConfiguredStatement { private final PreparedStatement statement; - @EffectivelyImmutable - private final ImmutableMap configOverrides; - @EffectivelyImmutable - private final ImmutableMap requestProperties; - private final KsqlConfig config; + private final SessionConfig config; public static ConfiguredStatement of( final PreparedStatement statement, - final Map overrides, - final KsqlConfig config + final SessionConfig config ) { - return new ConfiguredStatement<>(statement, overrides, config); - } - - public static ConfiguredStatement of( - final PreparedStatement statement, - final Map overrides, - final Map requestProperties, - final KsqlConfig config - ) { - return new ConfiguredStatement<>(statement, overrides, requestProperties, config); + return new ConfiguredStatement<>(statement, config); } private ConfiguredStatement( final PreparedStatement statement, - final Map configOverrides, - final KsqlConfig config + final SessionConfig config ) { this.statement = requireNonNull(statement, "statement"); - this.configOverrides = ImmutableMap.copyOf(requireNonNull(configOverrides, "overrides")); - this.config = requireNonNull(config, "config"); - this.requestProperties = ImmutableMap.of(); - } - - private ConfiguredStatement( - final PreparedStatement statement, - final Map configOverrides, - final Map requestProperties, - final KsqlConfig config - ) { - this.statement = requireNonNull(statement, "statement"); - this.configOverrides = ImmutableMap.copyOf(requireNonNull(configOverrides, "overrides")); - this.requestProperties = ImmutableMap.copyOf(requireNonNull( - requestProperties, "serverProperties")); this.config = requireNonNull(config, "config"); } @@ -94,35 +63,27 @@ public String getStatementText() { return statement.getStatementText(); } - public Map getConfigOverrides() { - return configOverrides; - } - - public Map getRequestProperties() { - return requestProperties; - } - - public KsqlConfig getConfig() { + public SessionConfig getSessionConfig() { return config; } public ConfiguredStatement withConfig(final KsqlConfig config) { - return new ConfiguredStatement<>(this.statement, this.configOverrides, config); + final SessionConfig newConfig = SessionConfig.of(config, this.config.getOverrides()); + return new ConfiguredStatement<>(statement, newConfig); } public ConfiguredStatement withConfigOverrides(final Map properties) { - return new ConfiguredStatement<>(this.statement, properties, this.config); - } + final SessionConfig newConfig = SessionConfig + .of(this.config.getConfig(false), properties); - public ConfiguredStatement withRequestProperties(final Map properties) { - return new ConfiguredStatement<>(this.statement, this.configOverrides, properties, this.config); + return new ConfiguredStatement<>(statement, newConfig); } public ConfiguredStatement withStatement( final String statementText, final T statement) { return new ConfiguredStatement<>( - PreparedStatement.of(statementText, statement), this.configOverrides, this.config); + PreparedStatement.of(statementText, statement), config); } @Override @@ -135,21 +96,18 @@ public boolean equals(final Object o) { } final ConfiguredStatement that = (ConfiguredStatement) o; return Objects.equals(statement, that.statement) - && Objects.equals(configOverrides, that.configOverrides) && Objects.equals(config, that.config); } @Override public int hashCode() { - return Objects.hash(statement, configOverrides, config); + return Objects.hash(statement, config); } @Override public String toString() { return "ConfiguredStatement{" + "statement=" + statement - + ", configOverrides=" + configOverrides - + ", requestProperties=" + requestProperties + ", config=" + config + '}'; } diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/statement/Injectors.java b/ksqldb-engine/src/main/java/io/confluent/ksql/statement/Injectors.java index 8f7b0a866822..d6f08cf5148b 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/statement/Injectors.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/statement/Injectors.java @@ -16,6 +16,7 @@ package io.confluent.ksql.statement; import io.confluent.ksql.KsqlExecutionContext; +import io.confluent.ksql.format.DefaultFormatInjector; import io.confluent.ksql.schema.ksql.inference.DefaultSchemaInjector; import io.confluent.ksql.schema.ksql.inference.SchemaRegisterInjector; import io.confluent.ksql.schema.ksql.inference.SchemaRegistryTopicSchemaSupplier; @@ -28,6 +29,7 @@ public enum Injectors implements BiFunction { NO_TOPIC_DELETE((ec, sc) -> InjectorChain.of( + new DefaultFormatInjector(), new DefaultSchemaInjector( new SchemaRegistryTopicSchemaSupplier(sc.getSchemaRegistryClient())), new TopicCreateInjector(ec, sc), diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/topic/TopicCreateInjector.java b/ksqldb-engine/src/main/java/io/confluent/ksql/topic/TopicCreateInjector.java index 40d98dd1f19b..c2e9332d2707 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/topic/TopicCreateInjector.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/topic/TopicCreateInjector.java @@ -139,11 +139,8 @@ private ConfiguredStatement injectForCreateAsSelec final ConfiguredStatement statement, final TopicProperties.Builder topicPropertiesBuilder ) { - final String prefix = - statement.getConfigOverrides().getOrDefault( - KsqlConfig.KSQL_OUTPUT_TOPIC_NAME_PREFIX_CONFIG, - statement.getConfig().getString(KsqlConfig.KSQL_OUTPUT_TOPIC_NAME_PREFIX_CONFIG)) - .toString(); + final String prefix = statement.getSessionConfig().getConfig(true) + .getString(KsqlConfig.KSQL_OUTPUT_TOPIC_NAME_PREFIX_CONFIG); final T createAsSelect = statement.getStatement(); final CreateSourceAsProperties properties = createAsSelect.getProperties(); diff --git a/ksqldb-engine/src/main/java/io/confluent/ksql/topic/TopicFactory.java b/ksqldb-engine/src/main/java/io/confluent/ksql/topic/TopicFactory.java index a0b3a020b2e0..532d509d9fdf 100644 --- a/ksqldb-engine/src/main/java/io/confluent/ksql/topic/TopicFactory.java +++ b/ksqldb-engine/src/main/java/io/confluent/ksql/topic/TopicFactory.java @@ -18,7 +18,7 @@ import io.confluent.ksql.execution.ddl.commands.KsqlTopic; import io.confluent.ksql.model.WindowType; import io.confluent.ksql.parser.properties.with.CreateSourceProperties; -import io.confluent.ksql.serde.FormatFactory; +import io.confluent.ksql.parser.properties.with.SourcePropertiesUtil; import io.confluent.ksql.serde.FormatInfo; import io.confluent.ksql.serde.KeyFormat; import io.confluent.ksql.serde.ValueFormat; @@ -37,13 +37,14 @@ public static KsqlTopic create(final CreateSourceProperties properties) { final Optional windowType = properties.getWindowType(); final Optional windowSize = properties.getWindowSize(); + final FormatInfo keyFormatInfo = SourcePropertiesUtil.getKeyFormat(properties); final KeyFormat keyFormat = windowType .map(type -> KeyFormat - .windowed(FormatInfo.of(FormatFactory.KAFKA.name()), WindowInfo.of(type, windowSize))) + .windowed(keyFormatInfo, WindowInfo.of(type, windowSize))) .orElseGet(() -> KeyFormat - .nonWindowed(FormatInfo.of(FormatFactory.KAFKA.name()))); + .nonWindowed(keyFormatInfo)); - final ValueFormat valueFormat = ValueFormat.of(properties.getFormatInfo()); + final ValueFormat valueFormat = ValueFormat.of(SourcePropertiesUtil.getValueFormat(properties)); return new KsqlTopic( kafkaTopicName, diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/analyzer/RewrittenAnalysisTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/analyzer/RewrittenAnalysisTest.java index 673f469244c5..be5447d6b6df 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/analyzer/RewrittenAnalysisTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/analyzer/RewrittenAnalysisTest.java @@ -15,12 +15,6 @@ package io.confluent.ksql.analyzer; -import static org.hamcrest.CoreMatchers.not; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.containsString; -import static org.junit.Assert.assertThrows; -import static org.mockito.Mockito.when; - import io.confluent.ksql.engine.rewrite.ExpressionTreeRewriter; import io.confluent.ksql.execution.expression.tree.Expression; import io.confluent.ksql.execution.windows.HoppingWindowExpression; @@ -33,8 +27,6 @@ import io.confluent.ksql.parser.tree.WindowExpression; import io.confluent.ksql.serde.RefinementInfo; import io.confluent.ksql.util.KsqlException; -import java.util.Optional; -import java.util.function.BiFunction; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -42,6 +34,16 @@ import org.mockito.junit.MockitoJUnit; import org.mockito.junit.MockitoRule; +import java.util.Optional; +import java.util.function.BiFunction; + +import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.MatcherAssert.assertThat; + +import static org.hamcrest.Matchers.containsString; +import static org.junit.Assert.assertThrows; +import static org.mockito.Mockito.when; + public class RewrittenAnalysisTest { @Mock diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/CommandFactoriesTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/CommandFactoriesTest.java index 7fc40055da5f..8af438f561dd 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/CommandFactoriesTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/CommandFactoriesTest.java @@ -26,6 +26,7 @@ import static org.mockito.Mockito.when; import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.execution.ddl.commands.CreateStreamCommand; import io.confluent.ksql.execution.ddl.commands.CreateTableCommand; import io.confluent.ksql.execution.ddl.commands.DdlCommand; @@ -82,7 +83,8 @@ public class CommandFactoriesTest { ELEMENT1 ); private static final String TOPIC_NAME = "some topic"; - private static final Map MINIMIM_PROPS = ImmutableMap.of( + private static final Map MINIMUM_PROPS = ImmutableMap.of( + CommonCreateConfigs.KEY_FORMAT_PROPERTY, new StringLiteral("KAFKA"), CommonCreateConfigs.VALUE_FORMAT_PROPERTY, new StringLiteral("JSON"), CommonCreateConfigs.KAFKA_TOPIC_NAME_PROPERTY, new StringLiteral(TOPIC_NAME) ); @@ -119,10 +121,9 @@ CommonCreateConfigs.KAFKA_TOPIC_NAME_PROPERTY, new StringLiteral(TOPIC_NAME) private CommandFactories commandFactories; private KsqlConfig ksqlConfig = new KsqlConfig(ImmutableMap.of()); private final CreateSourceProperties withProperties = - CreateSourceProperties.from(MINIMIM_PROPS); + CreateSourceProperties.from(MINIMUM_PROPS); @Before - @SuppressWarnings("unchecked") public void before() { when(serviceContext.getTopicClient()).thenReturn(topicClient); when(topicClient.isTopicExists(any())).thenReturn(true); @@ -161,7 +162,7 @@ public void shouldCreateCommandForCreateStream() { // When: final DdlCommand result = commandFactories - .create(sqlExpression, statement, ksqlConfig, emptyMap()); + .create(sqlExpression, statement, SessionConfig.of(ksqlConfig, emptyMap())); assertThat(result, is(createStreamCommand)); verify(createSourceFactory).createStreamCommand(statement, ksqlConfig); @@ -173,7 +174,7 @@ public void shouldCreateCommandForStreamWithOverriddenProperties() { final CreateStream statement = new CreateStream(SOME_NAME, SOME_ELEMENTS, false, true, withProperties); // When: - commandFactories.create(sqlExpression, statement, ksqlConfig, OVERRIDES); + commandFactories.create(sqlExpression, statement, SessionConfig.of(ksqlConfig, OVERRIDES)); verify(createSourceFactory).createStreamCommand( statement, @@ -191,7 +192,7 @@ public void shouldCreateCommandForCreateTable() { // When: final DdlCommand result = commandFactories - .create(sqlExpression, statement, ksqlConfig, emptyMap()); + .create(sqlExpression, statement, SessionConfig.of(ksqlConfig, emptyMap())); // Then: assertThat(result, is(createTableCommand)); @@ -208,7 +209,7 @@ public void shouldCreateCommandForCreateTableWithOverriddenProperties() { false, true, withProperties); // When: - commandFactories.create(sqlExpression, statement, ksqlConfig, OVERRIDES); + commandFactories.create(sqlExpression, statement, SessionConfig.of(ksqlConfig, OVERRIDES)); // Then: verify(createSourceFactory).createTableCommand( @@ -224,7 +225,7 @@ public void shouldCreateCommandForDropStream() { // When: final DdlCommand result = commandFactories - .create(sqlExpression, ddlStatement, ksqlConfig, emptyMap()); + .create(sqlExpression, ddlStatement, SessionConfig.of(ksqlConfig, emptyMap())); // Then: assertThat(result, is(dropSourceCommand)); @@ -238,7 +239,7 @@ public void shouldCreateCommandForDropTable() { // When: final DdlCommand result = commandFactories - .create(sqlExpression, ddlStatement, ksqlConfig, emptyMap()); + .create(sqlExpression, ddlStatement, SessionConfig.of(ksqlConfig, emptyMap())); // Then: assertThat(result, is(dropSourceCommand)); @@ -257,7 +258,7 @@ public void shouldCreateCommandForRegisterType() { // When: final DdlCommand result = commandFactories.create( - sqlExpression, ddlStatement, ksqlConfig, emptyMap()); + sqlExpression, ddlStatement, SessionConfig.of(ksqlConfig, emptyMap())); // Then: assertThat(result, is(registerTypeCommand)); @@ -273,8 +274,7 @@ public void shouldCreateDropType() { final DropTypeCommand cmd = (DropTypeCommand) commandFactories.create( "sqlExpression", dropType, - ksqlConfig, - emptyMap() + SessionConfig.of(ksqlConfig, emptyMap()) ); // Then: @@ -289,7 +289,7 @@ public void shouldThrowOnUnsupportedStatementType() { }; // Then: - commandFactories.create(sqlExpression, ddlStatement, ksqlConfig, emptyMap()); + commandFactories.create(sqlExpression, ddlStatement, SessionConfig.of(ksqlConfig, emptyMap())); } @Test @@ -309,7 +309,7 @@ public void shouldCreateStreamCommandWithSingleValueWrappingFromOverridesNotConf // When: final DdlCommand cmd = commandFactories - .create(sqlExpression, statement, ksqlConfig, overrides); + .create(sqlExpression, statement, SessionConfig.of(ksqlConfig, overrides)); // Then: assertThat(cmd, is(instanceOf(CreateStreamCommand.class))); @@ -334,7 +334,7 @@ public void shouldCreateTableCommandWithSingleValueWrappingFromOverridesNotConfi // When: final DdlCommand cmd = commandFactories - .create(sqlExpression, statement, ksqlConfig, overrides); + .create(sqlExpression, statement, SessionConfig.of(ksqlConfig, overrides)); // Then: assertThat(cmd, is(instanceOf(CreateTableCommand.class))); diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/CreateSourceFactoryTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/CreateSourceFactoryTest.java index 8b81bc3bf498..4813dcab34fc 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/CreateSourceFactoryTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/ddl/commands/CreateSourceFactoryTest.java @@ -55,6 +55,7 @@ import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.name.SourceName; import io.confluent.ksql.parser.properties.with.CreateSourceProperties; +import io.confluent.ksql.parser.properties.with.SourcePropertiesUtil; import io.confluent.ksql.parser.tree.CreateStream; import io.confluent.ksql.parser.tree.CreateTable; import io.confluent.ksql.parser.tree.TableElement; @@ -66,6 +67,7 @@ import io.confluent.ksql.schema.ksql.SystemColumns; import io.confluent.ksql.schema.ksql.types.SqlTypes; import io.confluent.ksql.serde.EnabledSerdeFeatures; +import io.confluent.ksql.serde.FormatFactory; import io.confluent.ksql.serde.FormatInfo; import io.confluent.ksql.serde.KeySerdeFactory; import io.confluent.ksql.serde.SerdeOption; @@ -125,6 +127,7 @@ public class CreateSourceFactoryTest { private static final String TOPIC_NAME = "some topic"; private static final Map MINIMUM_PROPS = ImmutableMap.of( + CommonCreateConfigs.KEY_FORMAT_PROPERTY, new StringLiteral("KAFKA"), CommonCreateConfigs.VALUE_FORMAT_PROPERTY, new StringLiteral("JSON"), CommonCreateConfigs.KAFKA_TOPIC_NAME_PROPERTY, new StringLiteral(TOPIC_NAME) ); @@ -459,8 +462,8 @@ public void shouldBuildSerdeOptionsForStream() { // Then: verify(serdeOptionsSupplier).build( schema, - KAFKA, - statement.getProperties().getValueFormat(), + FormatFactory.of(SourcePropertiesUtil.getKeyFormat(statement.getProperties())), + FormatFactory.of(SourcePropertiesUtil.getValueFormat(statement.getProperties())), statement.getProperties().getSerdeOptions(), ksqlConfig ); diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/embedded/KsqlContextTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/embedded/KsqlContextTest.java index 115f17d50664..bde3fd0b4ec8 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/embedded/KsqlContextTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/embedded/KsqlContextTest.java @@ -32,6 +32,7 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import io.confluent.ksql.KsqlExecutionContext; import io.confluent.ksql.KsqlExecutionContext.ExecuteResult; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.parser.KsqlParser.ParsedStatement; import io.confluent.ksql.parser.KsqlParser.PreparedStatement; @@ -79,22 +80,22 @@ public class KsqlContextTest { .of("sql 1", mock(Statement.class)); private final static ConfiguredStatement CFG_STMT_0 = ConfiguredStatement.of( - PREPARED_STMT_0, SOME_PROPERTIES, SOME_CONFIG); + PREPARED_STMT_0, SessionConfig.of(SOME_CONFIG, SOME_PROPERTIES)); private final static ConfiguredStatement CFG_STMT_1 = ConfiguredStatement.of( - PREPARED_STMT_1, SOME_PROPERTIES, SOME_CONFIG); + PREPARED_STMT_1, SessionConfig.of(SOME_CONFIG, SOME_PROPERTIES)); private final static ConfiguredStatement STMT_0_WITH_SCHEMA = ConfiguredStatement.of( - PREPARED_STMT_0, SOME_PROPERTIES, SOME_CONFIG); + PREPARED_STMT_0, SessionConfig.of(SOME_CONFIG, SOME_PROPERTIES)); private final static ConfiguredStatement STMT_1_WITH_SCHEMA = ConfiguredStatement.of( - PREPARED_STMT_1, SOME_PROPERTIES, SOME_CONFIG); + PREPARED_STMT_1, SessionConfig.of(SOME_CONFIG, SOME_PROPERTIES)); private final static ConfiguredStatement STMT_0_WITH_TOPIC = ConfiguredStatement.of( - PREPARED_STMT_0, SOME_PROPERTIES, SOME_CONFIG); + PREPARED_STMT_0, SessionConfig.of(SOME_CONFIG, SOME_PROPERTIES)); private final static ConfiguredStatement STMT_1_WITH_TOPIC = ConfiguredStatement.of( - PREPARED_STMT_1, SOME_PROPERTIES, SOME_CONFIG); + PREPARED_STMT_1, SessionConfig.of(SOME_CONFIG, SOME_PROPERTIES)); @Mock private ServiceContext serviceContext; @@ -398,12 +399,9 @@ public void shouldSetProperty() { // Then: verify(ksqlEngine).execute( serviceContext, - ConfiguredStatement.of( - PREPARED_STMT_0, ImmutableMap.of( - ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest" - ), - SOME_CONFIG - )); + ConfiguredStatement.of(PREPARED_STMT_0, SessionConfig.of(SOME_CONFIG, ImmutableMap.of( + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest" + )))); } @SuppressWarnings("unchecked") @@ -427,8 +425,7 @@ public void shouldSetPropertyOnlyOnCommandsFollowingTheSetStatement() { // Then: verify(ksqlEngine).execute( serviceContext, - ConfiguredStatement.of( - PREPARED_STMT_0, ImmutableMap.of(), SOME_CONFIG + ConfiguredStatement.of(PREPARED_STMT_0, SessionConfig.of(SOME_CONFIG, ImmutableMap.of()) )); } @@ -455,8 +452,7 @@ public void shouldUnsetProperty() { // Then: verify(ksqlEngine).execute( serviceContext, - ConfiguredStatement.of( - PREPARED_STMT_0, ImmutableMap.of(), SOME_CONFIG + ConfiguredStatement.of(PREPARED_STMT_0, SessionConfig.of(SOME_CONFIG, ImmutableMap.of()) )); } } diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/engine/InsertValuesExecutorTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/engine/InsertValuesExecutorTest.java index 48da962da6a1..6f90630a0d05 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/engine/InsertValuesExecutorTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/engine/InsertValuesExecutorTest.java @@ -33,6 +33,7 @@ import com.google.common.collect.Iterables; import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import io.confluent.ksql.GenericRow; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.execution.ddl.commands.KsqlTopic; import io.confluent.ksql.execution.expression.tree.ArithmeticUnaryExpression; import io.confluent.ksql.execution.expression.tree.BooleanLiteral; @@ -501,6 +502,7 @@ public void shouldThrowWhenInsertValuesOnReservedInternalTopic() { givenDataSourceWithSchema("_confluent-ksql-default__command-topic", SCHEMA, SerdeOptions.of(), false); + final KsqlConfig ksqlConfig = new KsqlConfig(ImmutableMap.of()); final ConfiguredStatement statement = ConfiguredStatement.of( PreparedStatement.of( "", @@ -512,8 +514,7 @@ public void shouldThrowWhenInsertValuesOnReservedInternalTopic() { new StringLiteral("str"), new LongLiteral(2L) ))), - ImmutableMap.of(), - new KsqlConfig(ImmutableMap.of()) + SessionConfig.of(ksqlConfig, ImmutableMap.of()) ); // When: @@ -533,6 +534,7 @@ public void shouldThrowWhenInsertValuesOnProcessingLogTopic() { givenDataSourceWithSchema("default_ksql_processing_log", SCHEMA, SerdeOptions.of(), false); + final KsqlConfig ksqlConfig = new KsqlConfig(ImmutableMap.of()); final ConfiguredStatement statement = ConfiguredStatement.of( PreparedStatement.of( "", @@ -544,8 +546,7 @@ public void shouldThrowWhenInsertValuesOnProcessingLogTopic() { new StringLiteral("str"), new LongLiteral(2L) ))), - ImmutableMap.of(), - new KsqlConfig(ImmutableMap.of()) + SessionConfig.of(ksqlConfig, ImmutableMap.of()) ); // When: @@ -872,13 +873,10 @@ private static ConfiguredStatement givenInsertValues( final List columns, final List values ) { - return ConfiguredStatement.of( - PreparedStatement.of( + return ConfiguredStatement.of(PreparedStatement.of( "", - new InsertValues(SourceName.of("TOPIC"), columns, values)), - ImmutableMap.of(), - new KsqlConfig(ImmutableMap.of()) - ); + new InsertValues(SourceName.of("TOPIC"), columns, values)), SessionConfig.of( + new KsqlConfig(ImmutableMap.of()), ImmutableMap.of())); } private void givenSourceStreamWithSchema( diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/engine/KsqlEngineTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/engine/KsqlEngineTest.java index 74d524300dc0..3134df7b3f09 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/engine/KsqlEngineTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/engine/KsqlEngineTest.java @@ -17,7 +17,6 @@ import static io.confluent.ksql.engine.KsqlEngineTestUtil.execute; import static io.confluent.ksql.metastore.model.MetaStoreMatchers.FieldMatchers.hasFullName; -import static io.confluent.ksql.statement.ConfiguredStatement.of; import static io.confluent.ksql.util.KsqlExceptionMatcher.rawMessage; import static io.confluent.ksql.util.KsqlExceptionMatcher.statementText; import static java.util.Collections.emptyMap; @@ -42,6 +41,7 @@ import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import io.confluent.kafka.schemaregistry.avro.AvroSchema; import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient; @@ -50,6 +50,7 @@ import io.confluent.ksql.KsqlConfigTestUtil; import io.confluent.ksql.KsqlExecutionContext; import io.confluent.ksql.KsqlExecutionContext.ExecuteResult; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.QueryCleanupService.QueryCleanupTask; import io.confluent.ksql.function.InternalFunctionRegistry; import io.confluent.ksql.metastore.MutableMetaStore; @@ -103,7 +104,9 @@ @RunWith(MockitoJUnitRunner.class) public class KsqlEngineTest { - private static final KsqlConfig KSQL_CONFIG = KsqlConfigTestUtil.create("what-eva"); + private static final KsqlConfig KSQL_CONFIG = KsqlConfigTestUtil.create( + "what-eva", + ImmutableMap.of(KsqlConfig.KSQL_KEY_FORMAT_ENABLED, true)); private MutableMetaStore metaStore; @Spy @@ -227,8 +230,8 @@ public void shouldExecuteInsertIntoStreamOnSandBox() { final ExecuteResult result = sandbox .execute(sandboxServiceContext, ConfiguredStatement.of( sandbox.prepare(statements.get(1)), - Collections.emptyMap(), - KSQL_CONFIG)); + SessionConfig.of(KSQL_CONFIG, Collections.emptyMap()) + )); // Then: assertThat(result.getQuery(), is(not(Optional.empty()))); @@ -464,7 +467,7 @@ public void shouldFailDDLStatementIfTopicDoesNotExist() { // Given: final ParsedStatement stmt = parse( "CREATE STREAM S1_NOTEXIST (COL1 BIGINT, COL2 VARCHAR) " - + "WITH (KAFKA_TOPIC = 'S1_NOTEXIST', VALUE_FORMAT = 'JSON');").get(0); + + "WITH (KAFKA_TOPIC = 'S1_NOTEXIST', VALUE_FORMAT = 'JSON', KEY_FORMAT = 'KAFKA');").get(0); final PreparedStatement prepared = prepare(stmt); @@ -473,7 +476,7 @@ public void shouldFailDDLStatementIfTopicDoesNotExist() { KsqlStatementException.class, () -> sandbox.execute( sandboxServiceContext, - ConfiguredStatement.of(prepared, Collections.emptyMap(), KSQL_CONFIG) + ConfiguredStatement.of(prepared, SessionConfig.of(KSQL_CONFIG, Collections.emptyMap())) ) ); @@ -511,11 +514,11 @@ public void shouldDropTableIfAllReferencedQueriesTerminated() { @Test public void shouldNotEnforceTopicExistenceWhileParsing() { final String runScriptContent = "CREATE STREAM S1 (COL1 BIGINT, COL2 VARCHAR) " - + "WITH (KAFKA_TOPIC = 's1_topic', VALUE_FORMAT = 'JSON');\n" + + "WITH (KAFKA_TOPIC = 's1_topic', VALUE_FORMAT = 'JSON', KEY_FORMAT = 'KAFKA');\n" + "CREATE TABLE T1 AS SELECT COL1, count(*) FROM " + "S1 GROUP BY COL1;\n" + "CREATE STREAM S2 (C1 BIGINT, C2 BIGINT) " - + "WITH (KAFKA_TOPIC = 'T1', VALUE_FORMAT = 'JSON');\n"; + + "WITH (KAFKA_TOPIC = 'T1', VALUE_FORMAT = 'JSON', KEY_FORMAT = 'KAFKA');\n"; final List parsedStatements = ksqlEngine.parse(runScriptContent); @@ -527,14 +530,14 @@ public void shouldThrowFromSandBoxOnPrepareIfSourceTopicDoesNotExist() { // Given: final PreparedStatement statement = prepare(parse( "CREATE STREAM S1 (COL1 BIGINT) " - + "WITH (KAFKA_TOPIC = 'i_do_not_exist', VALUE_FORMAT = 'JSON');").get(0)); + + "WITH (KAFKA_TOPIC = 'i_do_not_exist', VALUE_FORMAT = 'JSON', KEY_FORMAT = 'KAFKA');").get(0)); // When: final KsqlStatementException e = assertThrows( KsqlStatementException.class, () -> sandbox.execute( sandboxServiceContext, - ConfiguredStatement.of(statement, new HashMap<>(), KSQL_CONFIG) + ConfiguredStatement.of(statement, SessionConfig.of(KSQL_CONFIG, ImmutableMap.of())) ) ); @@ -543,7 +546,7 @@ public void shouldThrowFromSandBoxOnPrepareIfSourceTopicDoesNotExist() { "Kafka topic does not exist: i_do_not_exist"))); assertThat(e, statementText(is( "CREATE STREAM S1 (COL1 BIGINT)" - + " WITH (KAFKA_TOPIC = 'i_do_not_exist', VALUE_FORMAT = 'JSON');"))); + + " WITH (KAFKA_TOPIC = 'i_do_not_exist', VALUE_FORMAT = 'JSON', KEY_FORMAT = 'KAFKA');"))); } @Test @@ -551,14 +554,14 @@ public void shouldThrowFromExecuteIfSourceTopicDoesNotExist() { // Given: final PreparedStatement statement = prepare(parse( "CREATE STREAM S1 (COL1 BIGINT) " - + "WITH (KAFKA_TOPIC = 'i_do_not_exist', VALUE_FORMAT = 'JSON');").get(0)); + + "WITH (KAFKA_TOPIC = 'i_do_not_exist', VALUE_FORMAT = 'JSON', KEY_FORMAT = 'KAFKA');").get(0)); // When: final KsqlStatementException e = assertThrows( KsqlStatementException.class, () -> ksqlEngine.execute( serviceContext, - ConfiguredStatement.of(statement, new HashMap<>(), KSQL_CONFIG) + ConfiguredStatement.of(statement, SessionConfig.of(KSQL_CONFIG, ImmutableMap.of())) ) ); @@ -571,7 +574,7 @@ public void shouldHandleCommandsSpreadOverMultipleLines() { final String runScriptContent = "CREATE STREAM S1 \n" + "(COL1 BIGINT, COL2 VARCHAR)\n" + " WITH \n" - + "(KAFKA_TOPIC = 's1_topic', VALUE_FORMAT = 'JSON');\n"; + + "(KAFKA_TOPIC = 's1_topic', VALUE_FORMAT = 'JSON', KEY_FORMAT = 'KAFKA');\n"; final List parsedStatements = ksqlEngine.parse(runScriptContent); @@ -589,7 +592,7 @@ public void shouldThrowIfSchemaNotPresent() { () -> execute( serviceContext, ksqlEngine, - "create stream bar with (value_format='avro', kafka_topic='bar');", + "create stream bar with (key_format='kafka', value_format='avro', kafka_topic='bar');", KSQL_CONFIG, emptyMap() ) @@ -601,7 +604,7 @@ public void shouldThrowIfSchemaNotPresent() { "The statement does not define any columns."))); assertThat(e, statementText( is( - "create stream bar with (value_format='avro', kafka_topic='bar');"))); + "create stream bar with (key_format='kafka', value_format='avro', kafka_topic='bar');"))); } @Test @@ -976,10 +979,10 @@ public void shouldHandleMultipleStatements() { + " Multi-line comment\n" + "*/\n" + "CREATE STREAM S0 (a INT, b VARCHAR) " - + " WITH (kafka_topic='s0_topic', value_format='DELIMITED');\n" + + " WITH (kafka_topic='s0_topic', value_format='DELIMITED', key_format='KAFKA');\n" + "\n" + "CREATE TABLE T1 (f0 BIGINT PRIMARY KEY, f1 DOUBLE) " - + " WITH (kafka_topic='t1_topic', value_format='JSON');\n" + + " WITH (kafka_topic='t1_topic', value_format='JSON', key_format='KAFKA');\n" + "\n" + "CREATE STREAM S1 AS SELECT * FROM S0;\n" + "\n" @@ -998,7 +1001,8 @@ public void shouldHandleMultipleStatements() { final PreparedStatement prepared = ksqlEngine.prepare(stmt); final ExecuteResult result = ksqlEngine.execute( serviceContext, - ConfiguredStatement.of(prepared, new HashMap<>(), KSQL_CONFIG)); + ConfiguredStatement + .of(prepared, SessionConfig.of(KSQL_CONFIG, new HashMap<>()))); result.getQuery().ifPresent(queries::add); return prepared; }) @@ -1051,7 +1055,8 @@ public void shouldThrowWhenExecutingDuplicateTable() { KsqlStatementException.class, () -> ksqlEngine.execute( serviceContext, - ConfiguredStatement.of(prepared, new HashMap<>(), KSQL_CONFIG) + ConfiguredStatement + .of(prepared, SessionConfig.of(KSQL_CONFIG, new HashMap<>())) ) ); @@ -1109,7 +1114,8 @@ public void shouldThrowWhenExecutingDuplicateStream() { KsqlStatementException.class, () -> ksqlEngine.execute( serviceContext, - of(prepared, new HashMap<>(), KSQL_CONFIG) + ConfiguredStatement + .of(prepared, SessionConfig.of(KSQL_CONFIG, new HashMap<>())) ) ); @@ -1175,7 +1181,8 @@ public void shouldThrowWhenTryExecuteCsasThatCreatesTable() { KsqlStatementException.class, () -> sandbox.execute( serviceContext, - ConfiguredStatement.of(statement, new HashMap<>(), KSQL_CONFIG) + ConfiguredStatement + .of(statement, SessionConfig.of(KSQL_CONFIG, new HashMap<>())) ) ); @@ -1198,7 +1205,8 @@ public void shouldThrowWhenTryExecuteCtasThatCreatesStream() { KsqlStatementException.class, () -> sandbox.execute( serviceContext, - ConfiguredStatement.of(statement, new HashMap<>(), KSQL_CONFIG) + ConfiguredStatement + .of(statement, SessionConfig.of(KSQL_CONFIG, new HashMap<>())) ) ); @@ -1213,10 +1221,10 @@ public void shouldThrowWhenTryExecuteCtasThatCreatesStream() { @Test public void shouldThrowIfStatementMissingTopicConfig() { final List parsed = parse( - "CREATE TABLE FOO (viewtime BIGINT, pageid VARCHAR) WITH (VALUE_FORMAT='AVRO');" - + "CREATE STREAM FOO (viewtime BIGINT, pageid VARCHAR) WITH (VALUE_FORMAT='AVRO');" - + "CREATE TABLE FOO (viewtime BIGINT, pageid VARCHAR) WITH (VALUE_FORMAT='JSON');" - + "CREATE STREAM FOO (viewtime BIGINT, pageid VARCHAR) WITH (VALUE_FORMAT='JSON');" + "CREATE TABLE FOO (viewtime BIGINT, pageid VARCHAR) WITH (VALUE_FORMAT='AVRO', KEY_FORMAT='KAFKA');" + + "CREATE STREAM FOO (viewtime BIGINT, pageid VARCHAR) WITH (VALUE_FORMAT='AVRO', KEY_FORMAT='KAFKA');" + + "CREATE TABLE FOO (viewtime BIGINT, pageid VARCHAR) WITH (VALUE_FORMAT='JSON', KEY_FORMAT='KAFKA');" + + "CREATE STREAM FOO (viewtime BIGINT, pageid VARCHAR) WITH (VALUE_FORMAT='JSON', KEY_FORMAT='KAFKA');" ); for (final ParsedStatement statement : parsed) { @@ -1232,28 +1240,41 @@ public void shouldThrowIfStatementMissingTopicConfig() { } @Test - public void shouldThrowIfStatementMissingValueFormatConfig() { + public void shouldThrowOnUnsupportedKeyFormatForCreateSource() { // Given: givenTopicsExist("foo"); + final PreparedStatement prepared = + prepare(parse("CREATE STREAM FOO (a int) WITH (kafka_topic='foo', value_format='json', key_format='avro');").get(0)); - final List parsed = parse( - "CREATE TABLE FOO (viewtime BIGINT, pageid VARCHAR) WITH (KAFKA_TOPIC='foo');" - + "CREATE STREAM FOO (viewtime BIGINT, pageid VARCHAR) WITH (KAFKA_TOPIC='foo');" + // When: + final Exception e = assertThrows( + KsqlStatementException.class, + () -> sandbox.execute( + sandboxServiceContext, + ConfiguredStatement.of(prepared, SessionConfig.of(KSQL_CONFIG, Collections.emptyMap())) + ) ); - for (final ParsedStatement statement : parsed) { + // Then: + assertThat(e.getMessage(), containsString("The key format 'AVRO' is not currently supported.")); + } - try { - // When: - ksqlEngine.prepare(statement); + @Test + public void shouldThrowOnUnsupportedKeyFormatForCSAS() { + // When: + final KsqlStatementException e = assertThrows( + KsqlStatementException.class, + () -> KsqlEngineTestUtil.execute( + serviceContext, + ksqlEngine, + "CREATE STREAM FOO WITH (KEY_FORMAT='AVRO') AS SELECT * FROM ORDERS;", + KSQL_CONFIG, + Collections.emptyMap() + ) + ); - // Then: - Assert.fail(); - } catch (final KsqlStatementException e) { - assertThat(e.getMessage(), containsString( - "Missing required property \"VALUE_FORMAT\" which has no default value.")); - } - } + // Then: + assertThat(e, rawMessage(containsString("The key format 'AVRO' is not currently supported."))); } @Test @@ -1282,7 +1303,7 @@ public void shouldNotUpdateMetaStoreDuringTryExecute() { final int numPersistentQueries = ksqlEngine.getPersistentQueries().size(); final List statements = parse( - "CREATE STREAM S1 (COL1 BIGINT) WITH (KAFKA_TOPIC = 's1_topic', VALUE_FORMAT = 'JSON');" + "CREATE STREAM S1 (COL1 BIGINT) WITH (KAFKA_TOPIC = 's1_topic', VALUE_FORMAT = 'JSON', KEY_FORMAT='KAFKA');" + "CREATE TABLE BAR AS SELECT * FROM TEST2;" + "CREATE TABLE FOO AS SELECT * FROM TEST2;" + "DROP TABLE TEST3;"); @@ -1293,7 +1314,9 @@ public void shouldNotUpdateMetaStoreDuringTryExecute() { statements .forEach(stmt -> sandbox.execute( sandboxServiceContext, - ConfiguredStatement.of(sandbox.prepare(stmt), new HashMap<>(), KSQL_CONFIG))); + ConfiguredStatement + .of(sandbox.prepare(stmt), SessionConfig.of(KSQL_CONFIG, new HashMap<>()) + ))); // Then: assertThat(metaStore.getSource(SourceName.of("TEST3")), is(notNullValue())); @@ -1310,7 +1333,7 @@ public void shouldNotCreateAnyTopicsDuringTryExecute() { topicClient.preconditionTopicExists("s1_topic", 1, (short) 1, Collections.emptyMap()); final List statements = parse( - "CREATE STREAM S1 (COL1 BIGINT) WITH (KAFKA_TOPIC = 's1_topic', VALUE_FORMAT = 'JSON');" + "CREATE STREAM S1 (COL1 BIGINT) WITH (KAFKA_TOPIC = 's1_topic', KEY_FORMAT = 'KAFKA', VALUE_FORMAT = 'JSON');" + "CREATE TABLE BAR AS SELECT * FROM TEST2;" + "CREATE TABLE FOO AS SELECT * FROM TEST2;" + "DROP TABLE TEST3;"); @@ -1319,7 +1342,9 @@ public void shouldNotCreateAnyTopicsDuringTryExecute() { statements.forEach( stmt -> sandbox.execute( sandboxServiceContext, - ConfiguredStatement.of(sandbox.prepare(stmt), new HashMap<>(), KSQL_CONFIG)) + ConfiguredStatement + .of(sandbox.prepare(stmt), SessionConfig.of(KSQL_CONFIG, new HashMap<>()) + )) ); // Then: @@ -1336,7 +1361,8 @@ public void shouldNotIncrementQueryIdCounterDuringTryExecute() { // When: sandbox.execute( sandboxServiceContext, - ConfiguredStatement.of(statement, new HashMap<>(), KSQL_CONFIG) + ConfiguredStatement + .of(statement, SessionConfig.of(KSQL_CONFIG, new HashMap<>())) ); // Then: @@ -1360,7 +1386,8 @@ public void shouldNotRegisterAnySchemasDuringSandboxExecute() throws Exception { // When: sandbox.execute( sandboxServiceContext, - ConfiguredStatement.of(prepared, new HashMap<>(), KSQL_CONFIG) + ConfiguredStatement + .of(prepared, SessionConfig.of(KSQL_CONFIG, new HashMap<>())) ); // Then: @@ -1398,7 +1425,8 @@ public void shouldRegisterPersistentQueriesOnlyInSandbox() { // When: final ExecuteResult result = sandbox.execute( sandboxServiceContext, - ConfiguredStatement.of(prepared, new HashMap<>(), KSQL_CONFIG) + ConfiguredStatement + .of(prepared, SessionConfig.of(KSQL_CONFIG, new HashMap<>())) ); // Then: @@ -1458,12 +1486,13 @@ public void shouldExecuteDdlStatement() { // Given: givenTopicsExist("foo"); final PreparedStatement statement = - prepare(parse("CREATE STREAM FOO (a int) WITH (kafka_topic='foo', value_format='json');").get(0)); + prepare(parse("CREATE STREAM FOO (a int) WITH (kafka_topic='foo', value_format='json', key_format='kafka');").get(0)); // When: final ExecuteResult result = sandbox.execute( sandboxServiceContext, - ConfiguredStatement.of(statement, new HashMap<>(), KSQL_CONFIG) + ConfiguredStatement + .of(statement, SessionConfig.of(KSQL_CONFIG, new HashMap<>())) ); // Then: @@ -1595,15 +1624,18 @@ private void givenStatementAlreadyExecuted( ) { ksqlEngine.execute( serviceContext, - ConfiguredStatement.of(ksqlEngine.prepare(statement), new HashMap<>(), KSQL_CONFIG)); + ConfiguredStatement + .of(ksqlEngine.prepare(statement), SessionConfig.of(KSQL_CONFIG, new HashMap<>()) + )); sandbox = ksqlEngine.createSandbox(serviceContext); } private void givenSqlAlreadyExecuted(final String sql) { - parse(sql).forEach(stmt -> - ksqlEngine.execute( - serviceContext, - ConfiguredStatement.of(ksqlEngine.prepare(stmt), new HashMap<>(), KSQL_CONFIG))); + parse(sql).forEach(stmt -> ksqlEngine.execute( + serviceContext, + ConfiguredStatement + .of(ksqlEngine.prepare(stmt), SessionConfig.of(KSQL_CONFIG, new HashMap<>()) + ))); sandbox = ksqlEngine.createSandbox(serviceContext); } diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/engine/KsqlEngineTestUtil.java b/ksqldb-engine/src/test/java/io/confluent/ksql/engine/KsqlEngineTestUtil.java index 402204547ea9..3d87c70d90db 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/engine/KsqlEngineTestUtil.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/engine/KsqlEngineTestUtil.java @@ -18,6 +18,8 @@ import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import io.confluent.ksql.KsqlExecutionContext; import io.confluent.ksql.KsqlExecutionContext.ExecuteResult; +import io.confluent.ksql.config.SessionConfig; +import io.confluent.ksql.format.DefaultFormatInjector; import io.confluent.ksql.internal.KsqlEngineMetrics; import io.confluent.ksql.logging.processing.ProcessingLogContext; import io.confluent.ksql.metastore.MutableMetaStore; @@ -95,7 +97,7 @@ public static TransientQueryMetadata executeQuery( final ParsedStatement stmt = engine.parse(sql).get(0); final PreparedStatement prepared = engine.prepare(stmt); final ConfiguredStatement configured = ConfiguredStatement.of( - prepared, overriddenProperties, ksqlConfig).cast(); + prepared, SessionConfig.of(ksqlConfig, overriddenProperties)).cast(); try { return engine.executeQuery(serviceContext, configured); } catch (final KsqlStatementException e) { @@ -142,11 +144,12 @@ private static ExecuteResult execute( ) { final PreparedStatement prepared = executionContext.prepare(stmt); final ConfiguredStatement configured = ConfiguredStatement.of( - prepared, overriddenProperties, ksqlConfig); + prepared, SessionConfig.of(ksqlConfig, overriddenProperties)); + final ConfiguredStatement withFormats = new DefaultFormatInjector().inject(configured); final ConfiguredStatement withSchema = schemaInjector - .map(injector -> injector.inject(configured)) - .orElse((ConfiguredStatement) configured); + .map(injector -> injector.inject(withFormats)) + .orElse((ConfiguredStatement) withFormats); final ConfiguredStatement reformatted = new SqlFormatInjector(executionContext).inject(withSchema); try { diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/engine/KsqlPlanV1Test.java b/ksqldb-engine/src/test/java/io/confluent/ksql/engine/KsqlPlanV1Test.java index 1c646c5cd4d9..a2534b3956e3 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/engine/KsqlPlanV1Test.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/engine/KsqlPlanV1Test.java @@ -40,8 +40,9 @@ public void shouldImplementEquals() { new EqualsTester() .addEqualityGroup( new KsqlPlanV1("foo", Optional.of(ddlCommand1), Optional.of(queryPlan1)), - new KsqlPlanV1("foo", Optional.of(ddlCommand1), Optional.of(queryPlan1))) - .addEqualityGroup(new KsqlPlanV1("bar", Optional.of(ddlCommand1), Optional.of(queryPlan1))) + new KsqlPlanV1("foo", Optional.of(ddlCommand1), Optional.of(queryPlan1)), + // statementText is not checked as part of equals + new KsqlPlanV1("bar", Optional.of(ddlCommand1), Optional.of(queryPlan1))) .addEqualityGroup(new KsqlPlanV1("foo", Optional.of(ddlCommand2), Optional.of(queryPlan1))) .addEqualityGroup(new KsqlPlanV1("foo", Optional.of(ddlCommand1), Optional.of(queryPlan2))) .testEquals(); diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/engine/rewrite/StatementRewriterTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/engine/rewrite/StatementRewriterTest.java index 2915d47e4473..953832b79954 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/engine/rewrite/StatementRewriterTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/engine/rewrite/StatementRewriterTest.java @@ -59,10 +59,11 @@ import io.confluent.ksql.parser.tree.TableElements; import io.confluent.ksql.parser.tree.WindowExpression; import io.confluent.ksql.parser.tree.WithinExpression; -import io.confluent.ksql.serde.RefinementInfo; import java.util.Optional; import java.util.OptionalInt; import java.util.function.BiFunction; + +import io.confluent.ksql.serde.RefinementInfo; import org.junit.Before; import org.junit.Rule; import org.junit.Test; diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/format/DefaultFormatInjectorTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/format/DefaultFormatInjectorTest.java new file mode 100644 index 000000000000..4c2adc8223f6 --- /dev/null +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/format/DefaultFormatInjectorTest.java @@ -0,0 +1,243 @@ +/* + * Copyright 2020 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.format; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.sameInstance; +import static org.junit.Assert.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.config.SessionConfig; +import io.confluent.ksql.execution.expression.tree.Literal; +import io.confluent.ksql.execution.expression.tree.StringLiteral; +import io.confluent.ksql.name.SourceName; +import io.confluent.ksql.parser.KsqlParser.PreparedStatement; +import io.confluent.ksql.parser.properties.with.CreateSourceProperties; +import io.confluent.ksql.parser.tree.CreateSource; +import io.confluent.ksql.parser.tree.CreateStream; +import io.confluent.ksql.parser.tree.TableElements; +import io.confluent.ksql.statement.ConfiguredStatement; +import io.confluent.ksql.util.KsqlConfig; +import io.confluent.ksql.util.KsqlStatementException; +import java.util.HashMap; +import java.util.Map; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.junit.MockitoJUnitRunner; + +@RunWith(MockitoJUnitRunner.class) +public class DefaultFormatInjectorTest { + + @Mock + private CreateSource createSource; + + private ConfiguredStatement csStatement; + + private DefaultFormatInjector injector; + + @Before + public void setUp() { + when(createSource.getName()).thenReturn(SourceName.of("source")); + when(createSource.getElements()).thenReturn(TableElements.of()); + when(createSource.copyWith(any(), any())).thenAnswer(inv -> setupCopy(inv, createSource, mock(CreateStream.class))); + + injector = new DefaultFormatInjector(); + } + + @Test + public void shouldInjectMissingKeyFormat() { + // Given + givenConfig(ImmutableMap.of( + KsqlConfig.KSQL_DEFAULT_KEY_FORMAT_CONFIG, "KAFKA" + )); + givenSourceProps(ImmutableMap.of( + "VALUE_FORMAT", new StringLiteral("JSON") + )); + + // When + final ConfiguredStatement result = injector.inject(csStatement); + + // Then + assertThat(result.getStatementText(), containsString("KEY_FORMAT='KAFKA'")); + } + + @Test + public void shouldInjectMissingValueFormat() { + // Given + givenConfig(ImmutableMap.of( + KsqlConfig.KSQL_DEFAULT_VALUE_FORMAT_CONFIG, "JSON" + )); + givenSourceProps(ImmutableMap.of( + "KEY_FORMAT", new StringLiteral("KAFKA") + )); + + // When + final ConfiguredStatement result = injector.inject(csStatement); + + // Then + assertThat(result.getStatementText(), containsString("VALUE_FORMAT='JSON'")); + } + + @Test + public void shouldInjectMissingKeyAndValueFormat() { + // Given + givenConfig(ImmutableMap.of( + KsqlConfig.KSQL_DEFAULT_KEY_FORMAT_CONFIG, "KAFKA", + KsqlConfig.KSQL_DEFAULT_VALUE_FORMAT_CONFIG, "JSON" + )); + givenSourceProps(ImmutableMap.of()); + + // When + final ConfiguredStatement result = injector.inject(csStatement); + + // Then + assertThat(result.getStatementText(), containsString("KEY_FORMAT='KAFKA'")); + assertThat(result.getStatementText(), containsString("VALUE_FORMAT='JSON'")); + } + + @Test + public void shouldHandleExplicitKeyAndValueFormat() { + // Given + givenConfig(ImmutableMap.of()); + givenSourceProps(ImmutableMap.of( + "KEY_FORMAT", new StringLiteral("KAFKA"), + "VALUE_FORMAT", new StringLiteral("JSON") + )); + + // When + final ConfiguredStatement result = injector.inject(csStatement); + + // Then + assertThat(result, sameInstance(csStatement)); + } + + @Test + public void shouldHandleExplicitFormat() { + // Given + givenConfig(ImmutableMap.of()); + givenSourceProps(ImmutableMap.of( + "FORMAT", new StringLiteral("KAFKA") + )); + + // When + final ConfiguredStatement result = injector.inject(csStatement); + + // Then + assertThat(result, sameInstance(csStatement)); + } + + @Test + public void shouldDefaultToKafkaIfNoExplicitDefaultKeyFormat() { + // Given + givenConfig(ImmutableMap.of()); + givenSourceProps(ImmutableMap.of( + "VALUE_FORMAT", new StringLiteral("JSON") + )); + + // When + final ConfiguredStatement result = injector.inject(csStatement); + + // Then + assertThat(result.getStatementText(), containsString("KEY_FORMAT='KAFKA'")); + } + + @Test + public void shouldThrowIfMissingDefaultValueFormatConfig() { + // Given + givenConfig(ImmutableMap.of()); + givenSourceProps(ImmutableMap.of( + "KEY_FORMAT", new StringLiteral("KAFKA") + )); + + // Expect / When + final Exception e = assertThrows( + KsqlStatementException.class, + () -> injector.inject(csStatement) + ); + + // Then + assertThat(e.getMessage(), containsString( + "Statement is missing the 'VALUE_FORMAT' property from the WITH clause.")); + assertThat(e.getMessage(), containsString("Either provide one or set a default via the '" + + KsqlConfig.KSQL_DEFAULT_VALUE_FORMAT_CONFIG + "' config.")); + } + + @Test + public void shouldInjectUsingConfigOverrides() { + // Given + givenConfig( + ImmutableMap.of(), + ImmutableMap.of(KsqlConfig.KSQL_DEFAULT_VALUE_FORMAT_CONFIG, "JSON") + ); + givenSourceProps(ImmutableMap.of( + "KEY_FORMAT", new StringLiteral("KAFKA") + )); + + // When + final ConfiguredStatement result = injector.inject(csStatement); + + // Then + assertThat(result.getStatementText(), containsString("VALUE_FORMAT='JSON'")); + } + + private void givenConfig(final Map additionalConfigProps) { + givenConfig(additionalConfigProps, ImmutableMap.of()); + } + + private void givenConfig( + final Map additionalProps, + final Map configOverrides + ) { + final HashMap props = new HashMap<>(); + props.put(KsqlConfig.KSQL_KEY_FORMAT_ENABLED, true); + props.putAll(additionalProps); + + csStatement = ConfiguredStatement.of( + PreparedStatement.of("some sql", createSource), + SessionConfig.of(new KsqlConfig(props), configOverrides)); + } + + private void givenSourceProps( + final Map additionalProps + ) { + final HashMap props = new HashMap<>(); + props.put("KAFKA_TOPIC", new StringLiteral("some_topic")); + props.putAll(additionalProps); + + when(createSource.getProperties()).thenReturn(CreateSourceProperties.from(props)); + } + + private static Object setupCopy( + final InvocationOnMock inv, + final CreateSource source, + final CreateSource mock + ) { + final SourceName name = source.getName(); + when(mock.getName()).thenReturn(name); + when(mock.getElements()).thenReturn(inv.getArgument(0)); + when(mock.accept(any(), any())).thenCallRealMethod(); + when(mock.getProperties()).thenReturn(inv.getArgument(1)); + return mock; + } + +} \ No newline at end of file diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/integration/IntegrationTestHarness.java b/ksqldb-engine/src/test/java/io/confluent/ksql/integration/IntegrationTestHarness.java index 36b81ba132e5..f057e3ed2085 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/integration/IntegrationTestHarness.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/integration/IntegrationTestHarness.java @@ -23,6 +23,7 @@ import static org.hamcrest.Matchers.is; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; import com.google.common.collect.Multimap; import io.confluent.kafka.schemaregistry.ParsedSchema; import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient; @@ -33,6 +34,8 @@ import io.confluent.ksql.logging.processing.ProcessingLogContext; import io.confluent.ksql.schema.ksql.PersistenceSchema; import io.confluent.ksql.schema.ksql.PhysicalSchema; +import io.confluent.ksql.schema.ksql.SchemaConverters; +import io.confluent.ksql.schema.ksql.SimpleColumn; import io.confluent.ksql.serde.Format; import io.confluent.ksql.serde.FormatInfo; import io.confluent.ksql.serde.GenericRowSerDe; @@ -63,6 +66,7 @@ import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.test.TestUtils; @@ -636,47 +640,53 @@ protected void after() { } @SuppressWarnings({"unchecked", "rawtypes"}) + private static Serde getKeySerde(final PhysicalSchema schema) { + final PersistenceSchema keySchema = schema.keySchema(); + + final SimpleColumn onlyColumn = Iterables.getOnlyElement(keySchema.columns()); + + final Class javaType = SchemaConverters.sqlToJavaConverter() + .toJavaType(onlyColumn.type()); + + return (Serde) KafkaSerdeFactory + .getPrimitiveSerde(onlyColumn.type().baseType(), javaType); + } + private static Serializer getKeySerializer(final PhysicalSchema schema) { - return (Serializer) KafkaSerdeFactory - .getPrimitiveSerde(schema.keySchema()) - .serializer(); + return IntegrationTestHarness.getKeySerde(schema).serializer(); } - private Serializer getValueSerializer( + private static Deserializer getKeyDeserializer(final PhysicalSchema schema) { + return IntegrationTestHarness.getKeySerde(schema).deserializer(); + } + + private Serde getValueSerde( final Format format, - final PhysicalSchema schema + final PhysicalSchema schema, + final String loggerNamePrefix ) { return GenericRowSerDe.from( FormatInfo.of(format.name()), schema.valueSchema(), new KsqlConfig(Collections.emptyMap()), serviceContext.get().getSchemaRegistryClientFactory(), - "producer", + loggerNamePrefix, ProcessingLogContext.create() - ).serializer(); + ); } - @SuppressWarnings({"unchecked", "rawtypes"}) - private static Deserializer getKeyDeserializer( + private Serializer getValueSerializer( + final Format format, final PhysicalSchema schema ) { - return (Deserializer) KafkaSerdeFactory - .getPrimitiveSerde(schema.keySchema()) - .deserializer(); + return getValueSerde(format, schema, "producer").serializer(); } private Deserializer getValueDeserializer( final Format format, final PhysicalSchema schema ) { - return GenericRowSerDe.from( - FormatInfo.of(format.name()), - schema.valueSchema(), - new KsqlConfig(Collections.emptyMap()), - serviceContext.get().getSchemaRegistryClientFactory(), - "consumer", - ProcessingLogContext.create() - ).deserializer(); + return getValueSerde(format, schema, "consumer").deserializer(); } public void ensureSchema( diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/physical/PhysicalPlanBuilderTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/physical/PhysicalPlanBuilderTest.java index fd3aa482bd75..fcb260fa333e 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/physical/PhysicalPlanBuilderTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/physical/PhysicalPlanBuilderTest.java @@ -64,34 +64,28 @@ public class PhysicalPlanBuilderTest { private static final String CREATE_STREAM_TEST1 = "CREATE STREAM TEST1 " + "(ROWKEY STRING KEY, COL0 BIGINT, COL1 VARCHAR, COL2 DOUBLE) " - + "WITH (KAFKA_TOPIC = 'test1', VALUE_FORMAT = 'JSON');"; + + "WITH (KAFKA_TOPIC = 'test1', KEY_FORMAT = 'KAFKA', VALUE_FORMAT = 'JSON');"; private static final String CREATE_STREAM_TEST2 = "CREATE STREAM TEST2 " + "(ID2 BIGINT KEY, COL0 VARCHAR, COL1 BIGINT) " - + " WITH (KAFKA_TOPIC = 'test2', VALUE_FORMAT = 'JSON');"; + + " WITH (KAFKA_TOPIC = 'test2', KEY_FORMAT = 'KAFKA', VALUE_FORMAT = 'JSON');"; private static final String CREATE_STREAM_TEST3 = "CREATE STREAM TEST3 " + "(ID3 BIGINT KEY, COL0 BIGINT, COL1 DOUBLE) " - + " WITH (KAFKA_TOPIC = 'test3', VALUE_FORMAT = 'JSON');"; + + " WITH (KAFKA_TOPIC = 'test3', KEY_FORMAT = 'KAFKA', VALUE_FORMAT = 'JSON');"; private static final String CREATE_TABLE_TEST4 = "CREATE TABLE TEST4 " + "(ID BIGINT PRIMARY KEY, COL0 BIGINT, COL1 DOUBLE) " - + " WITH (KAFKA_TOPIC = 'test4', VALUE_FORMAT = 'JSON');"; + + " WITH (KAFKA_TOPIC = 'test4', KEY_FORMAT = 'KAFKA', VALUE_FORMAT = 'JSON');"; private static final String CREATE_TABLE_TEST5 = "CREATE TABLE TEST5 " + "(ID BIGINT PRIMARY KEY, COL0 BIGINT, COL1 DOUBLE) " - + " WITH (KAFKA_TOPIC = 'test5', VALUE_FORMAT = 'JSON');"; - - private static final String CREATE_STREAM_TEST6 = "CREATE STREAM TEST6 " - + "(ID BIGINT KEY, COL0 VARCHAR, COL1 DOUBLE) " - + " WITH (KAFKA_TOPIC = 'test6', VALUE_FORMAT = 'JSON');"; - - private static final String CREATE_STREAM_TEST7 = "CREATE STREAM TEST7 " - + "(ID BIGINT KEY, COL0 VARCHAR, COL1 DOUBLE) " - + " WITH (KAFKA_TOPIC = 'test7', VALUE_FORMAT = 'JSON');"; + + " WITH (KAFKA_TOPIC = 'test5', KEY_FORMAT = 'KAFKA', VALUE_FORMAT = 'JSON');"; private static final String simpleSelectFilter = "SELECT rowkey, col0, col2 FROM test1 WHERE col0 > 100 EMIT CHANGES;"; - private static final KsqlConfig INITIAL_CONFIG = KsqlConfigTestUtil.create("what-eva"); + private static final KsqlConfig INITIAL_CONFIG = KsqlConfigTestUtil.create( + "what-eva", + ImmutableMap.of(KsqlConfig.KSQL_KEY_FORMAT_ENABLED, true)); private final KafkaTopicClient kafkaTopicClient = new FakeKafkaTopicClient(); private KsqlEngine ksqlEngine; @@ -215,7 +209,7 @@ public void shouldCreateExecutionPlanForInsert() { public void shouldCreatePlanForInsertIntoStreamFromStream() { // Given: final String cs = "CREATE STREAM test1 (ROWKEY STRING KEY, col0 INT) " - + "WITH (KAFKA_TOPIC='test1', VALUE_FORMAT='JSON');"; + + "WITH (KAFKA_TOPIC='test1', KEY_FORMAT = 'KAFKA', VALUE_FORMAT='JSON');"; final String csas = "CREATE STREAM s0 AS SELECT * FROM test1;"; final String insertInto = "INSERT INTO s0 SELECT * FROM test1;"; givenKafkaTopicsExist("test1"); diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/ConfiguredKsqlPlanTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/ConfiguredKsqlPlanTest.java index 78492f475e65..1ebc38c8336d 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/ConfiguredKsqlPlanTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/ConfiguredKsqlPlanTest.java @@ -16,10 +16,8 @@ package io.confluent.ksql.planner.plan; import com.google.common.testing.EqualsTester; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.KsqlPlan; -import io.confluent.ksql.util.KsqlConfig; -import java.util.Collections; -import java.util.Map; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; @@ -27,27 +25,25 @@ @RunWith(MockitoJUnitRunner.class) public class ConfiguredKsqlPlanTest { - private static final Map OVERRIDES = Collections.singletonMap("foo", "bar"); - private static final Map OVERRIDES2 = Collections.singletonMap("biz", "baz"); @Mock private KsqlPlan plan; @Mock private KsqlPlan plan2; @Mock - private KsqlConfig ksqlConfig; + private SessionConfig config; @Mock - private KsqlConfig ksqlConfig2; + private SessionConfig config2; + @SuppressWarnings("UnstableApiUsage") @Test public void testEquality() { new EqualsTester() .addEqualityGroup( - ConfiguredKsqlPlan.of(plan, OVERRIDES, ksqlConfig), - ConfiguredKsqlPlan.of(plan, OVERRIDES, ksqlConfig)) - .addEqualityGroup(ConfiguredKsqlPlan.of(plan2, OVERRIDES, ksqlConfig)) - .addEqualityGroup(ConfiguredKsqlPlan.of(plan, OVERRIDES, ksqlConfig2)) - .addEqualityGroup(ConfiguredKsqlPlan.of(plan, OVERRIDES2, ksqlConfig)) + ConfiguredKsqlPlan.of(plan, config), + ConfiguredKsqlPlan.of(plan, config) + ) + .addEqualityGroup(ConfiguredKsqlPlan.of(plan2, config2)) .testEquals(); } } \ No newline at end of file diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/JoinNodeTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/JoinNodeTest.java index a8d33580a02c..76e0db1691de 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/JoinNodeTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/JoinNodeTest.java @@ -19,6 +19,7 @@ import static io.confluent.ksql.planner.plan.JoinNode.JoinType.INNER; import static io.confluent.ksql.planner.plan.JoinNode.JoinType.LEFT; import static io.confluent.ksql.planner.plan.JoinNode.JoinType.OUTER; +import static io.confluent.ksql.planner.plan.PlanTestUtil.SOURCE_NODE; import static io.confluent.ksql.planner.plan.PlanTestUtil.SOURCE_NODE_FORCE_CHANGELOG; import static io.confluent.ksql.planner.plan.PlanTestUtil.getNodeByName; import static java.util.Optional.empty; diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/SuppressNodeTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/SuppressNodeTest.java index fdbe2e3d962d..64ffd906372b 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/SuppressNodeTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/planner/plan/SuppressNodeTest.java @@ -20,6 +20,7 @@ import static org.hamcrest.Matchers.containsString; import static org.junit.Assert.assertThrows; import static org.mockito.ArgumentMatchers.any; + import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/query/QueryExecutorTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/query/QueryExecutorTest.java index 47eea5706874..575f4908a0e9 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/query/QueryExecutorTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/query/QueryExecutorTest.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import io.confluent.ksql.GenericRow; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.errors.ProductionExceptionHandlerUtil; import io.confluent.ksql.execution.context.QueryContext.Stacker; import io.confluent.ksql.execution.ddl.commands.KsqlTopic; @@ -165,6 +166,8 @@ public class QueryExecutorTest { private KTableHolder tableHolder; @Mock private KStreamHolder streamHolder; + @Mock + private SessionConfig config; @Captor private ArgumentCaptor> propertyCaptor; @@ -198,9 +201,10 @@ public void setup() { when(topoDesc.subtopologies()).thenReturn(ImmutableSet.of()); when(serviceContext.getTopicClient()).thenReturn(topicClient); when(streamsBuilder.build(any())).thenReturn(topology); + when(config.getConfig(true)).thenReturn(ksqlConfig); + when(config.getOverrides()).thenReturn(OVERRIDES); queryBuilder = new QueryExecutor( - ksqlConfig, - OVERRIDES, + config, processingLogContext, serviceContext, functionRegistry, @@ -212,8 +216,7 @@ public void setup() { serviceContext, ksMaterializationFactory, ksqlMaterializationFactory - ) - ); + )); } @Test diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/schema/ksql/inference/DefaultSchemaInjectorFunctionalTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/schema/ksql/inference/DefaultSchemaInjectorFunctionalTest.java index 349ba0f38c90..80fda1ab0210 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/schema/ksql/inference/DefaultSchemaInjectorFunctionalTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/schema/ksql/inference/DefaultSchemaInjectorFunctionalTest.java @@ -26,6 +26,7 @@ import io.confluent.kafka.schemaregistry.avro.AvroSchema; import io.confluent.kafka.schemaregistry.client.SchemaMetadata; import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.metastore.MetaStore; import io.confluent.ksql.parser.KsqlParser.PreparedStatement; import io.confluent.ksql.parser.tree.CreateSource; @@ -514,8 +515,9 @@ private void shouldInferSchema( .buildSingleAst(stmtNoSchema, metaStore); // When: + final KsqlConfig ksqlConfig = new KsqlConfig(ImmutableMap.of()); final ConfiguredStatement inferred = schemaInjector.inject( - ConfiguredStatement.of(prepared, ImmutableMap.of(), new KsqlConfig(ImmutableMap.of()))); + ConfiguredStatement.of(prepared, SessionConfig.of(ksqlConfig, ImmutableMap.of()))); // Then: final Statement withSchema = KsqlParserTestUtil diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/schema/ksql/inference/DefaultSchemaInjectorTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/schema/ksql/inference/DefaultSchemaInjectorTest.java index c8bd8dc82248..d5f0f252a63e 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/schema/ksql/inference/DefaultSchemaInjectorTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/schema/ksql/inference/DefaultSchemaInjectorTest.java @@ -28,6 +28,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.execution.expression.tree.Literal; import io.confluent.ksql.execution.expression.tree.StringLiteral; import io.confluent.ksql.execution.expression.tree.Type; @@ -139,8 +140,10 @@ public void setUp() { when(ct.copyWith(any(), any())).thenAnswer(inv -> setupCopy(inv, ct, mock(CreateTable.class))); final KsqlConfig config = new KsqlConfig(ImmutableMap.of()); - csStatement = ConfiguredStatement.of(PreparedStatement.of(SQL_TEXT, cs), ImmutableMap.of(), config); - ctStatement = ConfiguredStatement.of(PreparedStatement.of(SQL_TEXT, ct), ImmutableMap.of(), config); + csStatement = ConfiguredStatement.of(PreparedStatement.of(SQL_TEXT, cs), + SessionConfig.of(config, ImmutableMap.of())); + ctStatement = ConfiguredStatement.of(PreparedStatement.of(SQL_TEXT, ct), + SessionConfig.of(config, ImmutableMap.of())); when(schemaSupplier.getValueSchema(eq(KAFKA_TOPIC), any())) .thenReturn(SchemaResult.success(schemaAndId(SUPPORTED_SCHEMAS, SCHEMA_ID))); @@ -154,10 +157,10 @@ public void setUp() { @Test public void shouldReturnStatementUnchangedIfNotCreateStatement() { // Given: - final ConfiguredStatement prepared = ConfiguredStatement.of( - PreparedStatement.of("sql", statement), - ImmutableMap.of(), - new KsqlConfig(ImmutableMap.of())); + final ConfiguredStatement prepared = ConfiguredStatement + .of(PreparedStatement.of("sql", statement), + SessionConfig.of(new KsqlConfig(ImmutableMap.of()), ImmutableMap.of()) + ); // When: final ConfiguredStatement result = injector.inject(prepared); diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/schema/ksql/inference/SchemaRegisterInjectorTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/schema/ksql/inference/SchemaRegisterInjectorTest.java index 0037b06283de..5cf99088316a 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/schema/ksql/inference/SchemaRegisterInjectorTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/schema/ksql/inference/SchemaRegisterInjectorTest.java @@ -35,6 +35,7 @@ import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import io.confluent.ksql.KsqlExecutionContext; import io.confluent.ksql.KsqlExecutionContext.ExecuteResult; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.execution.ddl.commands.KsqlTopic; import io.confluent.ksql.function.InternalFunctionRegistry; import io.confluent.ksql.metastore.MetaStoreImpl; @@ -65,7 +66,6 @@ import io.confluent.ksql.util.KsqlStatementException; import io.confluent.ksql.util.PersistentQueryMetadata; import java.io.IOException; -import java.util.HashMap; import java.util.Optional; import org.apache.avro.Schema; import org.junit.Before; @@ -163,7 +163,7 @@ public void setUp() { public void shouldNotRegisterSchemaIfSchemaRegistryIsDisabled() { // Given: config = new KsqlConfig(ImmutableMap.of()); - givenStatement("CREATE STREAM sink (f1 VARCHAR) WITH(kafka_topic='expectedName', value_format='AVRO', partitions=1);"); + givenStatement("CREATE STREAM sink (f1 VARCHAR) WITH(kafka_topic='expectedName', key_format='KAFKA', value_format='AVRO', partitions=1);"); // When: final KsqlSchemaRegistryNotConfiguredException e = assertThrows(KsqlSchemaRegistryNotConfiguredException.class, () -> injector.inject(statement)); @@ -175,7 +175,7 @@ public void shouldNotRegisterSchemaIfSchemaRegistryIsDisabled() { @Test public void shouldNotRegisterSchemaForSchemaRegistryDisabledFormatCreateSource() { // Given: - givenStatement("CREATE STREAM sink (f1 VARCHAR) WITH(kafka_topic='expectedName', value_format='DELIMITED', partitions=1);"); + givenStatement("CREATE STREAM sink (f1 VARCHAR) WITH(kafka_topic='expectedName', key_format='KAFKA', value_format='DELIMITED', partitions=1);"); // When: injector.inject(statement); @@ -188,7 +188,7 @@ public void shouldNotRegisterSchemaForSchemaRegistryDisabledFormatCreateSource() public void shouldRegisterSchemaForSchemaRegistryEnabledFormatCreateSourceIfSubjectDoesntExist() throws Exception { // Given: - givenStatement("CREATE STREAM sink (f1 VARCHAR) WITH (kafka_topic='expectedName', value_format='AVRO', partitions=1);"); + givenStatement("CREATE STREAM sink (f1 VARCHAR) WITH (kafka_topic='expectedName', key_format='KAFKA', value_format='AVRO', partitions=1);"); // When: injector.inject(statement); @@ -202,7 +202,7 @@ public void shouldRegisterSchemaForSchemaRegistryEnabledFormatCreateSourceIfSubj public void shouldNotReplaceExistingSchemaForSchemaRegistryEnabledFormatCreateSource() throws Exception { // Given: - givenStatement("CREATE STREAM sink (f1 VARCHAR) WITH (kafka_topic='expectedName', value_format='AVRO', partitions=1);"); + givenStatement("CREATE STREAM sink (f1 VARCHAR) WITH (kafka_topic='expectedName', key_format='KAFKA', value_format='AVRO', partitions=1);"); when(schemaRegistryClient.getAllSubjects()).thenReturn(ImmutableSet.of("expectedName-value")); // When: @@ -293,6 +293,7 @@ public void shouldSupportPrimitiveValueSchemasInCreateStmts() throws Exception { givenStatement("CREATE STREAM source (f1 VARCHAR) " + "WITH (" + " kafka_topic='expectedName', " + + " key_format='KAFKA', " + " value_format='AVRO', " + " partitions=1, " + " wrap_single_value='false'" @@ -329,8 +330,8 @@ private void givenStatement(final String sql) { parser.prepare(parser.parse(sql).get(0), metaStore); statement = ConfiguredStatement.of( preparedStatement, - new HashMap<>(), - config); + SessionConfig.of(config, ImmutableMap.of()) + ); when(executionSandbox.execute(any(), eq(statement))) .thenReturn(ExecuteResult.of(queryMetadata)); } diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/serde/SerdeOptionsFactoryTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/serde/SerdeOptionsFactoryTest.java index bf2a71a92cbb..630f8a4515e6 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/serde/SerdeOptionsFactoryTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/serde/SerdeOptionsFactoryTest.java @@ -82,7 +82,8 @@ public void shouldGetSingleValueWrappingFromPropertiesBeforeConfigForCreateState ); // Then: - assertThat(result.valueWrapping(), is(Optional.of(SerdeOption.UNWRAP_SINGLE_VALUES))); + assertThat(result.findAny(SerdeOptions.VALUE_WRAPPING_OPTIONS), + is(Optional.of(SerdeOption.UNWRAP_SINGLE_VALUES))); } @Test @@ -102,7 +103,8 @@ public void shouldGetSingleValueWrappingFromConfigForCreateStatement() { ); // Then: - assertThat(result.valueWrapping(), is(Optional.of(SerdeOption.UNWRAP_SINGLE_VALUES))); + assertThat(result.findAny(SerdeOptions.VALUE_WRAPPING_OPTIONS), + is(Optional.of(SerdeOption.UNWRAP_SINGLE_VALUES))); } @Test @@ -117,7 +119,7 @@ public void shouldDefaultToNoSingleValueWrappingIfNoExplicitAndNoConfigDefault() ); // Then: - assertThat(result.valueWrapping(), is(Optional.empty())); + assertThat(result.findAny(SerdeOptions.VALUE_WRAPPING_OPTIONS), is(Optional.empty())); } @Test @@ -137,7 +139,7 @@ public void shouldNotGetSingleValueWrappingFromDefaultConfigIfFormatDoesNotSuppo ); // Then: - assertThat(result.valueWrapping(), is(Optional.empty())); + assertThat(result.findAny(SerdeOptions.VALUE_WRAPPING_OPTIONS), is(Optional.empty())); } @Test @@ -157,7 +159,7 @@ public void shouldNotGetSingleValueWrappingFromConfigForMultiFieldsForCreateStat ); // Then: - assertThat(result.valueWrapping(), is(Optional.empty())); + assertThat(result.findAny(SerdeOptions.VALUE_WRAPPING_OPTIONS), is(Optional.empty())); } @Test @@ -214,7 +216,8 @@ public void shouldGetSingleValueWrappingFromPropertiesBeforeDefaultsForCreateAsS ); // Then: - assertThat(result.valueWrapping(), is(Optional.of(SerdeOption.WRAP_SINGLE_VALUES))); + assertThat(result.findAny(SerdeOptions.VALUE_WRAPPING_OPTIONS), + is(Optional.of(SerdeOption.WRAP_SINGLE_VALUES))); } @Test @@ -234,7 +237,8 @@ public void shouldGetSingleValueWrappingFromDefaultsForCreateAsStatement() { ); // Then: - assertThat(result.valueWrapping(), is(Optional.of(SerdeOption.UNWRAP_SINGLE_VALUES))); + assertThat(result.findAny(SerdeOptions.VALUE_WRAPPING_OPTIONS), + is(Optional.of(SerdeOption.UNWRAP_SINGLE_VALUES))); } @Test @@ -254,7 +258,7 @@ public void shouldNotGetSingleValueWrappingFromDefaultForMultiFieldsForCreateAsS ); // Then: - assertThat(result.valueWrapping(), is(Optional.empty())); + assertThat(result.findAny(SerdeOptions.VALUE_WRAPPING_OPTIONS), is(Optional.empty())); } @Test @@ -306,7 +310,8 @@ public void shouldSetUnwrappedKeysIfKeyFormatSupportsBothWrappingAndUnwrapping() ); // Then: - assertThat(result.keyWrapping(), is(Optional.of(SerdeOption.UNWRAP_SINGLE_KEYS))); + assertThat(result.findAny(SerdeOptions.KEY_WRAPPING_OPTIONS), + is(Optional.of(SerdeOption.UNWRAP_SINGLE_KEYS))); } @Test @@ -321,7 +326,7 @@ public void shouldNotSetUnwrappedKeysIfKeyFormatsSupportsOnlyWrapping() { ); // Then: - assertThat(result.keyWrapping(), is(Optional.empty())); + assertThat(result.findAny(SerdeOptions.KEY_WRAPPING_OPTIONS), is(Optional.empty())); } @Test @@ -336,6 +341,6 @@ public void shouldNotSetUnwrappedKeysIfKeyFormatsSupportsOnlyUnwrapping() { ); // Then: - assertThat(result.keyWrapping(), is(Optional.empty())); + assertThat(result.findAny(SerdeOptions.KEY_WRAPPING_OPTIONS), is(Optional.empty())); } } \ No newline at end of file diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/statement/ConfiguredStatementTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/statement/ConfiguredStatementTest.java deleted file mode 100644 index c69fe900a0de..000000000000 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/statement/ConfiguredStatementTest.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * 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.statement; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.instanceOf; -import static org.hamcrest.Matchers.is; - -import com.google.common.collect.ImmutableMap; -import io.confluent.ksql.parser.KsqlParser.PreparedStatement; -import io.confluent.ksql.parser.tree.Statement; -import io.confluent.ksql.util.KsqlConfig; -import java.util.HashMap; -import java.util.Map; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; - -@RunWith(MockitoJUnitRunner.class) -public class ConfiguredStatementTest { - - private static final KsqlConfig CONFIG = new KsqlConfig(ImmutableMap.of()); - @Mock - private PreparedStatement prepared; - - @Test - public void shouldTakeDefensiveCopyOfProperties() { - // Given: - final Map props = new HashMap<>(); - props.put("this", "that"); - - final ConfiguredStatement statement = ConfiguredStatement - .of(prepared, props, CONFIG); - - // When: - props.put("other", "thing"); - - // Then: - assertThat(statement.getConfigOverrides(), is(ImmutableMap.of("this", "that"))); - } - - @Test - public void shouldReturnImmutableProperties() { - // Given: - final ConfiguredStatement statement = ConfiguredStatement - .of(prepared, new HashMap<>(), CONFIG); - - // Then: - assertThat(statement.getConfigOverrides(), is(instanceOf(ImmutableMap.class))); - } -} \ No newline at end of file diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/topic/TopicCreateInjectorTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/topic/TopicCreateInjectorTest.java index 6ff61ed280f5..a17988e86e38 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/topic/TopicCreateInjectorTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/topic/TopicCreateInjectorTest.java @@ -28,6 +28,7 @@ import static org.mockito.hamcrest.MockitoHamcrest.argThat; import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.execution.ddl.commands.KsqlTopic; import io.confluent.ksql.function.InternalFunctionRegistry; import io.confluent.ksql.metastore.MetaStoreImpl; @@ -469,8 +470,8 @@ private ConfiguredStatement givenStatement(final String sql) { final ConfiguredStatement configuredStatement = ConfiguredStatement.of( preparedStatement, - overrides, - config); + SessionConfig.of(config, overrides) + ); statement = configuredStatement; return configuredStatement; } diff --git a/ksqldb-engine/src/test/java/io/confluent/ksql/topic/TopicDeleteInjectorTest.java b/ksqldb-engine/src/test/java/io/confluent/ksql/topic/TopicDeleteInjectorTest.java index 5282d7704b0f..2aa334a27729 100644 --- a/ksqldb-engine/src/test/java/io/confluent/ksql/topic/TopicDeleteInjectorTest.java +++ b/ksqldb-engine/src/test/java/io/confluent/ksql/topic/TopicDeleteInjectorTest.java @@ -32,6 +32,7 @@ import com.google.common.collect.ImmutableMap; import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.execution.ddl.commands.KsqlTopic; import io.confluent.ksql.metastore.MutableMetaStore; import io.confluent.ksql.metastore.model.DataSource; @@ -284,10 +285,7 @@ private static ConfiguredStatement givenStatement( final String text, final T statement ) { - return ConfiguredStatement.of( - PreparedStatement.of(text, statement), - ImmutableMap.of(), - new KsqlConfig(ImmutableMap.of()) - ); + return ConfiguredStatement.of(PreparedStatement.of(text, statement), SessionConfig + .of(new KsqlConfig(ImmutableMap.of()), ImmutableMap.of())); } } \ No newline at end of file diff --git a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/plan/TableSuppress.java b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/plan/TableSuppress.java index d1d626ad592b..296025bbc1a5 100644 --- a/ksqldb-execution/src/main/java/io/confluent/ksql/execution/plan/TableSuppress.java +++ b/ksqldb-execution/src/main/java/io/confluent/ksql/execution/plan/TableSuppress.java @@ -18,6 +18,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.errorprone.annotations.Immutable; import io.confluent.ksql.serde.RefinementInfo; + import java.util.Collections; import java.util.List; import java.util.Objects; diff --git a/ksqldb-execution/src/test/java/io/confluent/ksql/execution/codegen/SqlToJavaVisitorTest.java b/ksqldb-execution/src/test/java/io/confluent/ksql/execution/codegen/SqlToJavaVisitorTest.java index e517a7202961..5c4ce1005aff 100644 --- a/ksqldb-execution/src/test/java/io/confluent/ksql/execution/codegen/SqlToJavaVisitorTest.java +++ b/ksqldb-execution/src/test/java/io/confluent/ksql/execution/codegen/SqlToJavaVisitorTest.java @@ -77,11 +77,12 @@ import io.confluent.ksql.schema.ksql.types.SqlDecimal; import io.confluent.ksql.schema.ksql.types.SqlPrimitiveType; import io.confluent.ksql.schema.ksql.types.SqlTypes; -import io.confluent.ksql.util.KsqlConfig; import java.math.BigDecimal; import java.util.Collections; import java.util.Optional; import java.util.concurrent.atomic.AtomicInteger; + +import io.confluent.ksql.util.KsqlConfig; import org.junit.Before; import org.junit.Rule; import org.junit.Test; diff --git a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/driver/AssertExecutor.java b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/driver/AssertExecutor.java index 09a86883b5e7..37bc274b67e5 100644 --- a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/driver/AssertExecutor.java +++ b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/driver/AssertExecutor.java @@ -39,12 +39,14 @@ import io.confluent.ksql.schema.ksql.SystemColumns; import io.confluent.ksql.schema.ksql.types.SqlTypes; import io.confluent.ksql.schema.utils.FormatOptions; +import io.confluent.ksql.serde.FormatInfo; import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.KsqlException; import io.confluent.ksql.util.TabularRow; import java.util.Iterator; import java.util.List; import java.util.Objects; +import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.Collectors; import org.apache.kafka.connect.data.Struct; @@ -60,41 +62,50 @@ public final class AssertExecutor { static final List MUST_MATCH = ImmutableList.builder() .add(new SourceProperty( DataSource::getSchema, - cs -> cs.getElements().toLogicalSchema(), + (cs, cfg) -> cs.getElements().toLogicalSchema(), "schema" )).add(new SourceProperty( DataSource::getDataSourceType, - cs -> cs instanceof CreateTable ? DataSourceType.KTABLE : DataSourceType.KSTREAM, + (cs, cfg) -> cs instanceof CreateTable ? DataSourceType.KTABLE : DataSourceType.KSTREAM, "type" )).add(new SourceProperty( DataSource::getKafkaTopicName, - cs -> cs.getProperties().getKafkaTopic(), + (cs, cfg) -> cs.getProperties().getKafkaTopic(), "kafka topic", CommonCreateConfigs.KAFKA_TOPIC_NAME_PROPERTY + )).add(new SourceProperty( + ds -> ds.getKsqlTopic().getKeyFormat().getFormatInfo().getFormat(), + (cs, cfg) -> cs.getProperties().getKeyFormat().map(FormatInfo::getFormat) + .orElse(cfg.getString(KsqlConfig.KSQL_DEFAULT_KEY_FORMAT_CONFIG)), + "key format", + CommonCreateConfigs.KEY_FORMAT_PROPERTY, + CommonCreateConfigs.FORMAT_PROPERTY )).add(new SourceProperty( ds -> ds.getKsqlTopic().getValueFormat().getFormatInfo().getFormat(), - cs -> cs.getProperties().getValueFormat().name(), + (cs, cfg) -> cs.getProperties().getValueFormat().map(FormatInfo::getFormat) + .orElse(cfg.getString(KsqlConfig.KSQL_DEFAULT_VALUE_FORMAT_CONFIG)), "value format", - CommonCreateConfigs.VALUE_FORMAT_PROPERTY + CommonCreateConfigs.VALUE_FORMAT_PROPERTY, + CommonCreateConfigs.FORMAT_PROPERTY )).add(new SourceProperty( ds -> ds.getKsqlTopic().getValueFormat().getFormatInfo().getProperties(), - cs -> cs.getProperties().getFormatInfo().getProperties(), + (cs, cfg) -> cs.getProperties().getValueFormatProperties(), "delimiter", CommonCreateConfigs.VALUE_AVRO_SCHEMA_FULL_NAME, CommonCreateConfigs.VALUE_DELIMITER_PROPERTY )).add(new SourceProperty( DataSource::getSerdeOptions, - cs -> cs.getProperties().getSerdeOptions(), + (cs, cfg) -> cs.getProperties().getSerdeOptions(), "serde options", CommonCreateConfigs.WRAP_SINGLE_VALUE )).add(new SourceProperty( ds -> ds.getTimestampColumn().map(TimestampColumn::getColumn), - cs -> cs.getProperties().getTimestampColumnName(), + (cs, cfg) -> cs.getProperties().getTimestampColumnName(), "timestamp column", CommonCreateConfigs.TIMESTAMP_NAME_PROPERTY )).add(new SourceProperty( ds -> ds.getTimestampColumn().flatMap(TimestampColumn::getFormat), - cs -> cs.getProperties().getTimestampFormat(), + (cs, cfg) -> cs.getProperties().getTimestampFormat(), "timestamp format", CommonCreateConfigs.TIMESTAMP_FORMAT_PROPERTY )).build(); @@ -226,39 +237,42 @@ private static TabularRow fromGenericRow( public static void assertStream( final KsqlExecutionContext engine, + final KsqlConfig config, final AssertStream assertStatement ) { - assertSourceMatch(engine, assertStatement.getStatement()); + assertSourceMatch(engine, config, assertStatement.getStatement()); } public static void assertTable( final KsqlExecutionContext engine, + final KsqlConfig config, final AssertTable assertStatement ) { - assertSourceMatch(engine, assertStatement.getStatement()); + assertSourceMatch(engine, config, assertStatement.getStatement()); } private static void assertSourceMatch( final KsqlExecutionContext engine, + final KsqlConfig config, final CreateSource statement ) { final SourceName source = statement.getName(); final DataSource dataSource = engine.getMetaStore().getSource(source); - MUST_MATCH.forEach(prop -> prop.compare(dataSource, statement)); + MUST_MATCH.forEach(prop -> prop.compare(dataSource, statement, config)); } @VisibleForTesting static final class SourceProperty { final Function extractSource; - final Function extractStatement; + final BiFunction extractStatement; final String propertyName; final String[] withClauseName; private SourceProperty( final Function extractSource, - final Function extractStatement, + final BiFunction extractStatement, final String propertyName, final String... withClauseName ) { @@ -268,8 +282,12 @@ private SourceProperty( this.withClauseName = withClauseName; } - private void compare(final DataSource dataSource, final CreateSource statement) { - final Object expected = extractStatement.apply(statement); + private void compare( + final DataSource dataSource, + final CreateSource statement, + final KsqlConfig config + ) { + final Object expected = extractStatement.apply(statement, config); final Object actual = extractSource.apply(dataSource); if (!actual.equals(expected)) { throw new KsqlException( diff --git a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestCaseBuilderUtil.java b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestCaseBuilderUtil.java index 582d8fa559ef..abe720240d35 100644 --- a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestCaseBuilderUtil.java +++ b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestCaseBuilderUtil.java @@ -19,7 +19,9 @@ import com.google.common.collect.Streams; import io.confluent.kafka.schemaregistry.ParsedSchema; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.execution.ddl.commands.KsqlTopic; +import io.confluent.ksql.format.DefaultFormatInjector; import io.confluent.ksql.function.FunctionRegistry; import io.confluent.ksql.metastore.MetaStore; import io.confluent.ksql.metastore.MetaStoreImpl; @@ -29,6 +31,7 @@ import io.confluent.ksql.parser.KsqlParser.ParsedStatement; import io.confluent.ksql.parser.KsqlParser.PreparedStatement; import io.confluent.ksql.parser.SqlBaseParser; +import io.confluent.ksql.parser.properties.with.SourcePropertiesUtil; import io.confluent.ksql.parser.tree.CreateSource; import io.confluent.ksql.parser.tree.RegisterType; import io.confluent.ksql.schema.ksql.LogicalSchema; @@ -37,11 +40,13 @@ import io.confluent.ksql.serde.SerdeOptions; import io.confluent.ksql.serde.SerdeOptionsFactory; import io.confluent.ksql.serde.ValueFormat; +import io.confluent.ksql.statement.ConfiguredStatement; import io.confluent.ksql.topic.TopicFactory; import io.confluent.ksql.util.KsqlConfig; import java.nio.file.Path; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -136,7 +141,7 @@ private static Topic createTopicFromStatement( ) { final KsqlParser parser = new DefaultKsqlParser(); - final Function, Topic> extractTopic = (PreparedStatement stmt) -> { + final Function, Topic> extractTopic = (ConfiguredStatement stmt) -> { final CreateSource statement = (CreateSource) stmt.getStatement(); final KsqlTopic ksqlTopic = TopicFactory.create(statement.getProperties()); @@ -150,8 +155,8 @@ private static Topic createTopicFromStatement( try { serdeOptions = SerdeOptionsFactory.buildForCreateStatement( logicalSchema, - FormatFactory.KAFKA, - statement.getProperties().getValueFormat(), + FormatFactory.of(SourcePropertiesUtil.getKeyFormat(statement.getProperties())), + FormatFactory.of(SourcePropertiesUtil.getValueFormat(statement.getProperties())), statement.getProperties().getSerdeOptions(), ksqlConfig ); @@ -198,7 +203,10 @@ private static Topic createTopicFromStatement( if (isCsOrCT(stmt)) { final PreparedStatement prepare = parser.prepare(stmt, metaStore); - topics.add(extractTopic.apply(prepare)); + final ConfiguredStatement configured = + ConfiguredStatement.of(prepare, SessionConfig.of(ksqlConfig, Collections.emptyMap())); + final ConfiguredStatement withFormats = new DefaultFormatInjector().inject(configured); + topics.add(extractTopic.apply(withFormats)); } } diff --git a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestExecutor.java b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestExecutor.java index 6c82766178fd..d3df9b4b6fd0 100644 --- a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestExecutor.java +++ b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestExecutor.java @@ -87,7 +87,6 @@ public class TestExecutor implements Closeable { .put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, 0) .put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") .put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0) - .put(StreamsConfig.MAX_TASK_IDLE_MS_CONFIG, 0L) .put(KsqlConfig.KSQL_SERVICE_ID_CONFIG, "some.ksql.service.id") .build(); diff --git a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestExecutorUtil.java b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestExecutorUtil.java index 25419dedaa0d..9e0c25645c3c 100644 --- a/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestExecutorUtil.java +++ b/ksqldb-functional-tests/src/main/java/io/confluent/ksql/test/tools/TestExecutorUtil.java @@ -30,11 +30,13 @@ import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import io.confluent.ksql.KsqlExecutionContext; import io.confluent.ksql.KsqlExecutionContext.ExecuteResult; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.engine.KsqlPlan; import io.confluent.ksql.engine.SqlFormatInjector; import io.confluent.ksql.engine.StubInsertValuesExecutor; import io.confluent.ksql.execution.json.PlanJsonMapper; +import io.confluent.ksql.format.DefaultFormatInjector; import io.confluent.ksql.function.FunctionRegistry; import io.confluent.ksql.metastore.MetaStore; import io.confluent.ksql.metastore.model.DataSource; @@ -158,7 +160,7 @@ static Iterable planTestCase( && testCase.getExpectedTopology().get().getPlan().isPresent()) { return testCase.getExpectedTopology().get().getPlan().get() .stream() - .map(p -> ConfiguredKsqlPlan.of(p, testCase.properties(), ksqlConfig)) + .map(p -> ConfiguredKsqlPlan.of(p, SessionConfig.of(ksqlConfig, testCase.properties()))) .collect(Collectors.toList()); } return PlannedStatementIterator.of(engine, testCase, ksqlConfig, srClient, stubKafkaService); @@ -426,7 +428,9 @@ public Iterator iterator() { private Optional planStatement(final ParsedStatement stmt) { final PreparedStatement prepared = executionContext.prepare(stmt); final ConfiguredStatement configured = ConfiguredStatement.of( - prepared, sessionProperties.getMutableScopedProperties(), ksqlConfig); + prepared, + SessionConfig.of(ksqlConfig, sessionProperties.getMutableScopedProperties()) + ); if (prepared.getStatement() instanceof InsertValues) { StubInsertValuesExecutor.of(stubKafkaService, executionContext).execute( @@ -438,21 +442,19 @@ private Optional planStatement(final ParsedStatement stmt) { return Optional.empty(); } + final ConfiguredStatement withFormats = + new DefaultFormatInjector().inject(configured); final ConfiguredStatement withSchema = schemaInjector - .map(injector -> injector.inject(configured)) - .orElse((ConfiguredStatement) configured); + .map(injector -> injector.inject(withFormats)) + .orElse((ConfiguredStatement) withFormats); final ConfiguredStatement reformatted = new SqlFormatInjector(executionContext).inject(withSchema); final KsqlPlan plan = executionContext .plan(executionContext.getServiceContext(), reformatted); return Optional.of( - ConfiguredKsqlPlan.of( - rewritePlan(plan), - reformatted.getConfigOverrides(), - reformatted.getConfig() - ) + ConfiguredKsqlPlan.of(rewritePlan(plan), reformatted.getSessionConfig()) ); } diff --git a/ksqldb-functional-tests/src/test/java/io/confluent/ksql/test/driver/AssertExecutorMetaTest.java b/ksqldb-functional-tests/src/test/java/io/confluent/ksql/test/driver/AssertExecutorMetaTest.java index bfcb7776f5f2..de6fdd13e4ef 100644 --- a/ksqldb-functional-tests/src/test/java/io/confluent/ksql/test/driver/AssertExecutorMetaTest.java +++ b/ksqldb-functional-tests/src/test/java/io/confluent/ksql/test/driver/AssertExecutorMetaTest.java @@ -31,7 +31,7 @@ public class AssertExecutorMetaTest { public void shouldCoverAllWithClauses() { // Given: final ConfigDef def = new ConfigDef(); - CommonCreateConfigs.addToConfigDef(def, true, true); + CommonCreateConfigs.addToConfigDef(def, true); final Set allValidProps = def.names(); // When: diff --git a/ksqldb-functional-tests/src/test/java/io/confluent/ksql/test/driver/KsqlTesterTest.java b/ksqldb-functional-tests/src/test/java/io/confluent/ksql/test/driver/KsqlTesterTest.java index 580a20808a34..f06a6f5a2952 100644 --- a/ksqldb-functional-tests/src/test/java/io/confluent/ksql/test/driver/KsqlTesterTest.java +++ b/ksqldb-functional-tests/src/test/java/io/confluent/ksql/test/driver/KsqlTesterTest.java @@ -21,9 +21,11 @@ import io.confluent.ksql.GenericRow; import io.confluent.ksql.KsqlExecutionContext.ExecuteResult; import io.confluent.ksql.ServiceInfo; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.engine.generic.GenericRecordFactory; import io.confluent.ksql.engine.generic.KsqlGenericRecord; +import io.confluent.ksql.format.DefaultFormatInjector; import io.confluent.ksql.logging.processing.NoopProcessingLogContext; import io.confluent.ksql.metastore.MetaStoreImpl; import io.confluent.ksql.metastore.model.DataSource; @@ -43,11 +45,13 @@ import io.confluent.ksql.query.id.SequentialQueryIdGenerator; import io.confluent.ksql.schema.ksql.PersistenceSchema; import io.confluent.ksql.schema.utils.FormatOptions; +import io.confluent.ksql.serde.GenericKeySerDe; import io.confluent.ksql.serde.GenericRowSerDe; import io.confluent.ksql.services.FakeKafkaTopicClient; import io.confluent.ksql.services.ServiceContext; import io.confluent.ksql.services.TestServiceContext; import io.confluent.ksql.statement.ConfiguredStatement; +import io.confluent.ksql.statement.Injector; import io.confluent.ksql.test.KsqlTestException; import io.confluent.ksql.test.driver.TestDriverPipeline.TopicInfo; import io.confluent.ksql.test.parser.SqlTestLoader; @@ -108,6 +112,7 @@ public class KsqlTesterTest { private ServiceContext serviceContext; private KsqlEngine engine; private KsqlConfig config; + private Injector formatInjector; private TestDriverPipeline driverPipeline; private FakeKafkaTopicClient topicClient; @@ -145,6 +150,7 @@ public void setUp() { this.topicClient = new FakeKafkaTopicClient(); this.serviceContext = TestServiceContext.create(topicClient, () -> srClient); this.config = new KsqlConfig(BASE_CONFIG); + this.formatInjector = new DefaultFormatInjector(); final MetaStoreImpl metaStore = new MetaStoreImpl(TestFunctionRegistry.INSTANCE.get()); this.engine = new KsqlEngine( @@ -193,8 +199,8 @@ public void test() { @SuppressWarnings("unchecked") private void execute(final ParsedStatement parsedStatement) { final PreparedStatement engineStatement = engine.prepare(parsedStatement); - final ConfiguredStatement configured = ConfiguredStatement.of( - engineStatement, overrides, config); + final ConfiguredStatement configured = ConfiguredStatement + .of(engineStatement, SessionConfig.of(config, overrides)); createTopics(engineStatement); @@ -209,9 +215,10 @@ private void execute(final ParsedStatement parsedStatement) { return; } + final ConfiguredStatement injected = formatInjector.inject(configured); final ExecuteResult result = engine.execute( serviceContext, - configured); + injected); // is DDL statement if (!result.getQuery().isPresent()) { @@ -326,9 +333,9 @@ private void doAssert(final AssertStatement statement) { } else if (statement instanceof AssertTombstone) { AssertExecutor.assertTombstone(engine, config, (AssertTombstone) statement, driverPipeline); } else if (statement instanceof AssertStream) { - AssertExecutor.assertStream(engine, ((AssertStream) statement)); + AssertExecutor.assertStream(engine, config, ((AssertStream) statement)); } else if (statement instanceof AssertTable) { - AssertExecutor.assertTable(engine, ((AssertTable) statement)); + AssertExecutor.assertTable(engine, config, ((AssertTable) statement)); } } @@ -338,14 +345,14 @@ private Serde keySerde(final DataSource sinkSource) { sinkSource.getSerdeOptions().keyFeatures() ); - return sinkSource.getKsqlTopic().getKeyFormat() - .getFormat() - .getSerde( - schema, - sinkSource.getKsqlTopic().getKeyFormat().getFormatInfo().getProperties(), - config, - serviceContext.getSchemaRegistryClientFactory() - ); + return new GenericKeySerDe().create( + sinkSource.getKsqlTopic().getKeyFormat().getFormatInfo(), + schema, + config, + serviceContext.getSchemaRegistryClientFactory(), + "", + NoopProcessingLogContext.INSTANCE + ); } private Serde valueSerde(final DataSource sinkSource) { diff --git a/ksqldb-functional-tests/src/test/java/io/confluent/ksql/test/planned/PlannedTestUtils.java b/ksqldb-functional-tests/src/test/java/io/confluent/ksql/test/planned/PlannedTestUtils.java index 13738e50b060..9c12a5b21703 100644 --- a/ksqldb-functional-tests/src/test/java/io/confluent/ksql/test/planned/PlannedTestUtils.java +++ b/ksqldb-functional-tests/src/test/java/io/confluent/ksql/test/planned/PlannedTestUtils.java @@ -50,7 +50,7 @@ public static boolean isNotExcluded(final TestCase testCase) { // Place temporary logic here to exclude test cases based on feature flags, etc. return !(boolean) testCase .properties() - .getOrDefault(KsqlConfig.KSQL_CREATE_OR_REPLACE_ENABLED, false); + .getOrDefault(KsqlConfig.KSQL_KEY_FORMAT_ENABLED, false); } public static boolean isSamePlan( diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/formats_-_explicit_format/6.1.0_1600714522363/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/formats_-_explicit_format/6.1.0_1600714522363/plan.json new file mode 100644 index 000000000000..04097bfad1e1 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/formats_-_explicit_format/6.1.0_1600714522363/plan.json @@ -0,0 +1,143 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (K STRING KEY, FOO STRING) WITH (FORMAT='KAFKA', KAFKA_TOPIC='input');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`K` STRING KEY, `FOO` STRING", + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "KAFKA" + } + }, + "orReplace" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `FOO` STRING", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "KAFKA" + } + }, + "orReplace" : false + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "KAFKA" + } + }, + "sourceSchema" : "`K` STRING KEY, `FOO` STRING" + }, + "keyColumnNames" : [ "K" ], + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "KAFKA" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "metric.reporters" : "", + "ksql.transient.prefix" : "transient_", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.streams.max.task.idle.ms" : "0", + "ksql.query.error.max.queue.size" : "10", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.key.format.enabled" : "false", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.create.or.replace.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.suppress.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.schema.registry.url" : "", + "ksql.properties.overrides.denylist" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.enable.metastore.backup" : "false", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.metrics.tags.custom" : "", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/formats_-_explicit_format/6.1.0_1600714522363/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/formats_-_explicit_format/6.1.0_1600714522363/spec.json new file mode 100644 index 000000000000..03f3fc34ebed --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/formats_-_explicit_format/6.1.0_1600714522363/spec.json @@ -0,0 +1,71 @@ +{ + "version" : "6.1.0", + "timestamp" : 1600714522363, + "path" : "query-validation-tests/formats.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `FOO` STRING", + "serdeOptions" : [ ] + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `FOO` STRING", + "serdeOptions" : [ ] + } + }, + "testCase" : { + "name" : "explicit format", + "topics" : [ { + "name" : "input", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (K STRING KEY, foo STRING) WITH (kafka_topic='input', format='KAFKA');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "properties" : { + "ksql.persistence.default.format.value" : "JSON" + }, + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `FOO` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "serdeOptions" : [ ] + }, { + "name" : "INPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `FOO` STRING", + "keyFormat" : { + "format" : "KAFKA" + }, + "serdeOptions" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "input", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "KAFKA" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "KAFKA" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/formats_-_explicit_format/6.1.0_1600714522363/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/formats_-_explicit_format/6.1.0_1600714522363/topology new file mode 100644 index 000000000000..a70a4e91e301 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/formats_-_explicit_format/6.1.0_1600714522363/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/formats_-_explicit_key_and_value_format/6.1.0_1600714522329/plan.json b/ksqldb-functional-tests/src/test/resources/historical_plans/formats_-_explicit_key_and_value_format/6.1.0_1600714522329/plan.json new file mode 100644 index 000000000000..0955cb700aa1 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/formats_-_explicit_key_and_value_format/6.1.0_1600714522329/plan.json @@ -0,0 +1,143 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (K STRING KEY, FOO INTEGER) WITH (KAFKA_TOPIC='input_topic', KEY_FORMAT='KAFKA', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`K` STRING KEY, `FOO` INTEGER", + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "orReplace" : false + } + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT *\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`K` STRING KEY, `FOO` INTEGER", + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "orReplace" : false + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "input_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "sourceSchema" : "`K` STRING KEY, `FOO` INTEGER" + }, + "keyColumnNames" : [ "K" ], + "selectExpressions" : [ "FOO AS FOO" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA" + }, + "valueFormat" : { + "format" : "JSON" + } + }, + "topicName" : "OUTPUT" + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "metric.reporters" : "", + "ksql.transient.prefix" : "transient_", + "ksql.query.status.running.threshold.seconds" : "300", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.persistence.default.format.key" : "KAFKA", + "ksql.streams.max.task.idle.ms" : "0", + "ksql.query.error.max.queue.size" : "10", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.key.format.enabled" : "false", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.query.pull.max.qps" : "2147483647", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.query.pull.metrics.enabled" : "false", + "ksql.create.or.replace.enabled" : "true", + "ksql.metrics.extension" : null, + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.cast.strings.preserve.nulls" : "true", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.pull.queries.enable" : "true", + "ksql.suppress.enabled" : "false", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.query.persistent.active.limit" : "2147483647", + "ksql.persistence.wrap.single.values" : null, + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.query.retry.backoff.initial.ms" : "15000", + "ksql.schema.registry.url" : "", + "ksql.properties.overrides.denylist" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.enable.metastore.backup" : "false", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.streams.topology.optimization" : "all", + "ksql.query.retry.backoff.max.ms" : "900000", + "ksql.streams.num.stream.threads" : "4", + "ksql.timestamp.throw.on.invalid" : "false", + "ksql.metrics.tags.custom" : "", + "ksql.persistence.default.format.value" : null, + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.metastore.backup.location" : "", + "ksql.error.classifier.regex" : "", + "ksql.suppress.buffer.size.bytes" : "-1" + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/formats_-_explicit_key_and_value_format/6.1.0_1600714522329/spec.json b/ksqldb-functional-tests/src/test/resources/historical_plans/formats_-_explicit_key_and_value_format/6.1.0_1600714522329/spec.json new file mode 100644 index 000000000000..5f623fe85afc --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/formats_-_explicit_key_and_value_format/6.1.0_1600714522329/spec.json @@ -0,0 +1,89 @@ +{ + "version" : "6.1.0", + "timestamp" : 1600714522329, + "path" : "query-validation-tests/formats.json", + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : { + "schema" : "`K` STRING KEY, `FOO` INTEGER", + "serdeOptions" : [ ] + }, + "CSAS_OUTPUT_0.OUTPUT" : { + "schema" : "`K` STRING KEY, `FOO` INTEGER", + "serdeOptions" : [ ] + } + }, + "testCase" : { + "name" : "explicit key and value format", + "inputs" : [ { + "topic" : "input_topic", + "key" : null, + "value" : { + "FOO" : 10 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : null, + "value" : { + "FOO" : 10 + } + } ], + "topics" : [ { + "name" : "OUTPUT", + "replicas" : 1, + "numPartitions" : 4 + }, { + "name" : "input_topic", + "replicas" : 1, + "numPartitions" : 4 + } ], + "statements" : [ "CREATE STREAM INPUT (K STRING KEY, foo INT) WITH (kafka_topic='input_topic', value_format='JSON', key_format='KAFKA');", "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" ], + "properties" : { + "ksql.persistence.default.format.value" : "AVRO" + }, + "post" : { + "sources" : [ { + "name" : "OUTPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `FOO` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "serdeOptions" : [ ] + }, { + "name" : "INPUT", + "type" : "STREAM", + "schema" : "`K` STRING KEY, `FOO` INTEGER", + "keyFormat" : { + "format" : "KAFKA" + }, + "serdeOptions" : [ ] + } ], + "topics" : { + "topics" : [ { + "name" : "input_topic", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + }, { + "name" : "OUTPUT", + "keyFormat" : { + "formatInfo" : { + "format" : "KAFKA" + } + }, + "valueFormat" : { + "format" : "JSON" + }, + "partitions" : 4 + } ] + } + } + } +} \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/historical_plans/formats_-_explicit_key_and_value_format/6.1.0_1600714522329/topology b/ksqldb-functional-tests/src/test/resources/historical_plans/formats_-_explicit_key_and_value_format/6.1.0_1600714522329/topology new file mode 100644 index 000000000000..12f8f6574002 --- /dev/null +++ b/ksqldb-functional-tests/src/test/resources/historical_plans/formats_-_explicit_key_and_value_format/6.1.0_1600714522329/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [input_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksqldb-functional-tests/src/test/resources/query-validation-tests/formats.json b/ksqldb-functional-tests/src/test/resources/query-validation-tests/formats.json index 057b31700e5f..ff934cb27567 100644 --- a/ksqldb-functional-tests/src/test/resources/query-validation-tests/formats.json +++ b/ksqldb-functional-tests/src/test/resources/query-validation-tests/formats.json @@ -1,22 +1,327 @@ { "tests": [ { - "name": "invalid format name", + "name": "invalid value format - create source", "statements": [ "CREATE STREAM TEST WITH (kafka_topic='test_topic', value_format='FOO');" ], - "topics": [ - { - "name": "test_topic", - "format": "JSON" + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Unknown format: FOO" + } + }, + { + "name": "missing value format", + "statements": [ + "CREATE STREAM TEST WITH (kafka_topic='test_topic');" + ], + "expectedException": { + "type": "io.confluent.ksql.parser.exception.ParseFailedException", + "message": "Failed to prepare statement: Missing required property \"VALUE_FORMAT\" which has no default value." + } + }, + { + "name": "explicit key and optional value format", + "statements": [ + "CREATE STREAM INPUT (K STRING KEY, foo INT) WITH (kafka_topic='input_topic', key_format='KAFKA');", + "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" + ], + "properties": { + "ksql.persistence.default.format.value": "JSON", + "ksql.key.format.enabled": true + }, + "inputs": [ + {"topic": "input_topic", "value": {"FOO": 10}} + ], + "outputs": [ + {"topic": "OUTPUT", "value": {"FOO": 10}} + ], + "post": { + "topics": { + "topics": [ + { + "name" : "input_topic", + "keyFormat" : {"formatInfo" : {"format" : "KAFKA"}}, + "valueFormat" : {"format" : "JSON"}, + "partitions": 4 + }, + { + "name" : "OUTPUT", + "keyFormat" : {"formatInfo" : {"format" : "KAFKA"}}, + "valueFormat" : {"format" : "JSON"}, + "partitions": 4 + } + ] + } + } + }, + { + "name": "optional key and value format", + "statements": [ + "CREATE STREAM INPUT (K STRING KEY, foo INT) WITH (kafka_topic='input_topic');", + "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" + ], + "properties": { + "ksql.persistence.default.format.value": "JSON", + "ksql.key.format.enabled": true + }, + "inputs": [ + {"topic": "input_topic", "value": {"FOO": 10}} + ], + "outputs": [ + {"topic": "OUTPUT", "value": {"FOO": 10}} + ], + "post": { + "topics": { + "topics": [ + { + "name" : "input_topic", + "keyFormat" : {"formatInfo" : {"format" : "KAFKA"}}, + "valueFormat" : {"format" : "JSON"}, + "partitions": 4 + }, + { + "name" : "OUTPUT", + "keyFormat" : {"formatInfo" : {"format" : "KAFKA"}}, + "valueFormat" : {"format" : "JSON"}, + "partitions": 4 + } + ] + } + } + }, + { + "name": "unsupported key format - create source", + "statements": [ + "CREATE STREAM TEST (foo VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON', key_format='AVRO');" + ], + "properties": { + "ksql.key.format.enabled": true + }, + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "The key format 'AVRO' is not currently supported." + } + }, + { + "name": "invalid key format - create source", + "statements": [ + "CREATE STREAM TEST (foo VARCHAR) WITH (kafka_topic='test_topic', value_format='JSON', key_format='FOO');" + ], + "properties": { + "ksql.key.format.enabled": true + }, + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Unknown format: FOO" + } + }, + { + "name": "explicit key and value format", + "statements": [ + "CREATE STREAM INPUT (K STRING KEY, foo INT) WITH (kafka_topic='input_topic', value_format='JSON', key_format='KAFKA');", + "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" + ], + "properties": { + "ksql.persistence.default.format.value": "AVRO" + }, + "inputs": [ + {"topic": "input_topic", "value": {"FOO": 10}} + ], + "outputs": [ + {"topic": "OUTPUT", "value": {"FOO": 10}} + ], + "post": { + "topics": { + "topics": [ + { + "name" : "input_topic", + "keyFormat" : {"formatInfo" : {"format" : "KAFKA"}}, + "valueFormat" : {"format" : "JSON"}, + "partitions": 4 + }, + { + "name" : "OUTPUT", + "keyFormat" : {"formatInfo" : {"format" : "KAFKA"}}, + "valueFormat" : {"format" : "JSON"}, + "partitions": 4 + } + ] + } + } + }, + { + "name": "optional key and explicit value format", + "statements": [ + "CREATE STREAM INPUT (K STRING KEY, foo INT) WITH (kafka_topic='input_topic', value_format='JSON');", + "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" + ], + "properties": { + "ksql.key.format.enabled": true, + "ksql.persistence.default.format.value": "KAFKA" + }, + "inputs": [ + {"topic": "input_topic", "value": {"FOO": 10}} + ], + "outputs": [ + {"topic": "OUTPUT", "value": {"FOO": 10}} + ], + "post": { + "topics": { + "topics": [ + { + "name" : "input_topic", + "keyFormat" : {"formatInfo" : {"format" : "KAFKA"}}, + "valueFormat" : {"format" : "JSON"}, + "partitions": 4 + }, + { + "name" : "OUTPUT", + "keyFormat" : {"formatInfo" : {"format" : "KAFKA"}}, + "valueFormat" : {"format" : "JSON"}, + "partitions": 4 + } + ] + } + } + }, + { + "name": "explicit format", + "statements": [ + "CREATE STREAM INPUT (K STRING KEY, foo STRING) WITH (kafka_topic='input', format='KAFKA');", + "CREATE STREAM OUTPUT AS SELECT * FROM INPUT;" + ], + "properties": { + "ksql.persistence.default.format.value": "JSON" + }, + "input": [{"topic": "input", "key": "foo", "value": "bar"}], + "output": [{"topic": "OUTPUT", "key": "foo", "value": "bar"}], + "post": { + "topics": { + "topics": [ + { + "name" : "input", + "keyFormat" : {"formatInfo" : {"format" : "KAFKA"}}, + "valueFormat" : {"format" : "KAFKA"}, + "partitions": 4 + }, + { + "name" : "OUTPUT", + "keyFormat" : {"formatInfo" : {"format" : "KAFKA"}}, + "valueFormat" : {"format" : "KAFKA"}, + "partitions": 4 + } + ] } + } + }, + { + "name": "unsupported format - create source", + "statements": [ + "CREATE STREAM TEST (foo VARCHAR) WITH (kafka_topic='test_topic', format='DELIMITED');" ], + "properties": { + "ksql.key.format.enabled": true + }, + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "The key format 'DELIMITED' is not currently supported." + } + }, + { + "name": "invalid format - create source", + "statements": [ + "CREATE STREAM TEST WITH (kafka_topic='test_topic', format='FOO');" + ], + "properties": { + "ksql.key.format.enabled": true + }, "expectedException": { "type": "io.confluent.ksql.util.KsqlStatementException", "message": "Unknown format: FOO" + } + }, + { + "name": "explicit key format and format", + "statements": [ + "CREATE STREAM TEST WITH (kafka_topic='test_topic', key_format='KAFKA', format='KAFKA');" + ], + "properties": { + "ksql.persistence.default.format.value": "JSON", + "ksql.key.format.enabled": true + }, + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Cannot supply both 'KEY_FORMAT' and 'FORMAT' properties, as 'FORMAT' sets both key and value formats. Either use just 'FORMAT', or use 'KEY_FORMAT' and 'VALUE_FORMAT'." + } + }, + { + "name": "explicit value format and format", + "statements": [ + "CREATE STREAM TEST WITH (kafka_topic='test_topic', value_format='JSON', format='KAFKA');" + ], + "properties": { + "ksql.key.format.enabled": true }, - "inputs": [], - "outputs": [] + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Cannot supply both 'VALUE_FORMAT' and 'FORMAT' properties, as 'FORMAT' sets both key and value formats. Either use just 'FORMAT', or use 'KEY_FORMAT' and 'VALUE_FORMAT'." + } + }, + { + "name": "invalid value format - create as select", + "statements": [ + "CREATE STREAM TEST (foo VARCHAR) WITH (kafka_topic='test_topic', value_format='KAFKA');", + "CREATE STREAM COPY WITH (value_format='FOO') AS SELECT * FROM TEST EMIT CHANGES;" + ], + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Unknown format: FOO" + } + }, + { + "name": "invalid key format - create as select", + "statements": [ + "CREATE STREAM TEST (foo VARCHAR) WITH (kafka_topic='test_topic', value_format='KAFKA');", + "CREATE STREAM COPY WITH (key_format='FOO') AS SELECT * FROM TEST EMIT CHANGES;" + ], + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Unknown format: FOO" + } + }, + { + "name": "invalid format - create as select", + "statements": [ + "CREATE STREAM TEST (foo VARCHAR) WITH (kafka_topic='test_topic', value_format='KAFKA');", + "CREATE STREAM COPY WITH (format='FOO') AS SELECT * FROM TEST EMIT CHANGES;" + ], + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "Unknown format: FOO" + } + }, + { + "name": "unsupported key format - create as select", + "statements": [ + "CREATE STREAM TEST (foo VARCHAR) WITH (kafka_topic='test_topic', value_format='KAFKA');", + "CREATE STREAM COPY WITH (key_format='AVRO') AS SELECT * FROM TEST EMIT CHANGES;" + ], + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "The key format 'AVRO' is not currently supported" + } + }, + { + "name": "unsupported format - create as select", + "statements": [ + "CREATE STREAM TEST (foo VARCHAR) WITH (kafka_topic='test_topic', value_format='KAFKA');", + "CREATE STREAM COPY WITH (format='AVRO') AS SELECT * FROM TEST EMIT CHANGES;" + ], + "expectedException": { + "type": "io.confluent.ksql.util.KsqlStatementException", + "message": "The key format 'AVRO' is not currently supported" + } } ] } \ No newline at end of file diff --git a/ksqldb-functional-tests/src/test/resources/rest-query-validation-tests/insert-values.json b/ksqldb-functional-tests/src/test/resources/rest-query-validation-tests/insert-values.json index 52d1b773b191..2f2ab5ab2c90 100644 --- a/ksqldb-functional-tests/src/test/resources/rest-query-validation-tests/insert-values.json +++ b/ksqldb-functional-tests/src/test/resources/rest-query-validation-tests/insert-values.json @@ -15,7 +15,7 @@ {"topic": "test_topic", "timestamp": 1234, "key": "key", "value": {"ID": 10}} ], "responses": [ - {"admin": {"@type": "currentStatus", "statementText": "{STATEMENT}"}} + {"admin": {"@type": "currentStatus"}} ] }, { @@ -222,7 +222,7 @@ {"topic": "test_topic", "timestamp": 1234, "key": "key", "value": {"id!": 10}} ], "responses": [ - {"admin": {"@type": "currentStatus", "statementText": "{STATEMENT}"}} + {"admin": {"@type": "currentStatus"}} ] }, { @@ -381,7 +381,7 @@ {"topic": "test_topic", "timestamp": 1234, "key": "key", "value": 10} ], "responses": [ - {"admin": {"@type": "currentStatus", "statementText": "{STATEMENT}"}} + {"admin": {"@type": "currentStatus"}} ] } ] diff --git a/ksqldb-functional-tests/src/test/resources/sql-tests/test.sql b/ksqldb-functional-tests/src/test/resources/sql-tests/test.sql index 201e4fbe3106..e84cc5e3f884 100644 --- a/ksqldb-functional-tests/src/test/resources/sql-tests/test.sql +++ b/ksqldb-functional-tests/src/test/resources/sql-tests/test.sql @@ -165,7 +165,20 @@ CREATE STREAM bar AS SELECT * FROM foo; ASSERT STREAM bar (id INT KEY, col1 INT) WITH (kafka_topic='BAZ', value_format='JSON'); ---------------------------------------------------------------------------------------------------- ---@test: assert stream with wrong topic format should fail +--@test: assert stream with wrong key format should fail + +--@expected.error: io.confluent.ksql.util.KsqlException +--@expected.message: Expected key format does not match actual for source BAR +---------------------------------------------------------------------------------------------------- +SET 'ksql.key.format.enabled' = 'true'; + +CREATE STREAM foo (id INT KEY, col1 INT) WITH (kafka_topic='foo', key_format='KAFKA', value_format='JSON'); +CREATE STREAM bar AS SELECT * FROM foo; + +ASSERT STREAM bar (id INT KEY, col1 INT) WITH (kafka_topic='BAR', key_format='AVRO', value_format='JSON'); + +---------------------------------------------------------------------------------------------------- +--@test: assert stream with wrong value format should fail --@expected.error: io.confluent.ksql.util.KsqlException --@expected.message: Expected value format does not match actual for source BAR @@ -175,6 +188,39 @@ CREATE STREAM bar AS SELECT * FROM foo; ASSERT STREAM bar (id INT KEY, col1 INT) WITH (kafka_topic='BAR', value_format='AVRO'); +---------------------------------------------------------------------------------------------------- +--@test: assert stream with wrong format should fail + +--@expected.error: io.confluent.ksql.util.KsqlException +--@expected.message: Expected value format does not match actual for source BAR +---------------------------------------------------------------------------------------------------- +SET 'ksql.key.format.enabled' = 'true'; + +CREATE STREAM foo (id INT KEY, col1 INT) WITH (kafka_topic='foo', format='KAFKA'); +CREATE STREAM bar AS SELECT * FROM foo; + +ASSERT STREAM bar (id INT KEY, col1 INT) WITH (kafka_topic='BAR', key_format='KAFKA', value_format='JSON'); + +---------------------------------------------------------------------------------------------------- +--@test: assert stream with explicit format +---------------------------------------------------------------------------------------------------- +SET 'ksql.key.format.enabled' = 'true'; + +CREATE STREAM foo (id INT KEY, col1 INT) WITH (kafka_topic='foo', format='KAFKA'); +CREATE STREAM bar AS SELECT * FROM foo; + +ASSERT STREAM bar (id INT KEY, col1 INT) WITH (kafka_topic='BAR', key_format='KAFKA', value_format='KAFKA'); + +---------------------------------------------------------------------------------------------------- +--@test: assert stream with explicit expected format +---------------------------------------------------------------------------------------------------- +SET 'ksql.key.format.enabled' = 'true'; + +CREATE STREAM foo (id INT KEY, col1 INT) WITH (kafka_topic='foo', key_format='KAFKA', value_format='KAFKA'); +CREATE STREAM bar AS SELECT * FROM foo; + +ASSERT STREAM bar (id INT KEY, col1 INT) WITH (kafka_topic='BAR', format='KAFKA'); + ---------------------------------------------------------------------------------------------------- --@test: assert stream with wrong timestamp column diff --git a/ksqldb-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceAsProperties.java b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceAsProperties.java index 95bb63f277db..4fe541978ece 100644 --- a/ksqldb-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceAsProperties.java +++ b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceAsProperties.java @@ -25,8 +25,6 @@ import io.confluent.ksql.parser.ColumnReferenceParser; import io.confluent.ksql.properties.with.CommonCreateConfigs; import io.confluent.ksql.properties.with.CreateAsConfigs; -import io.confluent.ksql.serde.Format; -import io.confluent.ksql.serde.FormatFactory; import io.confluent.ksql.serde.FormatInfo; import io.confluent.ksql.serde.SerdeOption; import io.confluent.ksql.serde.SerdeOptions; @@ -66,13 +64,10 @@ public static CreateSourceAsProperties from(final Map literals) private CreateSourceAsProperties(final Map originals) { this.props = new PropertiesConfig(CreateAsConfigs.CONFIG_METADATA, originals); + CommonCreateConfigs.validateKeyValueFormats(props.originals()); props.validateDateTimeFormat(CommonCreateConfigs.TIMESTAMP_FORMAT_PROPERTY); } - public Optional getValueFormat() { - return getFormatInfo().map(FormatFactory::of); - } - public Optional getKafkaTopic() { return Optional.ofNullable(props.getString(CommonCreateConfigs.KAFKA_TOPIC_NAME_PROPERTY)); } @@ -105,12 +100,20 @@ public SerdeOptions getSerdeOptions() { return SerdeOptions.of(builder.build()); } - public Optional getFormatInfo() { - return Optional.ofNullable(props.getString(CommonCreateConfigs.VALUE_FORMAT_PROPERTY)) - .map(format -> FormatInfo.of(format, getFormatProperties())); + public Optional getKeyFormat() { + final String keyFormat = getFormatName() + .orElse(props.getString(CommonCreateConfigs.KEY_FORMAT_PROPERTY)); + return Optional.ofNullable(keyFormat).map(format -> FormatInfo.of(format, ImmutableMap.of())); + } + + public Optional getValueFormat() { + final String valueFormat = getFormatName() + .orElse(props.getString(CommonCreateConfigs.VALUE_FORMAT_PROPERTY)); + return Optional.ofNullable(valueFormat) + .map(format -> FormatInfo.of(format, getValueFormatProperties())); } - public Map getFormatProperties() { + public Map getValueFormatProperties() { final ImmutableMap.Builder builder = ImmutableMap.builder(); final String schemaName = props.getString(CommonCreateConfigs.VALUE_AVRO_SCHEMA_FULL_NAME); @@ -160,4 +163,8 @@ public boolean equals(final Object o) { public int hashCode() { return Objects.hash(props); } + + private Optional getFormatName() { + return Optional.ofNullable(props.getString(CommonCreateConfigs.FORMAT_PROPERTY)); + } } diff --git a/ksqldb-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceProperties.java b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceProperties.java index c2dfbfc03b71..ee5f620c7724 100644 --- a/ksqldb-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceProperties.java +++ b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/properties/with/CreateSourceProperties.java @@ -21,14 +21,13 @@ import com.google.errorprone.annotations.Immutable; import io.confluent.ksql.execution.expression.tree.IntegerLiteral; import io.confluent.ksql.execution.expression.tree.Literal; +import io.confluent.ksql.execution.expression.tree.StringLiteral; import io.confluent.ksql.model.WindowType; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.parser.ColumnReferenceParser; 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.Format; -import io.confluent.ksql.serde.FormatFactory; import io.confluent.ksql.serde.FormatInfo; import io.confluent.ksql.serde.SerdeOption; import io.confluent.ksql.serde.SerdeOptions; @@ -74,14 +73,11 @@ public static CreateSourceProperties from(final Map literals) { this.props = new PropertiesConfig(CreateConfigs.CONFIG_METADATA, originals); this.durationParser = Objects.requireNonNull(durationParser, "durationParser"); + CommonCreateConfigs.validateKeyValueFormats(props.originals()); props.validateDateTimeFormat(CommonCreateConfigs.TIMESTAMP_FORMAT_PROPERTY); validateWindowInfo(); } - public Format getValueFormat() { - return FormatFactory.of(getFormatInfo()); - } - public String getKafkaTopic() { return props.getString(CommonCreateConfigs.KAFKA_TOPIC_NAME_PROPERTY); } @@ -131,13 +127,20 @@ public Optional getSchemaId() { return Optional.ofNullable(props.getInt(CreateConfigs.SCHEMA_ID)); } - public FormatInfo getFormatInfo() { - return FormatInfo.of( - props.getString(CommonCreateConfigs.VALUE_FORMAT_PROPERTY), - getFormatProperties()); + public Optional getKeyFormat() { + final String keyFormat = getFormatName() + .orElse(props.getString(CommonCreateConfigs.KEY_FORMAT_PROPERTY)); + return Optional.ofNullable(keyFormat).map(format -> FormatInfo.of(format, ImmutableMap.of())); + } + + public Optional getValueFormat() { + final String valueFormat = getFormatName() + .orElse(props.getString(CommonCreateConfigs.VALUE_FORMAT_PROPERTY)); + return Optional.ofNullable(valueFormat) + .map(format -> FormatInfo.of(format, getValueFormatProperties())); } - private Map getFormatProperties() { + public Map getValueFormatProperties() { final ImmutableMap.Builder builder = ImmutableMap.builder(); final String schemaName = props.getString(CommonCreateConfigs.VALUE_AVRO_SCHEMA_FULL_NAME); @@ -182,6 +185,14 @@ public CreateSourceProperties withPartitionsAndReplicas( return new CreateSourceProperties(originals, durationParser); } + public CreateSourceProperties withFormats(final String keyFormat, final String valueFormat) { + final Map originals = props.copyOfOriginalLiterals(); + originals.put(CommonCreateConfigs.KEY_FORMAT_PROPERTY, new StringLiteral(keyFormat)); + originals.put(CommonCreateConfigs.VALUE_FORMAT_PROPERTY, new StringLiteral(valueFormat)); + + return new CreateSourceProperties(originals, durationParser); + } + public Map copyOfOriginalLiterals() { return props.copyOfOriginalLiterals(); } @@ -225,4 +236,9 @@ private void validateWindowInfo() { + "should not be set for SESSION windows."); } } + + private Optional getFormatName() { + return Optional.ofNullable(props.getString(CommonCreateConfigs.FORMAT_PROPERTY)); + } + } diff --git a/ksqldb-parser/src/main/java/io/confluent/ksql/parser/properties/with/PropertiesConfig.java b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/properties/with/PropertiesConfig.java index 3b65a340c945..a72119885d0c 100644 --- a/ksqldb-parser/src/main/java/io/confluent/ksql/parser/properties/with/PropertiesConfig.java +++ b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/properties/with/PropertiesConfig.java @@ -80,7 +80,7 @@ void validateDateTimeFormat(final String configName) { try { DateTimeFormatter.ofPattern(pattern); } catch (final Exception e) { - throw new KsqlException("Invalid datatime format for" + throw new KsqlException("Invalid datetime format for" + " config:" + configName + ", reason:" + e.getMessage(), e); } diff --git a/ksqldb-parser/src/main/java/io/confluent/ksql/parser/properties/with/SourcePropertiesUtil.java b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/properties/with/SourcePropertiesUtil.java new file mode 100644 index 000000000000..8882f2665138 --- /dev/null +++ b/ksqldb-parser/src/main/java/io/confluent/ksql/parser/properties/with/SourcePropertiesUtil.java @@ -0,0 +1,45 @@ +/* + * Copyright 2020 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.parser.properties.with; + +import io.confluent.ksql.serde.FormatInfo; + +public final class SourcePropertiesUtil { + + private SourcePropertiesUtil() { + } + + /** + * The {@code DefaultFormatInjector} ensures that CREATE STREAM and CREATE TABLE statements + * always contain key and format values by the time they reach the engine for processing. + * As a result, downstream code can assume these formats are present. + */ + public static FormatInfo getKeyFormat(final CreateSourceProperties properties) { + return properties.getKeyFormat() + .orElseThrow(() -> new IllegalStateException("Key format not present")); + } + + /** + * The {@code DefaultFormatInjector} ensures that CREATE STREAM and CREATE TABLE statements + * always contain key and format values by the time they reach the engine for processing. + * As a result, downstream code can assume these formats are present. + */ + public static FormatInfo getValueFormat(final CreateSourceProperties properties) { + return properties.getValueFormat() + .orElseThrow(() -> new IllegalStateException("Value format not present")); + } + +} diff --git a/ksqldb-parser/src/test/java/io/confluent/ksql/parser/KsqlParserTest.java b/ksqldb-parser/src/test/java/io/confluent/ksql/parser/KsqlParserTest.java index 6c06cca58f1f..16eb6443ff99 100644 --- a/ksqldb-parser/src/test/java/io/confluent/ksql/parser/KsqlParserTest.java +++ b/ksqldb-parser/src/test/java/io/confluent/ksql/parser/KsqlParserTest.java @@ -547,7 +547,7 @@ public void testCreateStream() { assertThat(Iterables.get(result.getElements(), 0).getName(), equalTo(ColumnName.of("ORDERTIME"))); assertThat(Iterables.get(result.getElements(), 6).getType().getSqlType().baseType(), equalTo(SqlBaseType.STRUCT)); assertThat(result.getProperties().getKafkaTopic(), equalTo("orders_topic")); - assertThat(result.getProperties().getValueFormat(), equalTo(FormatFactory.AVRO)); + assertThat(result.getProperties().getValueFormat().map(FormatInfo::getFormat), equalTo(Optional.of("AVRO"))); } @Test @@ -562,7 +562,7 @@ public void testCreateTable() { assertThat(Iterables.size(result.getElements()), equalTo(4)); assertThat(Iterables.get(result.getElements(), 0).getName(), equalTo(ColumnName.of("USERTIME"))); assertThat(result.getProperties().getKafkaTopic(), equalTo("foo")); - assertThat(result.getProperties().getValueFormat(), equalTo(FormatFactory.JSON)); + assertThat(result.getProperties().getValueFormat().map(FormatInfo::getFormat), equalTo(Optional.of("JSON"))); } @Test diff --git a/ksqldb-parser/src/test/java/io/confluent/ksql/parser/properties/with/CreateSourceAsPropertiesTest.java b/ksqldb-parser/src/test/java/io/confluent/ksql/parser/properties/with/CreateSourceAsPropertiesTest.java index fed4599e3475..9ace1eaf2832 100644 --- a/ksqldb-parser/src/test/java/io/confluent/ksql/parser/properties/with/CreateSourceAsPropertiesTest.java +++ b/ksqldb-parser/src/test/java/io/confluent/ksql/parser/properties/with/CreateSourceAsPropertiesTest.java @@ -17,7 +17,10 @@ import static com.google.common.collect.ImmutableMap.of; import static io.confluent.ksql.parser.properties.with.CreateSourceAsProperties.from; +import static io.confluent.ksql.properties.with.CommonCreateConfigs.FORMAT_PROPERTY; +import static io.confluent.ksql.properties.with.CommonCreateConfigs.KEY_FORMAT_PROPERTY; import static io.confluent.ksql.properties.with.CommonCreateConfigs.TIMESTAMP_FORMAT_PROPERTY; +import static io.confluent.ksql.properties.with.CommonCreateConfigs.VALUE_FORMAT_PROPERTY; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.is; @@ -53,10 +56,10 @@ public void shouldReturnOptionalEmptyForMissingProps() { // Then: assertThat(properties.getKafkaTopic(), is(Optional.empty())); - assertThat(properties.getValueFormat(), is(Optional.empty())); assertThat(properties.getTimestampColumnName(), is(Optional.empty())); assertThat(properties.getTimestampFormat(), is(Optional.empty())); - assertThat(properties.getFormatInfo(), is(Optional.empty())); + assertThat(properties.getKeyFormat(), is(Optional.empty())); + assertThat(properties.getValueFormat(), is(Optional.empty())); assertThat(properties.getReplicas(), is(Optional.empty())); assertThat(properties.getPartitions(), is(Optional.empty())); assertThat(properties.getSerdeOptions(), is(SerdeOptions.of())); @@ -106,7 +109,7 @@ public void shouldThrowOnInvalidTimestampFormat() { ); // Then: - assertThat(e.getMessage(), containsString("Invalid datatime format for config:TIMESTAMP_FORMAT, reason:Unknown pattern letter: i")); + assertThat(e.getMessage(), containsString("Invalid datetime format for config:TIMESTAMP_FORMAT, reason:Unknown pattern letter: i")); } @Test @@ -116,7 +119,7 @@ public void shouldSetValidAvroSchemaName() { ImmutableMap.of(CommonCreateConfigs.VALUE_AVRO_SCHEMA_FULL_NAME, new StringLiteral("schema"))); // Then: - assertThat(properties.getFormatProperties().get(AvroFormat.FULL_SCHEMA_NAME), is("schema")); + assertThat(properties.getValueFormatProperties().get(AvroFormat.FULL_SCHEMA_NAME), is("schema")); } @Test @@ -237,4 +240,53 @@ public void shouldNotQuoteNonStringPropValues() { // Then: assertThat(sql, containsString("WRAP_SINGLE_VALUE=true")); } + + @Test + public void shouldGetKeyAndValueFormatFromFormat() { + // Given: + final CreateSourceAsProperties props = CreateSourceAsProperties + .from(ImmutableMap.of( + KEY_FORMAT_PROPERTY, new StringLiteral("KAFKA"), + VALUE_FORMAT_PROPERTY, new StringLiteral("AVRO"))); + + // When / Then: + assertThat(props.getKeyFormat().get().getFormat(), is("KAFKA")); + assertThat(props.getValueFormat().get().getFormat(), is("AVRO")); + } + + @Test + public void shouldThrowIfKeyFormatAndFormatProvided() { + // When: + final Exception e = assertThrows( + KsqlException.class, + () -> CreateSourceAsProperties.from( + ImmutableMap.builder() + .put(KEY_FORMAT_PROPERTY, new StringLiteral("KAFKA")) + .put(FORMAT_PROPERTY, new StringLiteral("JSON")) + .build()) + ); + + // Then: + assertThat(e.getMessage(), containsString("Cannot supply both 'KEY_FORMAT' and 'FORMAT' properties, " + + "as 'FORMAT' sets both key and value formats.")); + assertThat(e.getMessage(), containsString("Either use just 'FORMAT', or use 'KEY_FORMAT' and 'VALUE_FORMAT'.")); + } + + @Test + public void shouldThrowIfValueFormatAndFormatProvided() { + // When: + final Exception e = assertThrows( + KsqlException.class, + () -> CreateSourceAsProperties.from( + ImmutableMap.builder() + .put(VALUE_FORMAT_PROPERTY, new StringLiteral("JSON")) + .put(FORMAT_PROPERTY, new StringLiteral("KAFKA")) + .build()) + ); + + // Then: + assertThat(e.getMessage(), containsString("Cannot supply both 'VALUE_FORMAT' and 'FORMAT' properties, " + + "as 'FORMAT' sets both key and value formats.")); + assertThat(e.getMessage(), containsString("Either use just 'FORMAT', or use 'KEY_FORMAT' and 'VALUE_FORMAT'.")); + } } \ No newline at end of file diff --git a/ksqldb-parser/src/test/java/io/confluent/ksql/parser/properties/with/CreateSourcePropertiesTest.java b/ksqldb-parser/src/test/java/io/confluent/ksql/parser/properties/with/CreateSourcePropertiesTest.java index e8bc5006b6c8..5e8cc484f51f 100644 --- a/ksqldb-parser/src/test/java/io/confluent/ksql/parser/properties/with/CreateSourcePropertiesTest.java +++ b/ksqldb-parser/src/test/java/io/confluent/ksql/parser/properties/with/CreateSourcePropertiesTest.java @@ -17,7 +17,9 @@ import static com.google.common.collect.ImmutableMap.of; import static io.confluent.ksql.parser.properties.with.CreateSourceAsProperties.from; +import static io.confluent.ksql.properties.with.CommonCreateConfigs.FORMAT_PROPERTY; import static io.confluent.ksql.properties.with.CommonCreateConfigs.KAFKA_TOPIC_NAME_PROPERTY; +import static io.confluent.ksql.properties.with.CommonCreateConfigs.KEY_FORMAT_PROPERTY; import static io.confluent.ksql.properties.with.CommonCreateConfigs.TIMESTAMP_FORMAT_PROPERTY; import static io.confluent.ksql.properties.with.CommonCreateConfigs.VALUE_FORMAT_PROPERTY; import static io.confluent.ksql.properties.with.CreateConfigs.WINDOW_SIZE_PROPERTY; @@ -39,7 +41,6 @@ import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.properties.with.CommonCreateConfigs; import io.confluent.ksql.properties.with.CreateConfigs; -import io.confluent.ksql.serde.FormatFactory; import io.confluent.ksql.serde.FormatInfo; import io.confluent.ksql.serde.SerdeOption; import io.confluent.ksql.serde.SerdeOptions; @@ -59,7 +60,6 @@ public class CreateSourcePropertiesTest { private static final java.util.Map MINIMUM_VALID_PROPS = ImmutableMap.of( - CommonCreateConfigs.VALUE_FORMAT_PROPERTY, new StringLiteral("AvRo"), CommonCreateConfigs.KAFKA_TOPIC_NAME_PROPERTY, new StringLiteral("foo") ); @@ -73,7 +73,6 @@ public void shouldSetMinimumValidProps() { // Then: assertThat(properties.getKafkaTopic(), is("foo")); - assertThat(properties.getValueFormat(), is(FormatFactory.AVRO)); } @Test @@ -86,7 +85,8 @@ public void shouldReturnOptionalEmptyForMissingProps() { assertThat(properties.getTimestampFormat(), is(Optional.empty())); assertThat(properties.getWindowType(), is(Optional.empty())); assertThat(properties.getSchemaId(), is(Optional.empty())); - assertThat(properties.getFormatInfo(), is(FormatInfo.of("AvRo"))); + assertThat(properties.getKeyFormat(), is(Optional.empty())); + assertThat(properties.getValueFormat(), is(Optional.empty())); assertThat(properties.getReplicas(), is(Optional.empty())); assertThat(properties.getPartitions(), is(Optional.empty())); assertThat(properties.getSerdeOptions(), is(SerdeOptions.of())); @@ -131,7 +131,7 @@ public void shouldThrowOnConstructionInvalidTimestampFormat() { ); // Then: - assertThat(e.getMessage(), containsString("Invalid datatime format for config:TIMESTAMP_FORMAT, reason:Unknown pattern letter: i")); + assertThat(e.getMessage(), containsString("Invalid datetime format for config:TIMESTAMP_FORMAT, reason:Unknown pattern letter: i")); } @Test @@ -290,11 +290,15 @@ public void shouldSetValidAvroSchemaName() { final CreateSourceProperties properties = CreateSourceProperties.from( ImmutableMap.builder() .putAll(MINIMUM_VALID_PROPS) + .put(CommonCreateConfigs.VALUE_FORMAT_PROPERTY, new StringLiteral("AvRo")) .put(CommonCreateConfigs.VALUE_AVRO_SCHEMA_FULL_NAME, new StringLiteral("schema")) .build()); // Then: - assertThat(properties.getFormatInfo().getProperties().get(AvroFormat.FULL_SCHEMA_NAME), is("schema")); + assertThat(properties.getValueFormat() + .map(FormatInfo::getProperties) + .map(props -> props.get(AvroFormat.FULL_SCHEMA_NAME)), + is(Optional.of("schema"))); } @Test @@ -391,22 +395,6 @@ public void shouldFailIfNoKafkaTopicName() { assertThat(e.getMessage(), containsString("Missing required property \"KAFKA_TOPIC\" which has no default value.")); } - @Test - public void shouldFailIfNoValueFormat() { - // Given: - final HashMap props = new HashMap<>(MINIMUM_VALID_PROPS); - props.remove(VALUE_FORMAT_PROPERTY); - - // When: - final Exception e = assertThrows( - KsqlException.class, - () -> CreateSourceProperties.from(props) - ); - - // Then: - assertThat(e.getMessage(), containsString("Missing required property \"VALUE_FORMAT\" which has no default value.")); - } - @Test public void shouldFailIfInvalidWindowConfig() { // When: @@ -469,7 +457,7 @@ public void shouldIncludeOnlyProvidedPropsInToString() { final String sql = props.toString(); // Then: - assertThat(sql, is("KAFKA_TOPIC='foo', VALUE_FORMAT='AvRo', WRAP_SINGLE_VALUE='True'")); + assertThat(sql, is("KAFKA_TOPIC='foo', WRAP_SINGLE_VALUE='True'")); } @Test @@ -487,4 +475,57 @@ public void shouldNotQuoteNonStringPropValues() { // Then: assertThat(sql, containsString("WRAP_SINGLE_VALUE=true")); } + + @Test + public void shouldGetKeyAndValueFormatFromFormat() { + // Given: + final CreateSourceProperties props = CreateSourceProperties + .from(ImmutableMap.builder() + .putAll(MINIMUM_VALID_PROPS) + .put(KEY_FORMAT_PROPERTY, new StringLiteral("KAFKA")) + .put(VALUE_FORMAT_PROPERTY, new StringLiteral("AVRO")) + .build()); + + // When / Then: + assertThat(props.getKeyFormat().get().getFormat(), is("KAFKA")); + assertThat(props.getValueFormat().get().getFormat(), is("AVRO")); + } + + @Test + public void shouldThrowIfKeyFormatAndFormatProvided() { + // When: + final Exception e = assertThrows( + KsqlException.class, + () -> CreateSourceProperties.from( + ImmutableMap.builder() + .putAll(MINIMUM_VALID_PROPS) + .put(KEY_FORMAT_PROPERTY, new StringLiteral("KAFKA")) + .put(FORMAT_PROPERTY, new StringLiteral("JSON")) + .build()) + ); + + // Then: + assertThat(e.getMessage(), containsString("Cannot supply both 'KEY_FORMAT' and 'FORMAT' properties, " + + "as 'FORMAT' sets both key and value formats.")); + assertThat(e.getMessage(), containsString("Either use just 'FORMAT', or use 'KEY_FORMAT' and 'VALUE_FORMAT'.")); + } + + @Test + public void shouldThrowIfValueFormatAndFormatProvided() { + // When: + final Exception e = assertThrows( + KsqlException.class, + () -> CreateSourceProperties.from( + ImmutableMap.builder() + .putAll(MINIMUM_VALID_PROPS) + .put(VALUE_FORMAT_PROPERTY, new StringLiteral("JSON")) + .put(FORMAT_PROPERTY, new StringLiteral("KAFKA")) + .build()) + ); + + // Then: + assertThat(e.getMessage(), containsString("Cannot supply both 'VALUE_FORMAT' and 'FORMAT' properties, " + + "as 'FORMAT' sets both key and value formats.")); + assertThat(e.getMessage(), containsString("Either use just 'FORMAT', or use 'KEY_FORMAT' and 'VALUE_FORMAT'.")); + } } \ No newline at end of file diff --git a/ksqldb-parser/src/test/java/io/confluent/ksql/parser/tree/QueryTest.java b/ksqldb-parser/src/test/java/io/confluent/ksql/parser/tree/QueryTest.java index a8801cd0e493..51144f24c306 100644 --- a/ksqldb-parser/src/test/java/io/confluent/ksql/parser/tree/QueryTest.java +++ b/ksqldb-parser/src/test/java/io/confluent/ksql/parser/tree/QueryTest.java @@ -25,8 +25,10 @@ import io.confluent.ksql.name.SourceName; import io.confluent.ksql.parser.NodeLocation; import io.confluent.ksql.serde.RefinementInfo; + import java.util.Optional; import java.util.OptionalInt; + import org.junit.Test; diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/impl/InsertsSubscriber.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/impl/InsertsSubscriber.java index 705904b3eea0..7d7c1790275a 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/impl/InsertsSubscriber.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/impl/InsertsSubscriber.java @@ -31,6 +31,7 @@ import io.confluent.ksql.serde.GenericRowSerDe; import io.confluent.ksql.serde.KeySerdeFactory; import io.confluent.ksql.serde.ValueSerdeFactory; +import io.confluent.ksql.serde.connect.ConnectSchemas; import io.confluent.ksql.services.ServiceContext; import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.VertxUtils; @@ -44,6 +45,7 @@ import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.connect.data.ConnectSchema; import org.apache.kafka.connect.data.Struct; import org.reactivestreams.Subscriber; import org.reactivestreams.Subscription; @@ -59,6 +61,7 @@ public final class InsertsSubscriber extends BaseSubscriber implemen private final Producer producer; private final DataSource dataSource; + private final ConnectSchema keySchema; private final Serializer keySerializer; private final Serializer valueSerializer; private final BufferedPublisher acksPublisher; @@ -67,10 +70,15 @@ public final class InsertsSubscriber extends BaseSubscriber implemen private boolean drainHandlerSet; private long sequence; - public static InsertsSubscriber createInsertsSubscriber(final ServiceContext serviceContext, - final JsonObject properties, final DataSource dataSource, final KsqlConfig ksqlConfig, - final Context context, final Subscriber acksSubscriber, - final WorkerExecutor workerExecutor) { + public static InsertsSubscriber createInsertsSubscriber( + final ServiceContext serviceContext, + final JsonObject properties, + final DataSource dataSource, + final KsqlConfig ksqlConfig, + final Context context, + final Subscriber acksSubscriber, + final WorkerExecutor workerExecutor + ) { final KsqlConfig configCopy = ksqlConfig.cloneWithPropertyOverwrite(properties.getMap()); final Producer producer = serviceContext .getKafkaClientSupplier() @@ -107,15 +115,19 @@ public static InsertsSubscriber createInsertsSubscriber(final ServiceContext ser valueSerde.serializer(), acksPublisher, workerExecutor); } - private InsertsSubscriber(final Context context, - final Producer producer, final DataSource dataSource, + private InsertsSubscriber( + final Context context, + final Producer producer, + final DataSource dataSource, final Serializer keySerializer, final Serializer valueSerializer, final BufferedPublisher acksPublisher, - final WorkerExecutor workerExecutor) { + final WorkerExecutor workerExecutor + ) { super(context); this.producer = Objects.requireNonNull(producer); this.dataSource = Objects.requireNonNull(dataSource); + this.keySchema = ConnectSchemas.columnsToConnectSchema(dataSource.getSchema().key()); this.keySerializer = Objects.requireNonNull(keySerializer); this.valueSerializer = Objects.requireNonNull(valueSerializer); this.acksPublisher = Objects.requireNonNull(acksPublisher); @@ -190,7 +202,7 @@ private void checkRequest() { } private Struct extractKey(final JsonObject values) { - return KeyValueExtractor.extractKey(values, dataSource.getSchema(), SQL_VALUE_COERCER); + return KeyValueExtractor.extractKey(values, keySchema, SQL_VALUE_COERCER); } private GenericRow extractValues(final JsonObject values) { diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/impl/KeyValueExtractor.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/impl/KeyValueExtractor.java index 68392a608568..4d88571a4d9e 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/impl/KeyValueExtractor.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/impl/KeyValueExtractor.java @@ -23,7 +23,6 @@ import io.confluent.ksql.schema.ksql.SchemaConverters; import io.confluent.ksql.schema.ksql.SqlValueCoercer; import io.confluent.ksql.schema.ksql.types.SqlType; -import io.confluent.ksql.serde.connect.ConnectSchemas; import io.confluent.ksql.util.ParserUtil; import io.vertx.core.json.JsonObject; import java.util.ArrayList; @@ -39,11 +38,10 @@ private KeyValueExtractor() { public static Struct extractKey( final JsonObject values, - final LogicalSchema logicalSchema, + final ConnectSchema connectSchema, final SqlValueCoercer sqlValueCoercer ) { - final ConnectSchema keySchema = ConnectSchemas.columnsToConnectSchema(logicalSchema.key()); - final Struct key = new Struct(keySchema); + final Struct key = new Struct(connectSchema); for (final Field field : key.schema().fields()) { final Object value = values.getValue(field.name()); if (value == null) { diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/impl/QueryEndpoint.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/impl/QueryEndpoint.java index 40edf92cf2ec..e72611337062 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/impl/QueryEndpoint.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/api/impl/QueryEndpoint.java @@ -15,8 +15,10 @@ package io.confluent.ksql.api.impl; +import com.google.common.collect.ImmutableMap; import io.confluent.ksql.api.server.PushQueryHandle; import io.confluent.ksql.api.spi.QueryPublisher; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.parser.KsqlParser.ParsedStatement; @@ -98,7 +100,7 @@ private QueryPublisher createPullQueryPublisher( final long startTimeNanos ) { final PullQueryResult result = pullQueryExecutor.execute( - statement, serviceContext, Optional.of(false), startTimeNanos); + statement, ImmutableMap.of(), serviceContext, Optional.of(false), startTimeNanos); final TableRows tableRows = result.getTableRows(); return new PullQueryPublisher( @@ -125,7 +127,7 @@ private ConfiguredStatement createStatement(final String queryString, } @SuppressWarnings("unchecked") final PreparedStatement psq = (PreparedStatement) ps; - return ConfiguredStatement.of(psq, properties, ksqlConfig); + return ConfiguredStatement.of(psq, SessionConfig.of(ksqlConfig, properties)); } private static List colTypesFromSchema(final List columns) { diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/logging/processing/ProcessingLogServerUtils.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/logging/processing/ProcessingLogServerUtils.java index 404c80d42d9d..e3630e905723 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/logging/processing/ProcessingLogServerUtils.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/logging/processing/ProcessingLogServerUtils.java @@ -78,13 +78,15 @@ public static String processingLogStreamCreateStatement( ) { return processingLogStreamCreateStatement( config.getString(ProcessingLogConfig.STREAM_NAME), - getTopicName(config, ksqlConfig) + getTopicName(config, ksqlConfig), + ksqlConfig.getBoolean(KsqlConfig.KSQL_KEY_FORMAT_ENABLED) ); } private static String processingLogStreamCreateStatement( final String name, - final String topicName + final String topicName, + final boolean withKeyFormat ) { final Schema schema = getMessageSchema(); @@ -92,6 +94,8 @@ private static String processingLogStreamCreateStatement( return "CREATE STREAM " + name + " (" + elements + ")" - + " WITH(KAFKA_TOPIC='" + topicName + "', VALUE_FORMAT='JSON');"; + + " WITH(KAFKA_TOPIC='" + topicName + "', VALUE_FORMAT='JSON'" + + (withKeyFormat ? ", KEY_FORMAT='KAFKA'" : "") + + ");"; } } diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/BackupReplayFile.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/BackupReplayFile.java index 08fb1c16dcaa..49d7b64829e7 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/BackupReplayFile.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/BackupReplayFile.java @@ -21,6 +21,7 @@ import io.confluent.ksql.rest.server.computation.Command; import io.confluent.ksql.util.KsqlException; import io.confluent.ksql.util.Pair; + import java.io.BufferedWriter; import java.io.Closeable; import java.io.File; diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java index f8e7b942d6b4..4504750791e7 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlRestApplication.java @@ -34,6 +34,7 @@ import io.confluent.ksql.api.impl.MonitoredEndpoints; import io.confluent.ksql.api.server.Server; import io.confluent.ksql.api.spi.Endpoints; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.engine.QueryMonitor; import io.confluent.ksql.execution.streams.RoutingFilter; @@ -894,13 +895,16 @@ private void registerCommandTopic() { final String createCmd = "CREATE STREAM " + COMMANDS_STREAM_NAME + " (STATEMENT STRING)" - + " WITH(VALUE_FORMAT='JSON', KAFKA_TOPIC='" + commandTopic + "');"; + + " WITH(KEY_FORMAT='KAFKA', VALUE_FORMAT='JSON', KAFKA_TOPIC='" + commandTopic + "');"; final ParsedStatement parsed = ksqlEngine.parse(createCmd).get(0); final PreparedStatement prepared = ksqlEngine.prepare(parsed); ksqlEngine.execute( serviceContext, - ConfiguredStatement.of(prepared, ImmutableMap.of(), ksqlConfigNoPort) + ConfiguredStatement.of( + prepared, + SessionConfig.of(ksqlConfigNoPort, ImmutableMap.of()) + ) ); } diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlServerMain.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlServerMain.java index b6215fc61329..00a30b4861c9 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlServerMain.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/KsqlServerMain.java @@ -17,7 +17,11 @@ import com.google.common.annotations.VisibleForTesting; import io.confluent.ksql.properties.PropertiesUtil; +import io.confluent.ksql.serde.Format; +import io.confluent.ksql.serde.FormatFactory; +import io.confluent.ksql.serde.KeyFormatUtils; import io.confluent.ksql.util.KsqlConfig; +import io.confluent.ksql.util.KsqlException; import io.confluent.ksql.util.KsqlServerException; import java.io.File; import java.io.IOException; @@ -51,10 +55,8 @@ public static void main(final String[] args) { final String installDir = properties.getOrDefault("ksql.server.install.dir", ""); final KsqlConfig ksqlConfig = new KsqlConfig(properties); - final String streamsStateDirPath = ksqlConfig.getKsqlStreamConfigProps().getOrDefault( - StreamsConfig.STATE_DIR_CONFIG, - StreamsConfig.configDef().defaultValues().get(StreamsConfig.STATE_DIR_CONFIG)).toString(); - enforceStreamStateDirAvailability(new File(streamsStateDirPath)); + validateConfig(ksqlConfig); + final Optional queriesFile = serverOptions.getQueriesFile(properties); final Executable executable = createExecutable( properties, queriesFile, installDir, ksqlConfig); @@ -101,6 +103,47 @@ void tryStartApp() throws Exception { } } + private static void validateConfig(final KsqlConfig config) { + validateStateDir(config); + validateDefaultTopicFormats(config); + } + + private static void validateStateDir(final KsqlConfig config) { + final String streamsStateDirPath = config.getKsqlStreamConfigProps().getOrDefault( + StreamsConfig.STATE_DIR_CONFIG, + StreamsConfig.configDef().defaultValues().get(StreamsConfig.STATE_DIR_CONFIG)).toString(); + enforceStreamStateDirAvailability(new File(streamsStateDirPath)); + } + + @VisibleForTesting + static void validateDefaultTopicFormats(final KsqlConfig config) { + validateTopicFormat(config, KsqlConfig.KSQL_DEFAULT_KEY_FORMAT_CONFIG, "key"); + validateTopicFormat(config, KsqlConfig.KSQL_DEFAULT_VALUE_FORMAT_CONFIG, "value"); + } + + private static void validateTopicFormat( + final KsqlConfig config, + final String configName, + final String type + ) { + final String formatName = config.getString(configName); + if (formatName == null) { + return; + } + + final Format format; + try { + format = FormatFactory.fromName(formatName); + } catch (KsqlException e) { + throw new KsqlException("Invalid value for config '" + configName + "': " + formatName, e); + } + + if (type.equals("key") && !KeyFormatUtils.isSupportedKeyFormat(config, format)) { + throw new KsqlException("Invalid value for config '" + configName + "': " + + "The supplied format is not currently supported as a key format. " + + "Format: '" + formatName + "'."); + } + } @SuppressWarnings("OptionalUsedAsFieldOrParameterType") private static Executable createExecutable( diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/StandaloneExecutor.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/StandaloneExecutor.java index 57bb95616e70..08d5ae753a94 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/StandaloneExecutor.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/StandaloneExecutor.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import io.confluent.ksql.KsqlExecutionContext; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.function.UserFunctionLoader; import io.confluent.ksql.logging.processing.ProcessingLogConfig; @@ -317,8 +318,8 @@ private ConfiguredStatement prepare( final ParsedStatement statement ) { final PreparedStatement prepared = executionContext.prepare(statement); - final ConfiguredStatement configured = ConfiguredStatement.of( - prepared, configOverrides, ksqlConfig); + final ConfiguredStatement configured = ConfiguredStatement + .of(prepared, SessionConfig.of(ksqlConfig, configOverrides)); return injector.inject(configured); } diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/Command.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/Command.java index 88c45539930c..9a74e4038b6e 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/Command.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/Command.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.KsqlPlan; import io.confluent.ksql.planner.plan.ConfiguredKsqlPlan; import io.confluent.ksql.rest.server.resources.IncomaptibleKsqlCommandVersionException; @@ -127,8 +128,8 @@ public Optional getVersion() { public static Command of(final ConfiguredKsqlPlan configuredPlan) { return new Command( configuredPlan.getPlan().getStatementText(), - configuredPlan.getOverrides(), - configuredPlan.getConfig().getAllConfigPropsWithSecretsObfuscated(), + configuredPlan.getConfig().getOverrides(), + configuredPlan.getConfig().getConfig(false).getAllConfigPropsWithSecretsObfuscated(), Optional.of(configuredPlan.getPlan()), Optional.of(VERSION), VERSION @@ -136,10 +137,12 @@ public static Command of(final ConfiguredKsqlPlan configuredPlan) { } public static Command of(final ConfiguredStatement configuredStatement) { + final SessionConfig sessionConfig = configuredStatement.getSessionConfig(); + return new Command( configuredStatement.getStatementText(), - configuredStatement.getConfigOverrides(), - configuredStatement.getConfig().getAllConfigPropsWithSecretsObfuscated(), + sessionConfig.getOverrides(), + sessionConfig.getConfig(false).getAllConfigPropsWithSecretsObfuscated(), Optional.empty(), Optional.of(VERSION), VERSION diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/InteractiveStatementExecutor.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/InteractiveStatementExecutor.java index 552d2516a82c..abc8f48af5e0 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/InteractiveStatementExecutor.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/InteractiveStatementExecutor.java @@ -17,6 +17,7 @@ import com.google.common.annotations.VisibleForTesting; import io.confluent.ksql.KsqlExecutionContext.ExecuteResult; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.engine.KsqlPlan; import io.confluent.ksql.exception.ExceptionUtil; @@ -42,7 +43,6 @@ import java.util.Objects; import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; - import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.streams.StreamsConfig; import org.slf4j.Logger; @@ -233,8 +233,7 @@ private void executePlan( final KsqlConfig mergedConfig = buildMergedConfig(command); final ConfiguredKsqlPlan configured = ConfiguredKsqlPlan.of( plan, - command.getOverwriteProperties(), - mergedConfig + SessionConfig.of(mergedConfig, command.getOverwriteProperties()) ); putStatus( commandId, diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/ValidatedCommandFactory.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/ValidatedCommandFactory.java index 0b74400f2153..1485812cbdf7 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/ValidatedCommandFactory.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/computation/ValidatedCommandFactory.java @@ -132,16 +132,12 @@ private static Command createForPlannedQuery( final KsqlExecutionContext context ) { final KsqlPlan plan = context.plan(serviceContext, statement); - context.execute( - serviceContext, - ConfiguredKsqlPlan.of( - plan, - statement.getConfigOverrides(), - statement.getConfig() - ) - ); - - return Command.of( - ConfiguredKsqlPlan.of(plan, statement.getConfigOverrides(), statement.getConfig())); + + final ConfiguredKsqlPlan configuredPlan = ConfiguredKsqlPlan + .of(plan, statement.getSessionConfig()); + + context.execute(serviceContext, configuredPlan); + + return Command.of(configuredPlan); } } diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/ExplainExecutor.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/ExplainExecutor.java index eea7d87726c0..9c5519ff197b 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/ExplainExecutor.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/ExplainExecutor.java @@ -39,12 +39,13 @@ import java.util.Optional; /** - * Explains the execution of either an existing persistent query or a statement - * that has not yet been issued. + * Explains the execution of either an existing persistent query or a statement that has not yet + * been issued. */ public final class ExplainExecutor { - private ExplainExecutor() { } + private ExplainExecutor() { + } public static Optional execute( final ConfiguredStatement statement, @@ -110,15 +111,13 @@ private static QueryDescription explainStatement( if (preparedStatement.getStatement() instanceof Query) { metadata = sandbox.executeQuery( serviceContext, - ConfiguredStatement.of( - preparedStatement, explain.getConfigOverrides(), explain.getConfig()).cast() + ConfiguredStatement.of(preparedStatement, explain.getSessionConfig()).cast() ); } else { metadata = sandbox .execute( serviceContext, - ConfiguredStatement - .of(preparedStatement, explain.getConfigOverrides(), explain.getConfig())) + ConfiguredStatement.of(preparedStatement, explain.getSessionConfig())) .getQuery() .orElseThrow(() -> new IllegalStateException("The provided statement did not run a ksql query")); diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/ListPropertiesExecutor.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/ListPropertiesExecutor.java index 51c2ef35ccb9..2aa8a7d3e197 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/ListPropertiesExecutor.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/ListPropertiesExecutor.java @@ -38,7 +38,8 @@ public final class ListPropertiesExecutor { - private ListPropertiesExecutor() { } + private ListPropertiesExecutor() { + } public static Optional execute( final ConfiguredStatement statement, @@ -48,8 +49,10 @@ public static Optional execute( ) { final KsqlConfigResolver resolver = new KsqlConfigResolver(); - final Map engineProperties - = statement.getConfig().getAllConfigPropsWithSecretsObfuscated(); + final Map engineProperties = statement + .getSessionConfig() + .getConfig(false) + .getAllConfigPropsWithSecretsObfuscated(); final List mergedProperties = mergedProperties(statement); @@ -75,8 +78,7 @@ private static List mergedProperties( final ConfiguredStatement statement) { final List mergedProperties = new ArrayList<>(); - statement.getConfig() - .cloneWithPropertyOverwrite(statement.getConfigOverrides()) + statement.getSessionConfig().getConfig(true) .getAllConfigPropsWithSecretsObfuscated() .forEach((key, value) -> mergedProperties.add(new Property(key, "KSQL", value))); @@ -88,9 +90,13 @@ private static List mergedProperties( } private static Map embeddedConnectWorkerProperties( - final ConfiguredStatement statement) { - final String configFile = statement.getConfig() + final ConfiguredStatement statement + ) { + final String configFile = statement + .getSessionConfig() + .getConfig(false) .getString(KsqlConfig.CONNECT_WORKER_CONFIG_FILE_PROPERTY); + return !configFile.isEmpty() ? Utils.propsToStringMap(getWorkerProps(configFile)) : Collections.emptyMap(); diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/ListSourceExecutor.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/ListSourceExecutor.java index 6a70b1040a53..648edfaa3506 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/ListSourceExecutor.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/ListSourceExecutor.java @@ -85,7 +85,7 @@ private static Optional sourceDescriptionList( final List descriptions = sources.stream() .map( s -> describeSource( - statement.getConfig(), + statement.getSessionConfig().getConfig(false), executionContext, serviceContext, s.getName(), @@ -161,7 +161,7 @@ public static Optional columns( ) { final ShowColumns showColumns = statement.getStatement(); final SourceDescriptionWithWarnings descriptionWithWarnings = describeSource( - statement.getConfig(), + statement.getSessionConfig().getConfig(false), executionContext, serviceContext, showColumns.getTable(), diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/ListTopicsExecutor.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/ListTopicsExecutor.java index 7d2bd56d2359..83659de93749 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/ListTopicsExecutor.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/ListTopicsExecutor.java @@ -86,7 +86,8 @@ private static Map listTopics( final KafkaTopicClient topicClient, final ConfiguredStatement statement ) { - final ReservedInternalTopics internalTopics = new ReservedInternalTopics(statement.getConfig()); + final ReservedInternalTopics internalTopics = + new ReservedInternalTopics(statement.getSessionConfig().getConfig(false)); final Set topics = statement.getStatement().getShowAll() ? topicClient.listTopicNames() diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/PullQueryExecutor.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/PullQueryExecutor.java index 75fd104cae0f..8d6a4fc2f245 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/PullQueryExecutor.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/PullQueryExecutor.java @@ -31,6 +31,7 @@ import io.confluent.ksql.analyzer.PullQueryValidator; import io.confluent.ksql.analyzer.QueryAnalyzer; import io.confluent.ksql.analyzer.RewrittenAnalysis; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.rewrite.ExpressionTreeRewriter.Context; import io.confluent.ksql.execution.context.QueryContext; import io.confluent.ksql.execution.context.QueryContext.Stacker; @@ -173,6 +174,7 @@ public static void validate( public PullQueryResult execute( final ConfiguredStatement statement, + final Map requestProperties, final ServiceContext serviceContext, final Optional isInternalRequest, final long startTimeNanos @@ -181,7 +183,10 @@ public PullQueryResult execute( throw new IllegalArgumentException("Executor can only handle pull queries"); } - if (!statement.getConfig().getBoolean(KsqlConfig.KSQL_PULL_QUERIES_ENABLE_CONFIG)) { + final SessionConfig sessionConfig = statement.getSessionConfig(); + + if (!sessionConfig.getConfig(false) + .getBoolean(KsqlConfig.KSQL_PULL_QUERIES_ENABLE_CONFIG)) { throw new KsqlStatementException( "Pull queries are disabled." + PullQueryValidator.PULL_QUERY_SYNTAX_HELP @@ -193,7 +198,10 @@ public PullQueryResult execute( try { final RoutingOptions routingOptions = new ConfigRoutingOptions( - statement.getConfig(), statement.getConfigOverrides(), statement.getRequestProperties()); + sessionConfig.getConfig(true), + requestProperties + ); + // If internal listeners are in use, we require the request to come from that listener to // treat it as having been forwarded. final boolean isAlreadyForwarded = routingOptions.skipForwardRequest() @@ -404,7 +412,7 @@ private static TableRows forwardTo( .makeQueryRequest( owner.location(), statement.getStatementText(), - statement.getConfigOverrides(), + statement.getSessionConfig().getOverrides(), requestProperties ); @@ -902,8 +910,7 @@ private static List> handleSelects( }; } - final KsqlConfig ksqlConfig = statement.getConfig() - .cloneWithPropertyOverwrite(statement.getConfigOverrides()); + final KsqlConfig ksqlConfig = statement.getSessionConfig().getConfig(true); final SelectValueMapper select = SelectValueMapperFactory.create( projection, @@ -1079,24 +1086,14 @@ public Optional visitQualifiedColumnReference( private static final class ConfigRoutingOptions implements RoutingOptions { private final KsqlConfig ksqlConfig; - private final Map configOverrides; private final Map requestProperties; ConfigRoutingOptions( final KsqlConfig ksqlConfig, - final Map configOverrides, final Map requestProperties ) { - this.ksqlConfig = ksqlConfig; - this.configOverrides = configOverrides; - this.requestProperties = requestProperties; - } - - private long getLong(final String key) { - if (configOverrides.containsKey(key)) { - return (Long) configOverrides.get(key); - } - return ksqlConfig.getLong(key); + this.ksqlConfig = Objects.requireNonNull(ksqlConfig, "ksqlConfig"); + this.requestProperties = Objects.requireNonNull(requestProperties, "requestProperties"); } private boolean getForwardedFlag(final String key) { @@ -1115,7 +1112,7 @@ public boolean isDebugRequest() { @Override public long getOffsetLagAllowed() { - return getLong(KsqlConfig.KSQL_QUERY_PULL_MAX_ALLOWED_OFFSET_LAG_CONFIG); + return ksqlConfig.getLong(KsqlConfig.KSQL_QUERY_PULL_MAX_ALLOWED_OFFSET_LAG_CONFIG); } @Override diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/RequestHandler.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/RequestHandler.java index c2441f38915e..f41f982ae341 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/RequestHandler.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/execution/RequestHandler.java @@ -15,6 +15,7 @@ package io.confluent.ksql.rest.server.execution; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.parser.KsqlParser.ParsedStatement; import io.confluent.ksql.parser.KsqlParser.PreparedStatement; @@ -74,8 +75,9 @@ public KsqlEntityList execute( final KsqlEntityList entities = new KsqlEntityList(); for (final ParsedStatement parsed : statements) { final PreparedStatement prepared = ksqlEngine.prepare(parsed); - final ConfiguredStatement configured = ConfiguredStatement.of( - prepared, sessionProperties.getMutableScopedProperties(), ksqlConfig); + final ConfiguredStatement configured = ConfiguredStatement.of(prepared, + SessionConfig.of(ksqlConfig, sessionProperties.getMutableScopedProperties()) + ); executeStatement( securityContext, diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/PullQueryPublisher.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/PullQueryPublisher.java index aa078473b7f9..c4fbfc117610 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/PullQueryPublisher.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/PullQueryPublisher.java @@ -19,6 +19,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; import io.confluent.ksql.GenericRow; import io.confluent.ksql.execution.streams.materialization.Locator.KsqlNode; import io.confluent.ksql.parser.tree.Query; @@ -60,7 +61,8 @@ class PullQueryPublisher implements Flow.Publisher> { public synchronized void subscribe(final Subscriber> subscriber) { final PullQuerySubscription subscription = new PullQuerySubscription( subscriber, - () -> pullQueryExecutor.execute(query, serviceContext, Optional.of(false), startTimeNanos) + () -> pullQueryExecutor + .execute(query, ImmutableMap.of(), serviceContext, Optional.of(false), startTimeNanos) ); subscriber.onSubscribe(subscription); diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/StreamedQueryResource.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/StreamedQueryResource.java index bd4273d3825e..c943b19d5a04 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/StreamedQueryResource.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/StreamedQueryResource.java @@ -19,6 +19,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import io.confluent.ksql.GenericRow; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.execution.streams.materialization.Locator.KsqlNode; import io.confluent.ksql.parser.KsqlParser.PreparedStatement; @@ -256,11 +257,12 @@ private EndpointResponse handlePullQuery( final Optional isInternalRequest, final long startTimeNanos ) { - final ConfiguredStatement configured = - ConfiguredStatement.of(statement, configOverrides, requestProperties, ksqlConfig); + final ConfiguredStatement configured = ConfiguredStatement + .of(statement, SessionConfig.of(ksqlConfig, configOverrides)); final PullQueryResult result = pullQueryExecutor - .execute(configured, serviceContext, isInternalRequest, startTimeNanos); + .execute(configured, requestProperties, serviceContext, isInternalRequest, startTimeNanos); + final TableRows tableRows = result.getTableRows(); final Optional host = result.getSourceNode() .map(KsqlNode::location) @@ -288,8 +290,8 @@ private EndpointResponse handlePushQuery( final Map streamsProperties, final CompletableFuture connectionClosedFuture ) { - final ConfiguredStatement configured = - ConfiguredStatement.of(statement, streamsProperties, ksqlConfig); + final ConfiguredStatement configured = ConfiguredStatement + .of(statement, SessionConfig.of(ksqlConfig, streamsProperties)); final TransientQueryMetadata query = ksqlEngine.executeQuery(serviceContext, configured); diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/WSQueryEndpoint.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/WSQueryEndpoint.java index baa7699e082d..f07eebdc4204 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/WSQueryEndpoint.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/resources/streaming/WSQueryEndpoint.java @@ -20,6 +20,7 @@ import static io.netty.handler.codec.http.websocketx.WebSocketCloseStatus.TRY_AGAIN_LATER; import com.google.common.util.concurrent.ListeningScheduledExecutorService; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.parser.KsqlParser.PreparedStatement; import io.confluent.ksql.parser.tree.PrintTopic; @@ -263,8 +264,8 @@ private void handleQuery(final RequestContext info, final Query query, final PreparedStatement statement = PreparedStatement.of(info.request.getKsql(), query); - final ConfiguredStatement configured = - ConfiguredStatement.of(statement, clientLocalProperties, ksqlConfig); + final ConfiguredStatement configured = ConfiguredStatement + .of(statement, SessionConfig.of(ksqlConfig, clientLocalProperties)); if (query.isPullQuery()) { pullQueryPublisher.start( diff --git a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/validation/RequestValidator.java b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/validation/RequestValidator.java index 4bfa1a989b87..7d4872e5baed 100644 --- a/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/validation/RequestValidator.java +++ b/ksqldb-rest-app/src/main/java/io/confluent/ksql/rest/server/validation/RequestValidator.java @@ -19,6 +19,7 @@ import static java.util.Objects.requireNonNull; import io.confluent.ksql.KsqlExecutionContext; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.parser.KsqlParser.ParsedStatement; import io.confluent.ksql.parser.KsqlParser.PreparedStatement; @@ -104,8 +105,9 @@ public int validate( int numPersistentQueries = 0; for (final ParsedStatement parsed : statements) { final PreparedStatement prepared = ctx.prepare(parsed); - final ConfiguredStatement configured = ConfiguredStatement.of( - prepared, sessionProperties.getMutableScopedProperties(), ksqlConfig); + final ConfiguredStatement configured = ConfiguredStatement.of(prepared, + SessionConfig.of(ksqlConfig, sessionProperties.getMutableScopedProperties()) + ); numPersistentQueries += validate(serviceContext, configured, sessionProperties, ctx, injector); diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/logging/processing/ProcessingLogServerUtilsTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/logging/processing/ProcessingLogServerUtilsTest.java index 9c76cd3503bc..003fec7db352 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/logging/processing/ProcessingLogServerUtilsTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/logging/processing/ProcessingLogServerUtilsTest.java @@ -122,6 +122,36 @@ public void shouldBuildCorrectStreamCreateDDL() { + ") WITH(KAFKA_TOPIC='processing_log_topic', VALUE_FORMAT='JSON');")); } + @Test + public void shouldBuildCorrectStreamCreateDDLWithKeyFormat() { + // Given: + serviceContext.getTopicClient().createTopic(TOPIC, 1, (short) 1); + + // When: + final String statement = + ProcessingLogServerUtils.processingLogStreamCreateStatement( + config, + ksqlConfig.cloneWithPropertyOverwrite(ImmutableMap.of( + KsqlConfig.KSQL_KEY_FORMAT_ENABLED, + true + ))); + + // Then: + assertThat(statement, equalTo( + "CREATE STREAM PROCESSING_LOG_STREAM (" + + "logger VARCHAR, " + + "level VARCHAR, " + + "time BIGINT, " + + "message STRUCT<" + + "type INT, " + + "deserializationError STRUCT, `topic` VARCHAR>, " + + "recordProcessingError STRUCT>, " + + "productionError STRUCT, " + + "serializationError STRUCT, `topic` VARCHAR>" + + ">" + + ") WITH(KAFKA_TOPIC='processing_log_topic', VALUE_FORMAT='JSON', KEY_FORMAT='KAFKA');")); + } + @Test public void shouldBuildCorrectStreamCreateDDLWithDefaultTopicName() { // Given: diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/parser/ParserMatchers.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/parser/ParserMatchers.java index 16d5149eaecc..c5d507491aea 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/parser/ParserMatchers.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/parser/ParserMatchers.java @@ -104,8 +104,8 @@ public static Matcher> configured( return new TypeSafeMatcher>() { @Override protected boolean matchesSafely(final ConfiguredStatement item) { - return Objects.equals(properties, item.getConfigOverrides()) - && Objects.equals(config, item.getConfig()); + return Objects.equals(properties, item.getSessionConfig().getOverrides()) + && Objects.equals(config, item.getSessionConfig().getConfig(false)); } @Override @@ -124,8 +124,8 @@ public static Matcher> configured( @Override protected boolean matchesSafely(final ConfiguredStatement item) { return statement.matches(PreparedStatement.of(item.getStatementText(), item.getStatement())) - && Objects.equals(properties, item.getConfigOverrides()) - && Objects.equals(config, item.getConfig()); + && Objects.equals(properties, item.getSessionConfig().getOverrides()) + && Objects.equals(config, item.getSessionConfig().getConfig(false)); } @Override diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/BackupReplayFileTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/BackupReplayFileTest.java index a9066f79326b..ba7ad945d7c5 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/BackupReplayFileTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/BackupReplayFileTest.java @@ -15,12 +15,16 @@ package io.confluent.ksql.rest.server; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.is; - import io.confluent.ksql.rest.entity.CommandId; import io.confluent.ksql.rest.server.computation.Command; import io.confluent.ksql.util.Pair; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.mockito.junit.MockitoJUnitRunner; + import java.io.File; import java.io.IOException; import java.nio.charset.StandardCharsets; @@ -28,12 +32,9 @@ import java.util.Arrays; import java.util.List; import java.util.Optional; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.mockito.junit.MockitoJUnitRunner; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; @RunWith(MockitoJUnitRunner.class) public class BackupReplayFileTest { diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicBackupImplTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicBackupImplTest.java index 6e7cb8c24fd3..c553f1785836 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicBackupImplTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicBackupImplTest.java @@ -15,27 +15,11 @@ package io.confluent.ksql.rest.server; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.not; -import static org.junit.Assert.assertThrows; -import static org.mockito.Mockito.when; -import static org.mockito.Mockito.verify; - import com.google.common.base.Ticker; import io.confluent.ksql.rest.entity.CommandId; import io.confluent.ksql.rest.server.computation.Command; import io.confluent.ksql.util.KsqlServerException; import io.confluent.ksql.util.Pair; -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.nio.file.attribute.PosixFilePermissions; -import java.util.List; -import java.util.Optional; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.junit.Before; import org.junit.Rule; @@ -45,6 +29,21 @@ import org.mockito.Mock; import org.mockito.junit.MockitoJUnitRunner; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.attribute.PosixFilePermissions; +import java.util.List; +import java.util.Optional; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThrows; +import static org.mockito.Mockito.when; + @RunWith(MockitoJUnitRunner.class) public class CommandTopicBackupImplTest { private static final String COMMAND_TOPIC_NAME = "command_topic"; diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicTest.java index 480960ff872b..b6bf0af3392c 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/CommandTopicTest.java @@ -29,6 +29,9 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.rest.entity.CommandId; +import io.confluent.ksql.rest.server.computation.Command; +import io.confluent.ksql.rest.server.computation.QueuedCommand; import java.nio.charset.Charset; import java.time.Duration; diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/KsqlServerMainTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/KsqlServerMainTest.java index 94ddd93cb33a..eb18a36f788b 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/KsqlServerMainTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/KsqlServerMainTest.java @@ -27,8 +27,13 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.KsqlConfigTestUtil; +import io.confluent.ksql.util.KsqlConfig; +import io.confluent.ksql.util.KsqlException; import io.confluent.ksql.util.KsqlServerException; import java.io.File; +import java.util.Map; import java.util.concurrent.Executor; import org.easymock.Capture; import org.easymock.EasyMockRunner; @@ -192,4 +197,89 @@ public void shouldFailIfStreamsStateDirectoryIsNotExacutable() { + " or change it to a writable directory by setting 'ksql.streams.state.dir' " + "config in the properties file.")); } + + @Test + public void shouldValidateDefaultFormatsWithCaseInsensitivity() { + // Given: + final KsqlConfig config = configWith(ImmutableMap.of( + KsqlConfig.KSQL_DEFAULT_VALUE_FORMAT_CONFIG, "avro" + )); + + // When: + KsqlServerMain.validateDefaultTopicFormats(config); + + // Then: No exception + } + + @Test + public void shouldValidateEmptyDefaultFormat() { + // Given: + // Default value format is empty + final KsqlConfig config = configWith(ImmutableMap.of()); + + // When: + KsqlServerMain.validateDefaultTopicFormats(config); + + // Then: No exception + } + + @Test + public void shouldFailOnInvalidDefaultKeyFormat() { + // Given: + final KsqlConfig config = configWith(ImmutableMap.of( + KsqlConfig.KSQL_DEFAULT_KEY_FORMAT_CONFIG, "bad" + )); + + // When: + final Exception e = assertThrows( + KsqlException.class, + () -> KsqlServerMain.validateDefaultTopicFormats(config) + ); + + // Then: + assertThat(e.getMessage(), containsString( + "Invalid value for config '" + KsqlConfig.KSQL_DEFAULT_KEY_FORMAT_CONFIG + "': bad")); + } + + @Test + public void shouldFailOnInvalidDefaultValueFormat() { + // Given: + final KsqlConfig config = configWith(ImmutableMap.of( + KsqlConfig.KSQL_DEFAULT_VALUE_FORMAT_CONFIG, "bad" + )); + + // When: + final Exception e = assertThrows( + KsqlException.class, + () -> KsqlServerMain.validateDefaultTopicFormats(config) + ); + + // Then: + assertThat(e.getMessage(), containsString( + "Invalid value for config '" + KsqlConfig.KSQL_DEFAULT_VALUE_FORMAT_CONFIG + "': bad")); + } + + @Test + public void shouldFailOnUnsupportedDefaultKeyFormat() { + // Given: + final KsqlConfig config = configWith(ImmutableMap.of( + KsqlConfig.KSQL_DEFAULT_KEY_FORMAT_CONFIG, "avro" + )); + + // When: + final Exception e = assertThrows( + KsqlException.class, + () -> KsqlServerMain.validateDefaultTopicFormats(config) + ); + + // Then: + assertThat(e.getMessage(), containsString( + "Invalid value for config '" + KsqlConfig.KSQL_DEFAULT_KEY_FORMAT_CONFIG + "': " + + "The supplied format is not currently supported as a key format. " + + "Format: 'avro'.")); + } + + private static KsqlConfig configWith(final Map additionalConfigs) { + return KsqlConfigTestUtil.create("unused", additionalConfigs); + } } diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/StandaloneExecutorFunctionalTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/StandaloneExecutorFunctionalTest.java index 389ae13b2cbb..c443e8e55195 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/StandaloneExecutorFunctionalTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/StandaloneExecutorFunctionalTest.java @@ -52,6 +52,7 @@ import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.FixMethodOrder; +import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/StandaloneExecutorTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/StandaloneExecutorTest.java index 1800c2237093..0ce5fcced0e4 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/StandaloneExecutorTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/StandaloneExecutorTest.java @@ -40,6 +40,7 @@ import com.google.common.collect.ImmutableMap; import io.confluent.ksql.KsqlExecutionContext; import io.confluent.ksql.KsqlExecutionContext.ExecuteResult; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.execution.expression.tree.StringLiteral; import io.confluent.ksql.execution.expression.tree.Type; @@ -184,11 +185,11 @@ public class StandaloneExecutorTest { private final static PreparedStatement PREPARED_CSAS = PreparedStatement. of("CSAS", CREATE_STREAM_AS_SELECT); - private final static ConfiguredStatement CFG_STMT_0 = ConfiguredStatement.of( - PREPARED_STMT_0, emptyMap(), ksqlConfig); + private final static ConfiguredStatement CFG_STMT_0 = ConfiguredStatement + .of(PREPARED_STMT_0, SessionConfig.of(ksqlConfig, emptyMap())); - private final static ConfiguredStatement CFG_STMT_1 = ConfiguredStatement.of( - PREPARED_STMT_1, emptyMap(), ksqlConfig); + private final static ConfiguredStatement CFG_STMT_1 = ConfiguredStatement + .of(PREPARED_STMT_1, SessionConfig.of(ksqlConfig, emptyMap())); private final static PreparedStatement STMT_0_WITH_SCHEMA = PreparedStatement .of("sql 0", new CreateStream( @@ -199,8 +200,8 @@ public class StandaloneExecutorTest { JSON_PROPS )); - private final static ConfiguredStatement CFG_0_WITH_SCHEMA = ConfiguredStatement.of( - STMT_0_WITH_SCHEMA, emptyMap(), ksqlConfig); + private final static ConfiguredStatement CFG_0_WITH_SCHEMA = ConfiguredStatement + .of(STMT_0_WITH_SCHEMA, SessionConfig.of(ksqlConfig, emptyMap())); private final static PreparedStatement STMT_1_WITH_SCHEMA = PreparedStatement .of("sql 1", new CreateStream( @@ -211,15 +212,16 @@ public class StandaloneExecutorTest { JSON_PROPS )); - private final static ConfiguredStatement CFG_1_WITH_SCHEMA = ConfiguredStatement.of( - STMT_1_WITH_SCHEMA, emptyMap(), ksqlConfig); + private final static ConfiguredStatement CFG_1_WITH_SCHEMA = ConfiguredStatement + .of(STMT_1_WITH_SCHEMA, SessionConfig.of(ksqlConfig, emptyMap())); private final static PreparedStatement CSAS_WITH_TOPIC = PreparedStatement .of("CSAS_TOPIC", CREATE_STREAM_AS_SELECT); private final static ConfiguredStatement CSAS_CFG_WITH_TOPIC = - ConfiguredStatement.of(CSAS_WITH_TOPIC, emptyMap(), ksqlConfig); + ConfiguredStatement + .of(CSAS_WITH_TOPIC, SessionConfig.of(ksqlConfig, emptyMap())); @Mock private Query query; @@ -517,7 +519,8 @@ public void shouldRunCsStatement() { standaloneExecutor.startAsync(); // Then: - verify(ksqlEngine).execute(serviceContext, ConfiguredStatement.of(cs, emptyMap(), ksqlConfig)); + verify(ksqlEngine).execute(serviceContext, + ConfiguredStatement.of(cs, SessionConfig.of(ksqlConfig, emptyMap()))); } @Test @@ -532,7 +535,8 @@ public void shouldRunCtStatement() { standaloneExecutor.startAsync(); // Then: - verify(ksqlEngine).execute(serviceContext, ConfiguredStatement.of(ct, emptyMap(), ksqlConfig)); + verify(ksqlEngine).execute(serviceContext, + ConfiguredStatement.of(ct, SessionConfig.of(ksqlConfig, emptyMap()))); } @Test @@ -552,10 +556,9 @@ public void shouldRunSetStatements() { // Then: verify(ksqlEngine).execute( serviceContext, - ConfiguredStatement.of( - cs, - ImmutableMap.of(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"), - ksqlConfig)); + ConfiguredStatement.of(cs, SessionConfig + .of(ksqlConfig, ImmutableMap.of(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")) + )); } @Test @@ -573,11 +576,8 @@ public void shouldSetPropertyOnlyOnCommandsFollowingTheSetStatement() { standaloneExecutor.startAsync(); // Then: - verify(ksqlEngine).execute(serviceContext, ConfiguredStatement.of( - cs, - ImmutableMap.of(), - ksqlConfig - )); + verify(ksqlEngine).execute(serviceContext, ConfiguredStatement + .of(cs, SessionConfig.of(ksqlConfig, ImmutableMap.of()))); } @Test @@ -592,7 +592,8 @@ public void shouldRunUnSetStatements() { final PreparedStatement cs = PreparedStatement.of("CS", new CreateStream(SOME_NAME, SOME_ELEMENTS, false, false, JSON_PROPS)); - final ConfiguredStatement configured = ConfiguredStatement.of(cs, emptyMap(), ksqlConfig); + final ConfiguredStatement configured = ConfiguredStatement + .of(cs, SessionConfig.of(ksqlConfig, emptyMap())); givenQueryFileParsesTo(setProp, unsetProp, cs); @@ -608,7 +609,8 @@ public void shouldRunCsasStatements() { // Given: final PreparedStatement csas = PreparedStatement.of("CSAS1", new CreateStreamAsSelect(SOME_NAME, query, false, false, CreateSourceAsProperties.none())); - final ConfiguredStatement configured = ConfiguredStatement.of(csas, emptyMap(), ksqlConfig); + final ConfiguredStatement configured = ConfiguredStatement + .of(csas, SessionConfig.of(ksqlConfig, emptyMap())); givenQueryFileParsesTo(csas); when(sandBox.execute(sandBoxServiceContext, configured)) @@ -626,7 +628,8 @@ public void shouldRunCtasStatements() { // Given: final PreparedStatement ctas = PreparedStatement.of("CTAS", new CreateTableAsSelect(SOME_NAME, query, false, false, CreateSourceAsProperties.none())); - final ConfiguredStatement configured = ConfiguredStatement.of(ctas, emptyMap(), ksqlConfig); + final ConfiguredStatement configured = ConfiguredStatement + .of(ctas, SessionConfig.of(ksqlConfig, emptyMap())); givenQueryFileParsesTo(ctas); @@ -645,7 +648,8 @@ public void shouldRunInsertIntoStatements() { // Given: final PreparedStatement insertInto = PreparedStatement.of("InsertInto", new InsertInto(SOME_NAME, query)); - final ConfiguredStatement configured = ConfiguredStatement.of(insertInto, emptyMap(), ksqlConfig); + final ConfiguredStatement configured = ConfiguredStatement + .of(insertInto, SessionConfig.of(ksqlConfig, emptyMap())); givenQueryFileParsesTo(insertInto); diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/TemporaryEngine.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/TemporaryEngine.java index 85c466a6ae9c..b8a79f845f54 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/TemporaryEngine.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/TemporaryEngine.java @@ -18,6 +18,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.confluent.ksql.KsqlConfigTestUtil; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.engine.KsqlEngineTestUtil; import io.confluent.ksql.execution.ddl.commands.KsqlTopic; @@ -54,7 +55,6 @@ import io.confluent.ksql.statement.ConfiguredStatement; import io.confluent.ksql.util.KsqlConfig; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -170,10 +170,8 @@ public void givenKafkaTopic(final String name) { } public ConfiguredStatement configure(final String sql) { - return ConfiguredStatement.of( - getEngine().prepare(new DefaultKsqlParser().parse(sql).get(0)), - new HashMap<>(), - ksqlConfig); + return ConfiguredStatement.of(getEngine().prepare(new DefaultKsqlParser().parse(sql).get(0)), + SessionConfig.of(ksqlConfig, ImmutableMap.of())); } public KsqlConfig getKsqlConfig() { diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/TestKsqlRestApp.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/TestKsqlRestApp.java index c3dd38f26c3f..e600aa92523d 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/TestKsqlRestApp.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/TestKsqlRestApp.java @@ -23,6 +23,7 @@ import io.confluent.ksql.KsqlExecutionContext; import io.confluent.ksql.query.QueryId; import io.confluent.ksql.rest.client.BasicCredentials; +import io.confluent.ksql.rest.client.KsqlClient; import io.confluent.ksql.rest.client.KsqlRestClient; import io.confluent.ksql.rest.client.RestResponse; import io.confluent.ksql.rest.entity.CommandStatusEntity; @@ -41,6 +42,7 @@ import io.confluent.ksql.services.SimpleKsqlClient; import io.confluent.ksql.test.util.EmbeddedSingleNodeKafkaCluster; import io.confluent.ksql.util.KsqlConfig; +import io.confluent.ksql.util.KsqlConstants; import io.confluent.ksql.util.KsqlConstants.KsqlQueryType; import io.confluent.ksql.util.ReservedInternalTopics; import io.confluent.ksql.version.metrics.VersionCheckerAgent; diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/DistributingExecutorTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/DistributingExecutorTest.java index f409e745fab8..05cfd3e0d00a 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/DistributingExecutorTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/DistributingExecutorTest.java @@ -33,6 +33,7 @@ import com.google.common.collect.ImmutableMap; import io.confluent.ksql.KsqlExecutionContext; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.exception.KsqlTopicAuthorizationException; import io.confluent.ksql.execution.expression.tree.StringLiteral; import io.confluent.ksql.metastore.MetaStore; @@ -71,7 +72,6 @@ import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Supplier; - import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.common.errors.TimeoutException; import org.junit.Before; @@ -101,11 +101,8 @@ CommonCreateConfigs.VALUE_FORMAT_PROPERTY, new StringLiteral("json") )) ); private static final ConfiguredStatement CONFIGURED_STATEMENT = - ConfiguredStatement.of( - PreparedStatement.of("statement", STATEMENT), - ImmutableMap.of(), - KSQL_CONFIG - ); + ConfiguredStatement.of(PreparedStatement.of("statement", STATEMENT), + SessionConfig.of(KSQL_CONFIG, ImmutableMap.of())); private static final CommandIdAssigner IDGEN = new CommandIdAssigner(); @Mock @@ -272,7 +269,8 @@ public void shouldThrowFailureIfCannotInferSchema() { final PreparedStatement preparedStatement = PreparedStatement.of("", new ListProperties(Optional.empty())); final ConfiguredStatement configured = - ConfiguredStatement.of(preparedStatement, ImmutableMap.of(), KSQL_CONFIG); + ConfiguredStatement.of(preparedStatement, SessionConfig.of(KSQL_CONFIG, ImmutableMap.of()) + ); when(schemaInjector.inject(any())).thenThrow(new KsqlException("Could not infer!")); // When: @@ -295,7 +293,8 @@ public void shouldThrowExceptionIfUserServiceContextIsDeniedAuthorization() { final PreparedStatement preparedStatement = PreparedStatement.of("", new ListProperties(Optional.empty())); final ConfiguredStatement configured = - ConfiguredStatement.of(preparedStatement, ImmutableMap.of(), KSQL_CONFIG); + ConfiguredStatement.of(preparedStatement, SessionConfig.of(KSQL_CONFIG, ImmutableMap.of()) + ); doThrow(KsqlTopicAuthorizationException.class).when(authorizationValidator) .checkAuthorization(eq(userSecurityContext), any(), eq(configured.getStatement())); @@ -314,7 +313,8 @@ public void shouldThrowServerExceptionIfServerServiceContextIsDeniedAuthorizatio final PreparedStatement preparedStatement = PreparedStatement.of("", new ListProperties(Optional.empty())); final ConfiguredStatement configured = - ConfiguredStatement.of(preparedStatement, ImmutableMap.of(), KSQL_CONFIG); + ConfiguredStatement.of(preparedStatement, SessionConfig.of(KSQL_CONFIG, ImmutableMap.of()) + ); doNothing().when(authorizationValidator) .checkAuthorization(eq(userSecurityContext), any(), any()); doThrow(KsqlTopicAuthorizationException.class).when(authorizationValidator) @@ -339,7 +339,8 @@ public void shouldThrowExceptionWhenInsertIntoUnknownStream() { final PreparedStatement preparedStatement = PreparedStatement.of("", new InsertInto(SourceName.of("s1"), mock(Query.class))); final ConfiguredStatement configured = - ConfiguredStatement.of(preparedStatement, ImmutableMap.of(), KSQL_CONFIG); + ConfiguredStatement.of(preparedStatement, SessionConfig.of(KSQL_CONFIG, ImmutableMap.of()) + ); doReturn(null).when(metaStore).getSource(SourceName.of("s1")); // When: @@ -359,7 +360,8 @@ public void shouldThrowExceptionWhenInsertIntoReadOnlyTopic() { final PreparedStatement preparedStatement = PreparedStatement.of("", new InsertInto(SourceName.of("s1"), mock(Query.class))); final ConfiguredStatement configured = - ConfiguredStatement.of(preparedStatement, ImmutableMap.of(), KSQL_CONFIG); + ConfiguredStatement.of(preparedStatement, SessionConfig.of(KSQL_CONFIG, ImmutableMap.of()) + ); final DataSource dataSource = mock(DataSource.class); doReturn(dataSource).when(metaStore).getSource(SourceName.of("s1")); when(dataSource.getKafkaTopicName()).thenReturn("_confluent-ksql-default__command-topic"); @@ -382,7 +384,8 @@ public void shouldThrowExceptionWhenInsertIntoProcessingLogTopic() { final PreparedStatement preparedStatement = PreparedStatement.of("", new InsertInto(SourceName.of("s1"), mock(Query.class))); final ConfiguredStatement configured = - ConfiguredStatement.of(preparedStatement, ImmutableMap.of(), KSQL_CONFIG); + ConfiguredStatement.of(preparedStatement, SessionConfig.of(KSQL_CONFIG, ImmutableMap.of()) + ); final DataSource dataSource = mock(DataSource.class); doReturn(dataSource).when(metaStore).getSource(SourceName.of("s1")); when(dataSource.getKafkaTopicName()).thenReturn("default_ksql_processing_log"); diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/InteractiveStatementExecutorTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/InteractiveStatementExecutorTest.java index 70c83fd46cc4..bdd32fb80157 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/InteractiveStatementExecutorTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/InteractiveStatementExecutorTest.java @@ -37,6 +37,7 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import io.confluent.ksql.KsqlConfigTestUtil; import io.confluent.ksql.KsqlExecutionContext.ExecuteResult; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.engine.KsqlEngineTestUtil; import io.confluent.ksql.engine.KsqlPlan; @@ -99,6 +100,7 @@ public class InteractiveStatementExecutorTest { + "biz bigint," + " baz varchar) " + "WITH (kafka_topic = 'foo', " + + "key_format = 'kafka', " + "value_format = 'json');"; private static final CommandId COMMAND_ID = new CommandId(Type.STREAM, "foo", Action.CREATE); private static final QueryId QUERY_ID = new QueryId("qid"); @@ -264,12 +266,13 @@ public void shouldBuildQueriesWithPersistedConfig() { // get a statement instance final String ddlText = "CREATE STREAM pageviews (viewtime bigint, pageid varchar) " + - "WITH (kafka_topic='pageview_topic', VALUE_FORMAT='json');"; + "WITH (kafka_topic='pageview_topic', KEY_FORMAT='kafka', VALUE_FORMAT='json');"; final String statementText = "CREATE STREAM user1pv AS select * from pageviews WHERE userid = 'user1';"; final PreparedStatement ddlStatement = statementParser.parseSingleStatement(ddlText); final ConfiguredStatement configuredStatement = - ConfiguredStatement.of(ddlStatement, emptyMap(), originalConfig); + ConfiguredStatement + .of(ddlStatement, SessionConfig.of(originalConfig, emptyMap())); ksqlEngine.execute(serviceContext, configuredStatement); when(mockQueryMetadata.getQueryId()).thenReturn(mock(QueryId.class)); @@ -330,7 +333,10 @@ public void shouldExecutePlannedCommand() { // Then: final KsqlConfig expectedConfig = ksqlConfig.overrideBreakingConfigsWithOriginalValues( plannedCommand.getOriginalProperties()); - verify(mockEngine).execute(serviceContext, ConfiguredKsqlPlan.of(plan, emptyMap(), expectedConfig)); + verify(mockEngine).execute( + serviceContext, + ConfiguredKsqlPlan.of(plan, SessionConfig.of(expectedConfig, emptyMap())) + ); } @Test @@ -423,7 +429,10 @@ public void shouldExecutePlannedCommandWithMergedConfig() { // Then: verify(mockConfig).overrideBreakingConfigsWithOriginalValues(savedConfigs); - verify(mockEngine).execute(any(), eq(ConfiguredKsqlPlan.of(plan, emptyMap(), mergedConfig))); + verify(mockEngine).execute( + any(), + eq(ConfiguredKsqlPlan.of(plan, SessionConfig.of(mergedConfig, emptyMap()))) + ); } @Test @@ -442,9 +451,8 @@ public void shouldThrowExceptionIfCommandFails() { handleStatement(command, COMMAND_ID, Optional.of(status), 0L); } catch (final KsqlStatementException e) { // Then: - assertEquals("Cannot add stream 'FOO': A stream with the same name already exists\n" + - "Statement: " + CREATE_STREAM_FOO_STATEMENT, - e.getMessage()); + assertThat(e.getMessage(), + containsString("Cannot add stream 'FOO': A stream with the same name already exists")); } final InOrder inOrder = Mockito.inOrder(status); final ArgumentCaptor argCommandStatus = ArgumentCaptor.forClass(CommandStatus.class); @@ -669,7 +677,7 @@ private class ConfiguredKsqlPlanMatcher implements ArgumentMatcher implements Argumen private final ConfiguredStatement statement; ConfiguredStatementMatcher(final PreparedStatement preparedStatement) { - statement = ConfiguredStatement.of(preparedStatement, Collections.emptyMap(), ksqlConfig); + statement = ConfiguredStatement.of(preparedStatement, + SessionConfig.of(ksqlConfig, emptyMap())); } @Override @@ -770,6 +779,7 @@ private List> createStreamsAndStartTwoPersistentQueries + " pageid varchar, " + "userid varchar) " + "WITH (kafka_topic = 'pageview_topic_json', " + + "key_format = 'kafka', " + "value_format = 'json');", ksqlConfig.getAllConfigPropsWithSecretsObfuscated() ); @@ -988,7 +998,8 @@ private void givenMockPlannedQuery() { private Command commandWithPlan(final String sql, final Map originalProperties) { final PreparedStatement prepared = statementParser.parseSingleStatement(sql); - final ConfiguredStatement configured = ConfiguredStatement.of(prepared, Collections.emptyMap(), ksqlConfig); + final SessionConfig sessionConfig = SessionConfig.of(ksqlConfig, emptyMap()); + final ConfiguredStatement configured = ConfiguredStatement.of(prepared, sessionConfig); final KsqlPlan plan = ksqlEngine.plan(serviceContext, configured); return new Command(sql, Collections.emptyMap(), originalProperties, Optional.of(plan)); } diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/ValidatedCommandFactoryTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/ValidatedCommandFactoryTest.java index 8dcae49a45f2..e166c2dc82df 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/ValidatedCommandFactoryTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/computation/ValidatedCommandFactoryTest.java @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableList; import io.confluent.ksql.KsqlExecutionContext; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.KsqlPlan; import io.confluent.ksql.execution.ddl.commands.DdlCommand; import io.confluent.ksql.execution.ddl.commands.DdlCommandResult; @@ -180,7 +181,7 @@ public void shouldValidatePlannedQuery() { verify(executionContext).plan(serviceContext, configuredStatement); verify(executionContext).execute( serviceContext, - ConfiguredKsqlPlan.of(A_PLAN, overrides, config) + ConfiguredKsqlPlan.of(A_PLAN, SessionConfig.of(config, overrides)) ); } @@ -193,7 +194,7 @@ public void shouldCreateCommandForPlannedQuery() { final Command command = commandFactory.create(configuredStatement, executionContext); // Then: - assertThat(command, is(Command.of(ConfiguredKsqlPlan.of(A_PLAN, overrides, config)))); + assertThat(command, is(Command.of(ConfiguredKsqlPlan.of(A_PLAN, SessionConfig.of(config, overrides))))); } @Test @@ -242,11 +243,8 @@ private ConfiguredStatement configuredStatement( final String text, final T statement ) { - return ConfiguredStatement.of( - PreparedStatement.of(text, statement), - overrides, - config - ); + return ConfiguredStatement.of(PreparedStatement.of(text, statement), + SessionConfig.of(config, overrides)); } // Not a known subtype so will fail to deserialize: diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/ConnectExecutorTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/ConnectExecutorTest.java index 536ab47360b3..8cc54e51dd6e 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/ConnectExecutorTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/ConnectExecutorTest.java @@ -28,6 +28,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.execution.expression.tree.StringLiteral; import io.confluent.ksql.parser.KsqlParser.PreparedStatement; import io.confluent.ksql.parser.tree.CreateConnector; @@ -64,24 +65,19 @@ public class ConnectExecutorTest { false); private static final ConfiguredStatement CREATE_CONNECTOR_CONFIGURED = - ConfiguredStatement.of( - PreparedStatement.of( - "CREATE SOURCE CONNECTOR foo WITH ('foo'='bar');", - CREATE_CONNECTOR), - ImmutableMap.of(), - CONFIG); + ConfiguredStatement.of(PreparedStatement.of( + "CREATE SOURCE CONNECTOR foo WITH ('foo'='bar');", + CREATE_CONNECTOR), SessionConfig.of(CONFIG, ImmutableMap.of())); private static final CreateConnector CREATE_DUPLICATE_CONNECTOR = new CreateConnector( "foo", ImmutableMap.of("foo", new StringLiteral("bar")), Type.SOURCE, true); private static final ConfiguredStatement CREATE_DUPLICATE_CONNECTOR_CONFIGURED = - ConfiguredStatement.of( - PreparedStatement.of( - "CREATE SOURCE CONNECTOR IF NOT EXISTS foo WITH ('foo'='bar');", - CREATE_DUPLICATE_CONNECTOR), - ImmutableMap.of(), - CONFIG); + ConfiguredStatement.of(PreparedStatement.of( + "CREATE SOURCE CONNECTOR IF NOT EXISTS foo WITH ('foo'='bar');", + CREATE_DUPLICATE_CONNECTOR), SessionConfig.of(CONFIG, ImmutableMap.of()) + ); @Mock private ServiceContext serviceContext; diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/DescribeConnectorExecutorTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/DescribeConnectorExecutorTest.java index 624ab4ddef1b..be2e6a6510a1 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/DescribeConnectorExecutorTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/DescribeConnectorExecutorTest.java @@ -28,6 +28,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.confluent.ksql.KsqlExecutionContext; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.connect.Connector; import io.confluent.ksql.execution.ddl.commands.KsqlTopic; import io.confluent.ksql.metastore.MetaStore; @@ -60,13 +61,13 @@ import java.util.Map; import java.util.Optional; import java.util.function.Function; -import org.apache.hc.core5.http.HttpStatus; import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo.ConnectorState; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo.TaskState; import org.apache.kafka.connect.runtime.rest.entities.ConnectorType; +import org.apache.hc.core5.http.HttpStatus; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -146,10 +147,10 @@ public void setUp() { executor = new DescribeConnectorExecutor(connectorFactory); final DescribeConnector describeConnector = new DescribeConnector(Optional.empty(), "connector"); - describeStatement = ConfiguredStatement.of( - PreparedStatement.of("statementText", describeConnector), - ImmutableMap.of(), - new KsqlConfig(ImmutableMap.of())); + final KsqlConfig ksqlConfig = new KsqlConfig(ImmutableMap.of()); + describeStatement = ConfiguredStatement + .of(PreparedStatement.of("statementText", describeConnector), + SessionConfig.of(ksqlConfig, ImmutableMap.of())); } @After diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/DropConnectorExecutorTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/DropConnectorExecutorTest.java index 2d9219191de8..c8671402c083 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/DropConnectorExecutorTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/DropConnectorExecutorTest.java @@ -24,6 +24,7 @@ import static org.mockito.Mockito.when; import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.parser.KsqlParser.PreparedStatement; import io.confluent.ksql.parser.tree.DropConnector; import io.confluent.ksql.rest.SessionProperties; @@ -55,19 +56,14 @@ public class DropConnectorExecutorTest { new DropConnector(Optional.empty(), true, "foo"); private static final ConfiguredStatement DROP_CONNECTOR_CONFIGURED = - ConfiguredStatement.of( - PreparedStatement.of( - "DROP CONNECTOR \"foo\"", - DROP_CONNECTOR), - ImmutableMap.of(), - CONFIG); + ConfiguredStatement.of(PreparedStatement.of( + "DROP CONNECTOR \"foo\"", + DROP_CONNECTOR), SessionConfig.of(CONFIG, ImmutableMap.of())); private static final ConfiguredStatement DROP_CONNECTOR_IF_EXISTS_CONFIGURED = - ConfiguredStatement.of( - PreparedStatement.of( - "DROP CONNECTOR \"foo\"", - DROP_CONNECTOR_IF_EXISTS), - ImmutableMap.of(), - CONFIG); + ConfiguredStatement.of(PreparedStatement.of( + "DROP CONNECTOR \"foo\"", + DROP_CONNECTOR_IF_EXISTS), SessionConfig.of(CONFIG, ImmutableMap.of()) + ); @Mock private ServiceContext serviceContext; diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/ListConnectorsExecutorTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/ListConnectorsExecutorTest.java index 3054f60c1ea0..7038782ce1cf 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/ListConnectorsExecutorTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/ListConnectorsExecutorTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.confluent.ksql.KsqlExecutionContext; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.parser.KsqlParser.PreparedStatement; import io.confluent.ksql.parser.tree.ListConnectors; import io.confluent.ksql.parser.tree.ListConnectors.Scope; @@ -37,13 +38,13 @@ import io.confluent.ksql.statement.ConfiguredStatement; import io.confluent.ksql.util.KsqlConfig; import java.util.Optional; -import org.apache.hc.core5.http.HttpStatus; import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo.ConnectorState; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo.TaskState; import org.apache.kafka.connect.runtime.rest.entities.ConnectorType; +import org.apache.hc.core5.http.HttpStatus; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -106,11 +107,10 @@ public void shouldListValidConnector() { // Given: when(connectClient.connectors()) .thenReturn(ConnectResponse.success(ImmutableList.of("connector"), HttpStatus.SC_OK)); - final ConfiguredStatement statement = ConfiguredStatement.of( - PreparedStatement.of("", new ListConnectors(Optional.empty(), Scope.ALL)), - ImmutableMap.of(), - new KsqlConfig(ImmutableMap.of()) - ); + final KsqlConfig ksqlConfig = new KsqlConfig(ImmutableMap.of()); + final ConfiguredStatement statement = ConfiguredStatement + .of(PreparedStatement.of("", new ListConnectors(Optional.empty(), Scope.ALL)), + SessionConfig.of(ksqlConfig, ImmutableMap.of())); // When: final Optional entity = ListConnectorsExecutor @@ -136,11 +136,10 @@ public void shouldLabelConnectorsWithNoRunningTasksAsWarning() { .thenReturn(ConnectResponse.success(STATUS_WARNING, HttpStatus.SC_OK)); when(connectClient.connectors()) .thenReturn(ConnectResponse.success(ImmutableList.of("connector"), HttpStatus.SC_OK)); - final ConfiguredStatement statement = ConfiguredStatement.of( - PreparedStatement.of("", new ListConnectors(Optional.empty(), Scope.ALL)), - ImmutableMap.of(), - new KsqlConfig(ImmutableMap.of()) - ); + final KsqlConfig ksqlConfig = new KsqlConfig(ImmutableMap.of()); + final ConfiguredStatement statement = ConfiguredStatement + .of(PreparedStatement.of("", new ListConnectors(Optional.empty(), Scope.ALL)), + SessionConfig.of(ksqlConfig, ImmutableMap.of())); // When: final Optional entity = ListConnectorsExecutor @@ -165,11 +164,10 @@ public void shouldFilterNonMatchingConnectors() { when(connectClient.connectors()) .thenReturn(ConnectResponse.success(ImmutableList.of("connector", "connector2"), HttpStatus.SC_OK)); - final ConfiguredStatement statement = ConfiguredStatement.of( - PreparedStatement.of("", new ListConnectors(Optional.empty(), Scope.SINK)), - ImmutableMap.of(), - new KsqlConfig(ImmutableMap.of()) - ); + final ConfiguredStatement statement = ConfiguredStatement + .of(PreparedStatement.of("", new ListConnectors(Optional.empty(), Scope.SINK)), + SessionConfig.of(new KsqlConfig(ImmutableMap.of()), ImmutableMap.of()) + ); // When: final Optional entity = ListConnectorsExecutor @@ -191,11 +189,10 @@ public void shouldListInvalidConnectorWithNoInfo() { // Given: when(connectClient.connectors()) .thenReturn(ConnectResponse.success(ImmutableList.of("connector2"), HttpStatus.SC_OK)); - final ConfiguredStatement statement = ConfiguredStatement.of( - PreparedStatement.of("", new ListConnectors(Optional.empty(), Scope.ALL)), - ImmutableMap.of(), - new KsqlConfig(ImmutableMap.of()) - ); + final ConfiguredStatement statement = ConfiguredStatement + .of(PreparedStatement.of("", new ListConnectors(Optional.empty(), Scope.ALL)), + SessionConfig.of(new KsqlConfig(ImmutableMap.of()), ImmutableMap.of()) + ); // When: final Optional entity = ListConnectorsExecutor diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/ListSourceExecutorTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/ListSourceExecutorTest.java index f034523b4eff..fe61fe39df43 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/ListSourceExecutorTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/ListSourceExecutorTest.java @@ -34,6 +34,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import io.confluent.ksql.KsqlExecutionContext.ExecuteResult; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.metastore.model.DataSource; import io.confluent.ksql.metastore.model.KsqlStream; import io.confluent.ksql.metastore.model.KsqlTable; @@ -243,13 +244,10 @@ public void shouldShowColumnsSource() { // When: final SourceDescriptionEntity sourceDescription = (SourceDescriptionEntity) CustomExecutors.SHOW_COLUMNS.execute( - ConfiguredStatement.of( - PreparedStatement.of( - "DESCRIBE SINK;", - new ShowColumns(SourceName.of("SINK"), false)), - ImmutableMap.of(), - engine.getKsqlConfig() - ), + ConfiguredStatement.of(PreparedStatement.of( + "DESCRIBE SINK;", + new ShowColumns(SourceName.of("SINK"), false)), + SessionConfig.of(engine.getKsqlConfig(), ImmutableMap.of())), mock(SessionProperties.class), engine.getEngine(), engine.getServiceContext() diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/ListTypesExecutorTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/ListTypesExecutorTest.java index 1f6c780cc28f..b9df69bd71bc 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/ListTypesExecutorTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/ListTypesExecutorTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.confluent.ksql.KsqlExecutionContext; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.metastore.MetaStore; import io.confluent.ksql.metastore.TypeRegistry.CustomType; import io.confluent.ksql.parser.KsqlParser.PreparedStatement; @@ -66,11 +67,8 @@ public void setUp() { public void shouldListTypes() { // When: final Optional entity = ListTypesExecutor.execute( - ConfiguredStatement.of( - PreparedStatement.of("statement", new ListTypes(Optional.empty())), - ImmutableMap.of(), - KSQL_CONFIG - ), + ConfiguredStatement.of(PreparedStatement.of("statement", new ListTypes(Optional.empty())), + SessionConfig.of(KSQL_CONFIG, ImmutableMap.of())), mock(SessionProperties.class), context, null); diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/PullQueryExecutorTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/PullQueryExecutorTest.java index 863365208471..201f27e11d97 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/PullQueryExecutorTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/execution/PullQueryExecutorTest.java @@ -29,6 +29,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.execution.streams.RoutingFilter.RoutingFilterFactory; import io.confluent.ksql.execution.streams.RoutingFilters; import io.confluent.ksql.parser.KsqlParser.PreparedStatement; @@ -69,11 +70,9 @@ public void shouldThrowExceptionIfConfigDisabled() { // Given: final Query theQuery = mock(Query.class); when(theQuery.isPullQuery()).thenReturn(true); - final ConfiguredStatement query = ConfiguredStatement.of( - PreparedStatement.of("SELECT * FROM test_table;", theQuery), - ImmutableMap.of(), - engine.getKsqlConfig() - ); + final ConfiguredStatement query = ConfiguredStatement + .of(PreparedStatement.of("SELECT * FROM test_table;", theQuery), + SessionConfig.of(engine.getKsqlConfig(), ImmutableMap.of())); PullQueryExecutor pullQueryExecutor = new PullQueryExecutor( engine.getEngine(), ROUTING_FILTER_FACTORY, engine.getKsqlConfig(), engine.getEngine().getServiceId(), time); @@ -81,7 +80,7 @@ public void shouldThrowExceptionIfConfigDisabled() { // When: final Exception e = assertThrows( KsqlStatementException.class, - () -> pullQueryExecutor.execute(query, engine.getServiceContext(), Optional.empty(), 0L) + () -> pullQueryExecutor.execute(query, ImmutableMap.of(), engine.getServiceContext(), Optional.empty(), 0L) ); // Then: @@ -99,11 +98,9 @@ public static class Enabled { @Test public void shouldRedirectQueriesToQueryEndPoint() { // Given: - final ConfiguredStatement query = ConfiguredStatement.of( - PreparedStatement.of("SELECT * FROM test_table;", mock(Query.class)), - ImmutableMap.of(), - engine.getKsqlConfig() - ); + final ConfiguredStatement query = ConfiguredStatement + .of(PreparedStatement.of("SELECT * FROM test_table;", mock(Query.class)), + SessionConfig.of(engine.getKsqlConfig(), ImmutableMap.of())); // When: final KsqlRestException e = assertThrows( diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java index 36c24a147be8..5850eb3b962d 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/KsqlResourceTest.java @@ -74,6 +74,7 @@ import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException; import io.confluent.ksql.KsqlConfigTestUtil; import io.confluent.ksql.KsqlExecutionContext; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.engine.KsqlEngineTestUtil; import io.confluent.ksql.engine.KsqlPlan; @@ -81,6 +82,7 @@ import io.confluent.ksql.execution.ddl.commands.DropSourceCommand; import io.confluent.ksql.execution.ddl.commands.KsqlTopic; import io.confluent.ksql.execution.expression.tree.StringLiteral; +import io.confluent.ksql.format.DefaultFormatInjector; import io.confluent.ksql.function.FunctionCategory; import io.confluent.ksql.function.InternalFunctionRegistry; import io.confluent.ksql.function.MutableFunctionRegistry; @@ -234,12 +236,12 @@ public class KsqlResourceTest { true, CreateSourceProperties.from(ImmutableMap.of( "KAFKA_TOPIC", new StringLiteral("orders-topic"), + "KEY_FORMAT", new StringLiteral("kafka"), "VALUE_FORMAT", new StringLiteral("avro") )))); private static final ConfiguredStatement CFG_0_WITH_SCHEMA = ConfiguredStatement.of( STMT_0_WITH_SCHEMA, - ImmutableMap.of(), - new KsqlConfig(getDefaultKsqlConfig()) + SessionConfig.of(new KsqlConfig(getDefaultKsqlConfig()), ImmutableMap.of()) ); private static final PreparedStatement STMT_1_WITH_SCHEMA = PreparedStatement.of( @@ -251,13 +253,13 @@ public class KsqlResourceTest { true, CreateSourceProperties.from(ImmutableMap.of( "KAFKA_TOPIC", new StringLiteral("orders-topic"), + "KEY_FORMAT", new StringLiteral("kafka"), "VALUE_FORMAT", new StringLiteral("avro") )))); - private static final ConfiguredStatement CFG_1_WITH_SCHEMA = ConfiguredStatement.of( - STMT_1_WITH_SCHEMA, - ImmutableMap.of(), - new KsqlConfig(getDefaultKsqlConfig()) - ); + private static final ConfiguredStatement CFG_1_WITH_SCHEMA = ConfiguredStatement + .of(STMT_1_WITH_SCHEMA, + SessionConfig.of(new KsqlConfig(getDefaultKsqlConfig()), ImmutableMap.of()) + ); private static final LogicalSchema SOME_SCHEMA = LogicalSchema.builder() .keyColumn(SystemColumns.ROWKEY_NAME, SqlTypes.STRING) @@ -845,7 +847,7 @@ public void shouldReturnStatusEntityFromPersistentQuery() { public void shouldFailIfCreateStatementMissingKafkaTopicName() { // When: final KsqlErrorMessage result = makeFailingRequest( - "CREATE STREAM S (foo INT) WITH(VALUE_FORMAT='JSON');", + "CREATE STREAM S (foo INT) WITH(KEY_FORMAT='KAFKA', VALUE_FORMAT='JSON');", BAD_REQUEST.code()); // Then: @@ -854,7 +856,7 @@ public void shouldFailIfCreateStatementMissingKafkaTopicName() { assertThat(result.getMessage(), containsString("Missing required property \"KAFKA_TOPIC\" which has no default value.")); assertThat(((KsqlStatementErrorMessage) result).getStatementText(), - is("CREATE STREAM S (foo INT) WITH(VALUE_FORMAT='JSON');")); + is("CREATE STREAM S (foo INT) WITH(KEY_FORMAT='KAFKA', VALUE_FORMAT='JSON');")); } @Test @@ -900,7 +902,7 @@ public void shouldNotDistributeCreateStatementIfTopicDoesNotExist() { // When: final KsqlRestException e = assertThrows( KsqlRestException.class, - () -> makeRequest("CREATE STREAM S (foo INT) WITH(VALUE_FORMAT='JSON', KAFKA_TOPIC='unknown');") + () -> makeRequest("CREATE STREAM S (foo INT) WITH(KEY_FORMAT='KAFKA', VALUE_FORMAT='JSON', KAFKA_TOPIC='unknown');") ); // Then: @@ -912,14 +914,14 @@ public void shouldNotDistributeCreateStatementIfTopicDoesNotExist() { public void shouldDistributeAvroCreateStatementWithColumns() { // When: makeSingleRequest( - "CREATE STREAM S (foo INT) WITH(VALUE_FORMAT='AVRO', KAFKA_TOPIC='orders-topic');", + "CREATE STREAM S (foo INT) WITH(KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO', KAFKA_TOPIC='orders-topic');", CommandStatusEntity.class); // Then: verify(commandStore).enqueueCommand( any(), argThat(is(commandWithStatement( - "CREATE STREAM S (foo INT) WITH(VALUE_FORMAT='AVRO', KAFKA_TOPIC='orders-topic');"))), + "CREATE STREAM S (foo INT) WITH(KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO', KAFKA_TOPIC='orders-topic');"))), any(Producer.class) ); } @@ -936,7 +938,8 @@ public void shouldSupportTopicInferenceInVerification() { final PreparedStatement statementWithTopic = ksqlEngine.prepare(ksqlEngine.parse(sqlWithTopic).get(0)); final ConfiguredStatement configuredStatement = - ConfiguredStatement.of(statementWithTopic, ImmutableMap.of(), ksqlConfig); + ConfiguredStatement.of(statementWithTopic, SessionConfig.of(ksqlConfig, ImmutableMap.of()) + ); when(sandboxTopicInjector.inject(argThat(is(configured(preparedStatementText(sql)))))) .thenReturn((ConfiguredStatement) configuredStatement); @@ -965,7 +968,8 @@ public void shouldSupportTopicInferenceInExecution() { final PreparedStatement statementWithTopic = ksqlEngine.prepare(ksqlEngine.parse(sqlWithTopic).get(0)); final ConfiguredStatement configured = - ConfiguredStatement.of(statementWithTopic, ImmutableMap.of(), ksqlConfig); + ConfiguredStatement.of(statementWithTopic, SessionConfig.of(ksqlConfig, ImmutableMap.of()) + ); when(topicInjector.inject(argThat(is(configured(preparedStatementText(sql)))))) .thenReturn((ConfiguredStatement) configured); @@ -1023,7 +1027,7 @@ public void shouldSupportSchemaInference() { // Given: givenMockEngine(); - final String sql = "CREATE STREAM NO_SCHEMA WITH(VALUE_FORMAT='AVRO', KAFKA_TOPIC='orders-topic');"; + final String sql = "CREATE STREAM NO_SCHEMA WITH(KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO', KAFKA_TOPIC='orders-topic');"; when(sandboxSchemaInjector.inject(argThat(configured(preparedStatementText(sql))))) .thenReturn((ConfiguredStatement) CFG_0_WITH_SCHEMA); @@ -1051,7 +1055,7 @@ public void shouldFailWhenAvroInferenceFailsDuringValidate() { // When: final KsqlErrorMessage result = makeFailingRequest( - "CREATE STREAM S WITH(VALUE_FORMAT='AVRO', KAFKA_TOPIC='orders-topic');", + "CREATE STREAM S WITH(KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO', KAFKA_TOPIC='orders-topic');", BAD_REQUEST.code()); // Then: @@ -1071,7 +1075,7 @@ public void shouldFailWhenAvroInferenceFailsDuringExecute() { // When: final KsqlRestException e = assertThrows( KsqlRestException.class, - () -> makeRequest("CREATE STREAM S WITH(VALUE_FORMAT='AVRO', KAFKA_TOPIC='orders-topic');") + () -> makeRequest("CREATE STREAM S WITH(KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO', KAFKA_TOPIC='orders-topic');") ); // Then: @@ -1085,7 +1089,7 @@ public void shouldFailIfNoSchemaAndNotInferred() { // When: final KsqlRestException e = assertThrows( KsqlRestException.class, - () -> makeRequest("CREATE STREAM S WITH(VALUE_FORMAT='AVRO', KAFKA_TOPIC='orders-topic');") + () -> makeRequest("CREATE STREAM S WITH(KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO', KAFKA_TOPIC='orders-topic');") ); // Then: @@ -1101,7 +1105,7 @@ public void shouldFailWhenAvroSchemaCanNotBeEvolved() { // When: final KsqlErrorMessage result = makeFailingRequest( - "CREATE STREAM S1 WITH(VALUE_FORMAT='AVRO') AS SELECT * FROM test_stream;", + "CREATE STREAM S1 WITH(KEY_FORMAT='KAFKA', VALUE_FORMAT='AVRO') AS SELECT * FROM test_stream;", BAD_REQUEST.code()); // Then: @@ -1903,7 +1907,7 @@ public void shouldFailIfCreateExistingSourceStream() { givenSource(DataSourceType.KSTREAM, "SOURCE", "topic1", SINGLE_FIELD_SCHEMA); givenKafkaTopicExists("topic2"); final String createSql = - "CREATE STREAM SOURCE (val int) WITH (kafka_topic='topic2', value_format='json');"; + "CREATE STREAM SOURCE (val int) WITH (kafka_topic='topic2', key_format='kafka', value_format='json');"; // When: final KsqlRestException e = assertThrows( @@ -1923,7 +1927,7 @@ public void shouldFailIfCreateExistingSourceTable() { givenSource(DataSourceType.KTABLE, "SOURCE", "topic1", SINGLE_FIELD_SCHEMA); givenKafkaTopicExists("topic2"); final String createSql = - "CREATE TABLE SOURCE (id int primary key, val int) WITH (kafka_topic='topic2', value_format='json');"; + "CREATE TABLE SOURCE (id int primary key, val int) WITH (kafka_topic='topic2', key_format='kafka', value_format='json');"; // When: final KsqlRestException e = assertThrows( @@ -2372,6 +2376,7 @@ private static Properties getDefaultKsqlConfig() { configMap.put("ksql.command.topic.suffix", "commands"); configMap.put(KsqlRestConfig.LISTENERS_CONFIG, "http://localhost:8088"); configMap.put(StreamsConfig.APPLICATION_SERVER_CONFIG, APPLICATION_SERVER); + configMap.put(KsqlConfig.KSQL_DEFAULT_KEY_FORMAT_CONFIG, "KAFKA"); final Properties properties = new Properties(); properties.putAll(configMap); diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/streaming/PullQueryPublisherTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/streaming/PullQueryPublisherTest.java index 40678affe424..3d4d51392b44 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/streaming/PullQueryPublisherTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/streaming/PullQueryPublisherTest.java @@ -23,6 +23,7 @@ import static org.mockito.Mockito.when; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import io.confluent.ksql.GenericRow; import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.name.ColumnName; @@ -85,7 +86,7 @@ public void setUp() { TIME_NANOS); PullQueryResult result = new PullQueryResult(entity, Optional.empty()); - when(pullQueryExecutor.execute(any(), any(), any(), eq(TIME_NANOS))).thenReturn(result); + when(pullQueryExecutor.execute(any(), any(), any(), any(), eq(TIME_NANOS))).thenReturn(result); when(entity.getSchema()).thenReturn(SCHEMA); doAnswer(callRequestAgain()).when(subscriber).onNext(any()); @@ -109,7 +110,7 @@ public void shouldRunQueryWithCorrectParams() { subscription.request(1); // Then: - verify(pullQueryExecutor).execute(statement, serviceContext, Optional.of(false), TIME_NANOS); + verify(pullQueryExecutor).execute(statement, ImmutableMap.of(), serviceContext, Optional.of(false), TIME_NANOS); } @Test @@ -122,7 +123,7 @@ public void shouldOnlyExecuteOnce() { // Then: verify(subscriber).onNext(any()); - verify(pullQueryExecutor).execute(statement, serviceContext, Optional.of(false), TIME_NANOS); + verify(pullQueryExecutor).execute(statement, ImmutableMap.of(), serviceContext, Optional.of(false), TIME_NANOS); } @Test @@ -157,7 +158,7 @@ public void shouldCallOnErrorOnFailure() { // Given: givenSubscribed(); final Throwable e = new RuntimeException("Boom!"); - when(pullQueryExecutor.execute(any(), any(), any(), eq(TIME_NANOS))).thenThrow(e); + when(pullQueryExecutor.execute(any(), any(), any(), any(), eq(TIME_NANOS))).thenThrow(e); // When: subscription.request(1); diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/streaming/StreamedQueryResourceTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/streaming/StreamedQueryResourceTest.java index a1721541a036..339af1d414e7 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/streaming/StreamedQueryResourceTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/resources/streaming/StreamedQueryResourceTest.java @@ -46,6 +46,7 @@ import com.google.common.collect.ImmutableSet; import io.confluent.ksql.GenericRow; import io.confluent.ksql.api.server.StreamingOutput; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.engine.KsqlEngine; import io.confluent.ksql.exception.KsqlTopicAuthorizationException; import io.confluent.ksql.execution.streams.RoutingFilter.RoutingFilterFactory; @@ -476,7 +477,8 @@ public void shouldStreamRowsCorrectly() throws Throwable { 10); when(mockKsqlEngine.executeQuery(serviceContext, - ConfiguredStatement.of(query, requestStreamsProperties, VALID_CONFIG))) + ConfiguredStatement + .of(query, SessionConfig.of(VALID_CONFIG, requestStreamsProperties)))) .thenReturn(transientQueryMetadata); final EndpointResponse response = diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/validation/PrintTopicValidatorTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/validation/PrintTopicValidatorTest.java index e27e19f0b319..e263fdce6224 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/validation/PrintTopicValidatorTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/validation/PrintTopicValidatorTest.java @@ -28,6 +28,7 @@ import com.google.common.collect.ImmutableMap; import io.confluent.ksql.KsqlExecutionContext; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.parser.KsqlParser.PreparedStatement; import io.confluent.ksql.parser.tree.PrintTopic; import io.confluent.ksql.rest.SessionProperties; @@ -54,11 +55,9 @@ public class PrintTopicValidatorTest { @Test public void shouldThrowExceptionOnPrintTopic() { // Given: - final ConfiguredStatement query = ConfiguredStatement.of( - PreparedStatement.of("PRINT 'topic';", mock(PrintTopic.class)), - ImmutableMap.of(), - CONFIG - ); + final ConfiguredStatement query = ConfiguredStatement + .of(PreparedStatement.of("PRINT 'topic';", mock(PrintTopic.class)), + SessionConfig.of(CONFIG, ImmutableMap.of())); // When: final KsqlRestException e = assertThrows( diff --git a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/validation/PropertyOverriderTest.java b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/validation/PropertyOverriderTest.java index 971232dc531a..ff052c494f8e 100644 --- a/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/validation/PropertyOverriderTest.java +++ b/ksqldb-rest-app/src/test/java/io/confluent/ksql/rest/server/validation/PropertyOverriderTest.java @@ -24,6 +24,7 @@ import static org.mockito.Mockito.mock; import com.google.common.collect.ImmutableMap; +import io.confluent.ksql.config.SessionConfig; import io.confluent.ksql.parser.KsqlParser.PreparedStatement; import io.confluent.ksql.parser.tree.SetProperty; import io.confluent.ksql.parser.tree.UnsetProperty; @@ -56,13 +57,10 @@ public void shouldFailOnUnknownSetProperty() { final Exception e = assertThrows( KsqlStatementException.class, () -> CustomValidators.SET_PROPERTY.validate( - ConfiguredStatement.of( - PreparedStatement.of( - "SET 'consumer.invalid'='value';", - new SetProperty(Optional.empty(), "consumer.invalid", "value")), - new HashMap<>(), - engine.getKsqlConfig() - ), + ConfiguredStatement.of(PreparedStatement.of( + "SET 'consumer.invalid'='value';", + new SetProperty(Optional.empty(), "consumer.invalid", "value")), + SessionConfig.of(engine.getKsqlConfig(), ImmutableMap.of())), mock(SessionProperties.class), engine.getEngine(), engine.getServiceContext() @@ -83,13 +81,10 @@ public void shouldAllowSetKnownProperty() { // When: CustomValidators.SET_PROPERTY.validate( - ConfiguredStatement.of( - PreparedStatement.of( - "SET '" + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + "' = 'earliest';", - new SetProperty(Optional.empty(), ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")), - ImmutableMap.of(), - engine.getKsqlConfig() - ), + ConfiguredStatement.of(PreparedStatement.of( + "SET '" + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + "' = 'earliest';", + new SetProperty(Optional.empty(), ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")), + SessionConfig.of(engine.getKsqlConfig(), ImmutableMap.of())), sessionProperties, engine.getEngine(), engine.getServiceContext() @@ -110,12 +105,10 @@ public void shouldFailOnInvalidSetPropertyValue() { final Exception e = assertThrows( KsqlStatementException.class, () -> CustomValidators.SET_PROPERTY.validate( - ConfiguredStatement.of( - PreparedStatement.of( - "SET '" + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + "' = 'invalid';", - new SetProperty(Optional.empty(), ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "invalid")), - ImmutableMap.of(), - engine.getKsqlConfig() + ConfiguredStatement.of(PreparedStatement.of( + "SET '" + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + "' = 'invalid';", + new SetProperty(Optional.empty(), ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, + "invalid")), SessionConfig.of(engine.getKsqlConfig(), ImmutableMap.of()) ), sessionProperties, engine.getEngine(), @@ -139,13 +132,10 @@ public void shouldFailOnUnknownUnsetProperty() { final Exception e = assertThrows( KsqlStatementException.class, () -> CustomValidators.UNSET_PROPERTY.validate( - ConfiguredStatement.of( - PreparedStatement.of( - "UNSET 'consumer.invalid';", - new UnsetProperty(Optional.empty(), "consumer.invalid")), - new HashMap<>(), - engine.getKsqlConfig() - ), + ConfiguredStatement.of(PreparedStatement.of( + "UNSET 'consumer.invalid';", + new UnsetProperty(Optional.empty(), "consumer.invalid")), + SessionConfig.of(engine.getKsqlConfig(), new HashMap<>())), sessionProperties, engine.getEngine(), engine.getServiceContext() @@ -171,13 +161,10 @@ public void shouldAllowUnsetKnownProperty() { // When: CustomValidators.UNSET_PROPERTY.validate( - ConfiguredStatement.of( - PreparedStatement.of( - "UNSET '" + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + "';", - new UnsetProperty(Optional.empty(), ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)), - ImmutableMap.of(), - engine.getKsqlConfig() - ), + ConfiguredStatement.of(PreparedStatement.of( + "UNSET '" + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + "';", + new UnsetProperty(Optional.empty(), ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)), + SessionConfig.of(engine.getKsqlConfig(), ImmutableMap.of())), sessionProperties, engine.getEngine(), engine.getServiceContext() diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/Format.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/Format.java index 0111cf224db2..6a103667881d 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/Format.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/Format.java @@ -30,7 +30,6 @@ import java.util.function.Supplier; import javax.annotation.concurrent.ThreadSafe; import org.apache.kafka.common.serialization.Serde; -import org.apache.kafka.connect.data.Struct; /** * A {@code Format} is a serialization specification of a Kafka topic @@ -174,10 +173,11 @@ default Set getInheritableProperties() { * @param srClientFactory supplier of the SR client * @return a serde pair capable of (de)serializing the data in this format. */ - Serde getSerde( + Serde> getSerde( PersistenceSchema schema, Map formatProperties, KsqlConfig ksqlConfig, Supplier srClientFactory ); + } diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/FormatFactory.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/FormatFactory.java index 2d169e23e32a..c23e561eb8e0 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/FormatFactory.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/FormatFactory.java @@ -50,7 +50,7 @@ public static Format of(final FormatInfo formatInfo) { } public static Format fromName(final String name) { - switch (name) { + switch (name.toUpperCase()) { case AvroFormat.NAME: return AVRO; case JsonFormat.NAME: return JSON; case JsonSchemaFormat.NAME: return JSON_SR; diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericKeySerDe.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericKeySerDe.java index b4938643ec1b..8af9473f0e0d 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericKeySerDe.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericKeySerDe.java @@ -15,15 +15,27 @@ package io.confluent.ksql.serde; +import static java.util.Objects.requireNonNull; + import com.google.common.annotations.VisibleForTesting; import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; import io.confluent.ksql.logging.processing.ProcessingLogContext; import io.confluent.ksql.schema.ksql.PersistenceSchema; +import io.confluent.ksql.serde.connect.ConnectSchemas; import io.confluent.ksql.util.KsqlConfig; +import java.util.ArrayList; import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.function.Supplier; +import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.connect.data.ConnectSchema; +import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.WindowedSerdes.SessionWindowedSerde; @@ -94,14 +106,111 @@ private Serde createInner( final String loggerNamePrefix, final ProcessingLogContext processingLogContext ) { - final Serde formatSerde = innerFactory + final Serde> formatSerde = innerFactory .createFormatSerde("Key", format, schema, ksqlConfig, schemaRegistryClientFactory); + final Serde structSerde = toStructSerde(formatSerde, schema); + final Serde serde = innerFactory - .wrapInLoggingSerde(formatSerde, loggerNamePrefix, processingLogContext); + .wrapInLoggingSerde(structSerde, loggerNamePrefix, processingLogContext); serde.configure(Collections.emptyMap(), true); return serde; } + + private static Serde toStructSerde( + final Serde> inner, + final PersistenceSchema schema + ) { + final ConnectSchema connectSchema = ConnectSchemas.columnsToConnectSchema(schema.columns()); + return Serdes.serdeFrom( + new GenericKeySerializer(inner.serializer(), connectSchema.fields().size()), + new GenericKeyDeserializer(inner.deserializer(), connectSchema) + ); + } + + @VisibleForTesting + static class GenericKeySerializer implements Serializer { + + private final Serializer> inner; + private final int numColumns; + + GenericKeySerializer(final Serializer> inner, final int numColumns) { + this.inner = requireNonNull(inner, "inner"); + this.numColumns = numColumns; + } + + @Override + public void configure(final Map configs, final boolean isKey) { + inner.configure(configs, isKey); + } + + @Override + public byte[] serialize(final String topic, final Struct data) { + if (data == null) { + return inner.serialize(topic, null); + } + + final List fields = data.schema().fields(); + + SerdeUtils.throwOnColumnCountMismatch(numColumns, fields.size(), true, topic); + + final ArrayList values = new ArrayList<>(numColumns); + for (final Field field : fields) { + values.add(data.get(field)); + } + + return inner.serialize(topic, values); + } + + @Override + public void close() { + inner.close(); + } + } + + @VisibleForTesting + static class GenericKeyDeserializer implements Deserializer { + + private final Deserializer> inner; + private final ConnectSchema connectSchema; + + GenericKeyDeserializer(final Deserializer> inner, final ConnectSchema connectSchema) { + this.inner = requireNonNull(inner, "inner"); + this.connectSchema = requireNonNull(connectSchema, "connectSchema"); + } + + @Override + public void configure(final Map configs, final boolean isKey) { + inner.configure(configs, isKey); + } + + @Override + public void close() { + inner.close(); + } + + @Override + public Struct deserialize(final String topic, final byte[] data) { + final List values = inner.deserialize(topic, data); + if (values == null) { + return null; + } + + final List fields = connectSchema.fields(); + + SerdeUtils.throwOnColumnCountMismatch(fields.size(), values.size(), false, topic); + + final Struct row = new Struct(connectSchema); + + final Iterator fIt = fields.iterator(); + final Iterator vIt = values.iterator(); + while (fIt.hasNext()) { + row.put(fIt.next(), vIt.next()); + } + + return row; + } + } } diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericRowSerDe.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericRowSerDe.java index 103309aee27a..d63c3906f4e4 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericRowSerDe.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericRowSerDe.java @@ -23,23 +23,16 @@ import io.confluent.ksql.logging.processing.ProcessingLogContext; import io.confluent.ksql.schema.ksql.PersistenceSchema; import io.confluent.ksql.schema.ksql.SystemColumns; -import io.confluent.ksql.serde.connect.ConnectSchemas; import io.confluent.ksql.util.KsqlConfig; -import io.confluent.ksql.util.KsqlException; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.function.Supplier; -import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.connect.data.ConnectSchema; -import org.apache.kafka.connect.data.Field; -import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.errors.DataException; public final class GenericRowSerDe implements ValueSerdeFactory { @@ -100,7 +93,7 @@ public Serde create( final String loggerNamePrefix, final ProcessingLogContext processingLogContext ) { - final Serde formatSerde = + final Serde> formatSerde = innerFactory.createFormatSerde("Value", format, schema, ksqlConfig, srClientFactory); final Serde genericRowSerde = toGenericRowSerde(formatSerde, schema); @@ -114,16 +107,14 @@ public Serde create( } private static Serde toGenericRowSerde( - final Serde innerSerde, + final Serde> innerSerde, final PersistenceSchema schema ) { - final ConnectSchema connectSchema = ConnectSchemas.columnsToConnectSchema(schema.columns()); - final Serializer serializer = - new GenericRowSerializer(innerSerde.serializer(), connectSchema); + new GenericRowSerializer(innerSerde.serializer(), schema.columns().size()); final Deserializer deserializer = - new GenericRowDeserializer(innerSerde.deserializer(), connectSchema); + new GenericRowDeserializer(innerSerde.deserializer(), schema.columns().size()); return Serdes.serdeFrom(serializer, deserializer); } @@ -131,12 +122,12 @@ private static Serde toGenericRowSerde( @VisibleForTesting static class GenericRowSerializer implements Serializer { - private final Serializer inner; - private final ConnectSchema schema; + private final Serializer> inner; + private final int numColumns; - GenericRowSerializer(final Serializer inner, final ConnectSchema schema) { + GenericRowSerializer(final Serializer> inner, final int numColumns) { this.inner = requireNonNull(inner, "inner"); - this.schema = requireNonNull(schema, "schema"); + this.numColumns = numColumns; } @Override @@ -150,61 +141,26 @@ public byte[] serialize(final String topic, final GenericRow data) { return inner.serialize(topic, null); } - if (data.size() != schema.fields().size()) { - throw new SerializationException("Field count mismatch on serialization." - + " topic: " + topic - + ", expected: " + schema.fields().size() - + ", got: " + data.size() - ); - } + SerdeUtils.throwOnColumnCountMismatch(numColumns, data.size(), true, topic); - final Struct struct = new Struct(schema); - for (int i = 0; i < data.size(); i++) { - putField(struct, schema.fields().get(i), data.get(i)); - } - - return inner.serialize(topic, struct); + return inner.serialize(topic, data.values()); } @Override public void close() { inner.close(); } - - private static void putField(final Struct struct, final Field field, final Object value) { - try { - struct.put(field, value); - } catch (DataException e) { - // Add more info to error message in case of Struct to call out struct schemas - // with non-optional fields from incorrectly-written UDFs as a potential cause: - // https://github.com/confluentinc/ksql/issues/5364 - if (!(value instanceof Struct)) { - throw e; - } else { - throw new KsqlException( - "Failed to prepare Struct value field '" + field.name() + "' for serialization. " - + "This could happen if the value was produced by a user-defined function " - + "where the schema has non-optional return types. ksqlDB requires all " - + "schemas to be optional at all levels of the Struct: the Struct itself, " - + "schemas for all fields within the Struct, and so on.", - e); - } - } - } } @VisibleForTesting static class GenericRowDeserializer implements Deserializer { - private final Deserializer inner; + private final Deserializer> inner; private final int numColumns; - GenericRowDeserializer( - final Deserializer inner, - final ConnectSchema schema - ) { + GenericRowDeserializer(final Deserializer> inner, final int numColumns) { this.inner = requireNonNull(inner, "inner"); - this.numColumns = schema.fields().size(); + this.numColumns = numColumns; } @Override @@ -219,28 +175,15 @@ public void close() { @Override public GenericRow deserialize(final String topic, final byte[] data) { - final Struct struct = inner.deserialize(topic, data); - if (struct == null) { + final List values = inner.deserialize(topic, data); + if (values == null) { return null; } - final List fields = struct.schema().fields(); - - if (fields.size() != numColumns) { - throw new SerializationException("Field count mismatch on deserialization." - + " topic: " + topic - + ", expected: " + numColumns - + ", got: " + fields.size() - ); - } - - final GenericRow row = new GenericRow(fields.size() + ADDITIONAL_CAPACITY); - - for (final Field field : fields) { - final Object columnVal = struct.get(field); - row.append(columnVal); - } + SerdeUtils.throwOnColumnCountMismatch(numColumns, values.size(), false, topic); + final GenericRow row = new GenericRow(values.size() + ADDITIONAL_CAPACITY); + row.appendAll(values); return row; } } diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericSerdeFactory.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericSerdeFactory.java index 0f2e1ae864ef..1b5e9d3a09da 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericSerdeFactory.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/GenericSerdeFactory.java @@ -26,12 +26,12 @@ import io.confluent.ksql.logging.processing.ProcessingLogger; import io.confluent.ksql.schema.ksql.PersistenceSchema; import io.confluent.ksql.util.KsqlConfig; +import java.util.List; import java.util.Objects; import java.util.function.Function; import java.util.function.Supplier; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.connect.data.Struct; final class GenericSerdeFactory { @@ -49,7 +49,7 @@ final class GenericSerdeFactory { this.formatFactory = Objects.requireNonNull(formatFactory, "formatFactory"); } - Serde createFormatSerde( + Serde> createFormatSerde( final String target, final FormatInfo formatInfo, final PersistenceSchema schema, diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/KeyFormatUtils.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/KeyFormatUtils.java new file mode 100644 index 000000000000..c6d296aeaf42 --- /dev/null +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/KeyFormatUtils.java @@ -0,0 +1,46 @@ +/* + * Copyright 2020 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.serde; + +import com.google.common.collect.ImmutableList; +import io.confluent.ksql.util.KsqlConfig; +import java.util.List; + +public final class KeyFormatUtils { + + private static final List SUPPORTED_KEY_FORMATS = + ImmutableList.of(FormatFactory.KAFKA); + private static final List KEY_FORMATS_UNDER_DEVELOPMENT = + ImmutableList.of(FormatFactory.JSON); + + /** + * Until the primitive key work is complete, not all formats are supported as key formats. + * Once complete, this method may be removed. + * + * @return whether or not this format is supported for keys + */ + public static boolean isSupportedKeyFormat(final KsqlConfig config, final Format format) { + if (SUPPORTED_KEY_FORMATS.contains(format)) { + return true; + } + + return config.getBoolean(KsqlConfig.KSQL_KEY_FORMAT_ENABLED) + && KEY_FORMATS_UNDER_DEVELOPMENT.contains(format); + } + + private KeyFormatUtils() { + } +} diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/SerdeUtils.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/SerdeUtils.java index 55bb9bda35d2..b0dbb15694a4 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/SerdeUtils.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/SerdeUtils.java @@ -19,6 +19,7 @@ import io.confluent.ksql.schema.ksql.SchemaConverters; import io.confluent.ksql.serde.unwrapped.UnwrappedDeserializer; import io.confluent.ksql.serde.unwrapped.UnwrappedSerializer; +import java.util.List; import java.util.Set; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.serialization.Deserializer; @@ -26,7 +27,6 @@ import org.apache.kafka.connect.data.ConnectSchema; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.data.Struct; public final class SerdeUtils { @@ -75,20 +75,26 @@ public static Schema wrapSingle(final Schema fieldSchema) { .build(); } - public static Serializer unwrappedSerializer( - final ConnectSchema schema, + public static Serializer> unwrappedSerializer( final Serializer inner, final Class type ) { - return new UnwrappedSerializer<>(schema, inner, type); + return new UnwrappedSerializer<>(inner, type); } - public static Deserializer unwrappedDeserializer( - final ConnectSchema schema, - final Deserializer inner, - final Class type - ) { - return new UnwrappedDeserializer<>(schema, inner, type); + public static Deserializer> unwrappedDeserializer(final Deserializer inner) { + return new UnwrappedDeserializer(inner); + } + + @SuppressWarnings("unchecked") + public static T safeCast(final Object value, final Class javaType) { + if (value != null && !javaType.isAssignableFrom(value.getClass())) { + throw new SerializationException("value does not match expected type. " + + "expected: " + javaType.getSimpleName() + + ", but got: " + value.getClass().getSimpleName()); + } + + return (T) value; } public static void throwOnSchemaJavaTypeMismatch( @@ -102,6 +108,24 @@ public static void throwOnSchemaJavaTypeMismatch( } } + public static void throwOnColumnCountMismatch( + final int expectedCount, + final int actualCount, + final boolean serialization, + final String topic + ) { + if (expectedCount == actualCount) { + return; + } + + throw new SerializationException("Column count mismatch on " + + (serialization ? "serialization" : "deserialization") + + ". topic: " + topic + + ", expected: " + expectedCount + + ", got: " + actualCount + ); + } + @SuppressWarnings("unchecked") public static T castToTargetType(final Object val, final Class type) { if (val != null && !type.isAssignableFrom(val.getClass())) { diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/connect/ConnectFormat.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/connect/ConnectFormat.java index ef3e7b86ce8f..824c7e6d6342 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/connect/ConnectFormat.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/connect/ConnectFormat.java @@ -30,20 +30,25 @@ import io.confluent.ksql.serde.SerdeUtils; import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.KsqlException; +import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.connect.data.ConnectSchema; import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.Schema.Type; import org.apache.kafka.connect.data.Struct; -import org.jetbrains.annotations.NotNull; +import org.apache.kafka.connect.errors.DataException; /** * Base class for formats that internally leverage Connect's data model, i.e. it's {@link Schema} @@ -106,7 +111,7 @@ public ParsedSchema toParsedSchema( } @Override - public Serde getSerde( + public Serde> getSerde( final PersistenceSchema schema, final Map formatProps, final KsqlConfig config, @@ -121,21 +126,13 @@ public Serde getSerde( final Class targetType = SchemaConverters.connectToJavaTypeConverter() .toJavaType(innerSchema); - if (schema.features().enabled(SerdeFeature.UNWRAP_SINGLES)) { - return handleUnwrapping(innerSchema, outerSchema, formatProps, config, srFactory, targetType); - } - - if (!targetType.equals(Struct.class)) { - throw new IllegalArgumentException("Expected STRUCT, got " + targetType); - } - - return getConnectSerde(innerSchema, formatProps, config, srFactory, Struct.class); + return schema.features().enabled(SerdeFeature.UNWRAP_SINGLES) + ? handleUnwrapped(innerSchema, formatProps, config, srFactory, targetType) + : handleWrapped(innerSchema, formatProps, config, srFactory, targetType); } - @NotNull - private Serde handleUnwrapping( + private Serde> handleUnwrapped( final ConnectSchema innerSchema, - final ConnectSchema outerSchema, final Map formatProps, final KsqlConfig config, final Supplier srFactory, @@ -145,8 +142,28 @@ private Serde handleUnwrapping( getConnectSerde(innerSchema, formatProps, config, srFactory, targetType); return Serdes.serdeFrom( - SerdeUtils.unwrappedSerializer(outerSchema, innerSerde.serializer(), targetType), - SerdeUtils.unwrappedDeserializer(outerSchema, innerSerde.deserializer(), targetType) + SerdeUtils.unwrappedSerializer(innerSerde.serializer(), targetType), + SerdeUtils.unwrappedDeserializer(innerSerde.deserializer()) + ); + } + + private Serde> handleWrapped( + final ConnectSchema innerSchema, + final Map formatProps, + final KsqlConfig config, + final Supplier srFactory, + final Class targetType + ) { + if (!targetType.equals(Struct.class)) { + throw new IllegalArgumentException("Expected STRUCT, got " + targetType); + } + + final Serde connectSerde = + getConnectSerde(innerSchema, formatProps, config, srFactory, Struct.class); + + return Serdes.serdeFrom( + new ListToStructSerializer(connectSerde.serializer(), innerSchema), + new StructToListDeserializer(connectSerde.deserializer(), innerSchema.fields().size()) ); } @@ -192,4 +209,113 @@ public SqlType type() { return type; } } + + private static class ListToStructSerializer implements Serializer> { + + private final Serializer inner; + private final ConnectSchema structSchema; + + ListToStructSerializer( + final Serializer inner, + final ConnectSchema structSchema + ) { + this.inner = Objects.requireNonNull(inner, "inner"); + this.structSchema = Objects.requireNonNull(structSchema, "structSchema"); + } + + @Override + public void configure(final Map configs, final boolean isKey) { + inner.configure(configs, isKey); + } + + @Override + public byte[] serialize(final String topic, final List values) { + if (values == null) { + return null; + } + + final List fields = structSchema.fields(); + + SerdeUtils.throwOnColumnCountMismatch(fields.size(), values.size(), true, topic); + + final Struct struct = new Struct(structSchema); + + final Iterator fIt = fields.iterator(); + final Iterator vIt = values.iterator(); + + while (fIt.hasNext()) { + putField(struct, fIt.next(), vIt.next()); + } + + return inner.serialize(topic, struct); + } + + @Override + public void close() { + inner.close(); + } + + private static void putField(final Struct struct, final Field field, final Object value) { + try { + struct.put(field, value); + } catch (DataException e) { + // Add more info to error message in case of Struct to call out struct schemas + // with non-optional fields from incorrectly-written UDFs as a potential cause: + // https://github.com/confluentinc/ksql/issues/5364 + if (!(value instanceof Struct)) { + throw e; + } else { + throw new SerializationException( + "Failed to prepare Struct value field '" + field.name() + "' for serialization. " + + "This could happen if the value was produced by a user-defined function " + + "where the schema has non-optional return types. ksqlDB requires all " + + "schemas to be optional at all levels of the Struct: the Struct itself, " + + "schemas for all fields within the Struct, and so on.", + e); + } + } + } + } + + private static class StructToListDeserializer implements Deserializer> { + + private final Deserializer inner; + private final int numColumns; + + StructToListDeserializer(final Deserializer deserializer, final int numColumns) { + this.inner = Objects.requireNonNull(deserializer, "deserializer"); + this.numColumns = numColumns; + } + + @Override + public void configure(final Map configs, final boolean isKey) { + inner.configure(configs, isKey); + } + + @Override + public List deserialize(final String topic, final byte[] bytes) { + if (bytes == null) { + return null; + } + + final Struct struct = inner.deserialize(topic, bytes); + + final List fields = struct.schema().fields(); + + SerdeUtils.throwOnColumnCountMismatch(numColumns, fields.size(), false, topic); + + final List values = new ArrayList<>(numColumns); + + for (final Field field : fields) { + values.add(struct.get(field)); + } + + return values; + } + + @Override + public void close() { + inner.close(); + } + } } diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/delimited/DelimitedFormat.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/delimited/DelimitedFormat.java index 60186bceb05b..40ab7cbaa67b 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/delimited/DelimitedFormat.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/delimited/DelimitedFormat.java @@ -23,11 +23,13 @@ import io.confluent.ksql.serde.SerdeFeature; import io.confluent.ksql.serde.SerdeUtils; import io.confluent.ksql.util.KsqlConfig; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.Supplier; +import org.apache.commons.csv.CSVFormat; import org.apache.kafka.common.serialization.Serde; -import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.common.serialization.Serdes; public final class DelimitedFormat implements Format { @@ -56,7 +58,7 @@ public Set getSupportedProperties() { } @Override - public Serde getSerde( + public Serde> getSerde( final PersistenceSchema schema, final Map formatProperties, final KsqlConfig ksqlConfig, @@ -66,8 +68,12 @@ public Serde getSerde( final Delimiter delimiter = getDelimiter(formatProperties); - return new KsqlDelimitedSerdeFactory(delimiter) - .createSerde(schema, ksqlConfig, srClientFactory); + final CSVFormat csvFormat = CSVFormat.DEFAULT.withDelimiter(delimiter.getDelimiter()); + + return Serdes.serdeFrom( + new KsqlDelimitedSerializer(schema, csvFormat), + new KsqlDelimitedDeserializer(schema, csvFormat) + ); } private static Delimiter getDelimiter(final Map formatProperties) { diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializer.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializer.java index ce79459fe557..da6151599b34 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializer.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializer.java @@ -17,56 +17,66 @@ import com.google.common.collect.ImmutableMap; import io.confluent.ksql.schema.ksql.PersistenceSchema; -import io.confluent.ksql.serde.connect.ConnectSchemas; +import io.confluent.ksql.schema.ksql.SimpleColumn; +import io.confluent.ksql.schema.ksql.types.SqlBaseType; +import io.confluent.ksql.schema.ksql.types.SqlDecimal; +import io.confluent.ksql.schema.ksql.types.SqlType; +import io.confluent.ksql.serde.FormatFactory; +import io.confluent.ksql.serde.SerdeUtils; import io.confluent.ksql.util.DecimalUtil; import io.confluent.ksql.util.KsqlException; import java.math.BigDecimal; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.function.Function; import org.apache.commons.csv.CSVFormat; import org.apache.commons.csv.CSVParser; import org.apache.commons.csv.CSVRecord; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.connect.data.ConnectSchema; -import org.apache.kafka.connect.data.Field; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.Schema.Type; -import org.apache.kafka.connect.data.Struct; - -public class KsqlDelimitedDeserializer implements Deserializer { - - private static final Map> PARSERS = ImmutableMap.of( - Type.BOOLEAN, Boolean::parseBoolean, - Type.INT32, Integer::parseInt, - Type.INT64, Long::parseLong, - Type.FLOAT64, Double::parseDouble, - Type.STRING, s -> s - ); - - private final ConnectSchema schema; + +class KsqlDelimitedDeserializer implements Deserializer> { + + private interface Parser { + + Object parse(String value); + } + + private interface ParserFactory { + + Parser build(SqlType type); + } + + private static final Map PARSERS = ImmutableMap + .builder() + .put(SqlBaseType.BOOLEAN, t -> Boolean::parseBoolean) + .put(SqlBaseType.INTEGER, t -> Integer::parseInt) + .put(SqlBaseType.BIGINT, t -> Long::parseLong) + .put(SqlBaseType.DOUBLE, t -> Double::parseDouble) + .put(SqlBaseType.STRING, t -> v -> v) + .put(SqlBaseType.DECIMAL, KsqlDelimitedDeserializer::decimalParser) + .build(); + private final CSVFormat csvFormat; + private final List parsers; KsqlDelimitedDeserializer( final PersistenceSchema schema, final CSVFormat csvFormat ) { - this.schema = ConnectSchemas - .columnsToConnectSchema(Objects.requireNonNull(schema, "schema").columns()); - throwOnUnsupported(this.schema); this.csvFormat = Objects.requireNonNull(csvFormat, "csvFormat"); + this.parsers = buildParsers(schema); } @Override - public void configure(final Map map, final boolean b) { + public void configure(final Map configs, final boolean isKey) { } @Override - public Struct deserialize(final String topic, final byte[] bytes) { + public List deserialize(final String topic, final byte[] bytes) { if (bytes == null) { return null; } @@ -77,41 +87,33 @@ public Struct deserialize(final String topic, final byte[] bytes) { .getRecords(); if (csvRecords.isEmpty()) { - throw new KsqlException("No fields in record"); + throw new SerializationException("No fields in record"); } final CSVRecord csvRecord = csvRecords.get(0); if (csvRecord == null || csvRecord.size() == 0) { - throw new KsqlException("No fields in record."); - } - - if (csvRecord.size() != schema.fields().size()) { - throw new KsqlException( - String.format( - "Unexpected field count, csvFields:%d schemaFields:%d", - csvRecord.size(), - schema.fields().size() - ) - ); + throw new SerializationException("No fields in record."); } - final Struct struct = new Struct(schema); + SerdeUtils.throwOnColumnCountMismatch(parsers.size(), csvRecord.size(), false, topic); - final Iterator it = schema.fields().iterator(); + final List values = new ArrayList<>(parsers.size()); + final Iterator pIt = parsers.iterator(); for (int i = 0; i < csvRecord.size(); i++) { - final Field field = it.next(); - if (csvRecord.get(i) == null) { - struct.put(field, null); - } else { - final Object coerced = enforceFieldType(field.schema(), csvRecord.get(i)); - struct.put(field, coerced); - } + final String value = csvRecord.get(i); + final Parser parser = pIt.next(); + + final Object parsed = value == null || value.isEmpty() + ? null + : parser.parse(value); + values.add(parsed); } - return struct; + + return values; } catch (final Exception e) { - throw new SerializationException("Error deserializing delimited row", e); + throw new SerializationException("Error deserializing delimited", e); } } @@ -119,37 +121,23 @@ public Struct deserialize(final String topic, final byte[] bytes) { public void close() { } - private static Object enforceFieldType( - final Schema fieldSchema, - final String delimitedField - ) { - if (delimitedField.isEmpty()) { - return null; - } - - if (DecimalUtil.isDecimal(fieldSchema)) { - return DecimalUtil.ensureFit(new BigDecimal(delimitedField),fieldSchema); - } - - final Function parser = PARSERS.get(fieldSchema.type()); - if (parser == null) { - throw new KsqlException("Type is not supported: " + fieldSchema.type()); - } - - return parser.apply(delimitedField); + private static Parser decimalParser(final SqlType sqlType) { + final SqlDecimal decimalType = (SqlDecimal) sqlType; + return v -> DecimalUtil.ensureFit(new BigDecimal(v), decimalType); } - private static void throwOnUnsupported(final Schema schema) { - if (schema.type() != Type.STRUCT) { - throw new IllegalArgumentException("DELIMITED expects all top level schemas to be STRUCTs"); - } - - schema.fields().forEach(field -> { - final Type type = field.schema().type(); - if (!PARSERS.containsKey(type) && !DecimalUtil.isDecimal(field.schema())) { - throw new UnsupportedOperationException( - "DELIMITED does not support type: " + type + ", field: " + field.name()); + private static List buildParsers(final PersistenceSchema schema) { + final List parsers = new ArrayList<>(schema.columns().size()); + for (final SimpleColumn column : schema.columns()) { + final SqlBaseType baseType = column.type().baseType(); + final ParserFactory parserFactory = PARSERS.get(baseType); + if (parserFactory == null) { + throw new KsqlException("The '" + FormatFactory.DELIMITED.name() + + "' format does not support type '" + baseType + "', column: " + column.name()); } - }); + + parsers.add(parserFactory.build(column.type())); + } + return parsers; } } diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java deleted file mode 100644 index c1f135209395..000000000000 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactory.java +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Copyright 2018 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.delimited; - -import com.google.errorprone.annotations.Immutable; -import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; -import io.confluent.ksql.schema.ksql.PersistenceSchema; -import io.confluent.ksql.schema.ksql.SqlTypeWalker; -import io.confluent.ksql.schema.ksql.types.SqlDecimal; -import io.confluent.ksql.schema.ksql.types.SqlPrimitiveType; -import io.confluent.ksql.schema.ksql.types.SqlType; -import io.confluent.ksql.serde.Delimiter; -import io.confluent.ksql.serde.FormatFactory; -import io.confluent.ksql.testing.EffectivelyImmutable; -import io.confluent.ksql.util.KsqlConfig; -import io.confluent.ksql.util.KsqlException; -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.Struct; - - -@Immutable -class KsqlDelimitedSerdeFactory { - - @EffectivelyImmutable - private final CSVFormat csvFormat; - - KsqlDelimitedSerdeFactory(final Delimiter delimiter) { - this.csvFormat = CSVFormat.DEFAULT.withDelimiter(delimiter.getDelimiter()); - } - - public Serde createSerde( - final PersistenceSchema schema, - final KsqlConfig ksqlConfig, - final Supplier schemaRegistryClientFactory - ) { - validate(schema); - - return Serdes.serdeFrom( - new KsqlDelimitedSerializer(csvFormat), - new KsqlDelimitedDeserializer(schema, csvFormat) - ); - } - - private static void validate(final PersistenceSchema schema) { - schema.columns() - .forEach(c -> SqlTypeWalker.visit(c.type(), new SchemaValidator())); - } - - private static class SchemaValidator implements SqlTypeWalker.Visitor { - - @Override - public Void visitPrimitive(final SqlPrimitiveType type) { - // Primitive types are allowed. - return null; - } - - @Override - public Void visitDecimal(final SqlDecimal type) { - // Decimal types are allowed. - return null; - } - - @Override - public Void visitType(final SqlType schema) { - throw new KsqlException("The '" + FormatFactory.DELIMITED.name() - + "' format does not support type '" + schema.baseType() + "'"); - } - } -} diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializer.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializer.java index 744a8c021795..42d113b4b50c 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializer.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializer.java @@ -15,29 +15,35 @@ package io.confluent.ksql.serde.delimited; +import io.confluent.ksql.schema.ksql.PersistenceSchema; +import io.confluent.ksql.schema.ksql.SimpleColumn; +import io.confluent.ksql.schema.ksql.types.SqlBaseType; +import io.confluent.ksql.schema.ksql.types.SqlDecimal; import io.confluent.ksql.util.DecimalUtil; -import io.confluent.ksql.util.KsqlException; import java.io.StringWriter; import java.math.BigDecimal; import java.nio.charset.StandardCharsets; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.Objects; import org.apache.commons.csv.CSVFormat; import org.apache.commons.csv.CSVPrinter; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.connect.data.Field; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.Struct; -public class KsqlDelimitedSerializer implements Serializer { +class KsqlDelimitedSerializer implements Serializer> { + private final PersistenceSchema schema; private final CSVFormat csvFormat; - public KsqlDelimitedSerializer(final CSVFormat csvFormat) { - this.csvFormat = csvFormat; + KsqlDelimitedSerializer( + final PersistenceSchema schema, + final CSVFormat csvFormat + ) { + this.schema = Objects.requireNonNull(schema, "schema"); + this.csvFormat = Objects.requireNonNull(csvFormat, "csvFormat"); } @Override @@ -45,7 +51,7 @@ public void configure(final Map map, final boolean b) { } @Override - public byte[] serialize(final String topic, final Struct data) { + public byte[] serialize(final String topic, final List data) { if (data == null) { return null; } @@ -53,7 +59,7 @@ public byte[] serialize(final String topic, final Struct data) { try { final StringWriter stringWriter = new StringWriter(); final CSVPrinter csvPrinter = new CSVPrinter(stringWriter, csvFormat); - csvPrinter.printRecord(() -> new FieldIterator(data)); + csvPrinter.printRecord(() -> new FieldIterator(data, schema)); final String result = stringWriter.toString(); return result.substring(0, result.length() - 2).getBytes(StandardCharsets.UTF_8); } catch (final Exception e) { @@ -67,46 +73,31 @@ public void close() { private static class FieldIterator implements Iterator { - private final Struct data; - private final Iterator fieldIt; + private final Iterator dataIt; + private final Iterator columnIt; - FieldIterator(final Struct data) { - this.data = Objects.requireNonNull(data, "data"); - this.fieldIt = data.schema().fields().iterator(); + FieldIterator(final List data, final PersistenceSchema schema) { + this.dataIt = data.iterator(); + this.columnIt = schema.columns().iterator(); } @Override public boolean hasNext() { - return fieldIt.hasNext(); + return columnIt.hasNext(); } @Override public Object next() { - final Field field = fieldIt.next(); - throwOnUnsupportedType(field.schema()); - if (DecimalUtil.isDecimal(field.schema())) { - return getDecimal(field); - } - return data.get(field); - } - - private String getDecimal(final Field field) { - final BigDecimal value = (BigDecimal) data.get(field); - final int precision = DecimalUtil.precision(field.schema()); - final int scale = DecimalUtil.scale(field.schema()); + final Object value = dataIt.next(); + final SimpleColumn column = columnIt.next(); - return DecimalUtil.format(precision, scale, value); + return column.type().baseType().equals(SqlBaseType.DECIMAL) + ? handleDecimal((BigDecimal) value, (SqlDecimal) column.type()) + : value; } - private static void throwOnUnsupportedType(final Schema schema) { - switch (schema.type()) { - case ARRAY: - case MAP: - case STRUCT: - throw new KsqlException("DELIMITED does not support type: " + schema.type()); - - default: - } + private static String handleDecimal(final BigDecimal value, final SqlDecimal sqlType) { + return DecimalUtil.format(sqlType.getPrecision(), sqlType.getScale(), value); } } } diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/json/JsonFormat.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/json/JsonFormat.java index ad53a00d8d3c..09479d3ed3e2 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/json/JsonFormat.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/json/JsonFormat.java @@ -79,4 +79,5 @@ protected ParsedSchema fromConnectSchema(final Schema schema, final FormatInfo formatInfo) { return jsonData.fromConnectSchema(schema); } + } diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/kafka/KafkaFormat.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/kafka/KafkaFormat.java index 0c1dda74f64e..228489d816aa 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/kafka/KafkaFormat.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/kafka/KafkaFormat.java @@ -22,11 +22,11 @@ import io.confluent.ksql.serde.SerdeFeature; import io.confluent.ksql.serde.SerdeUtils; import io.confluent.ksql.util.KsqlConfig; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.Supplier; import org.apache.kafka.common.serialization.Serde; -import org.apache.kafka.connect.data.Struct; public class KafkaFormat implements Format { @@ -47,7 +47,7 @@ public Set supportedFeatures() { } @Override - public Serde getSerde( + public Serde> getSerde( final PersistenceSchema schema, final Map formatProperties, final KsqlConfig ksqlConfig, @@ -57,4 +57,5 @@ public Serde getSerde( return KafkaSerdeFactory.createSerde(schema); } + } diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/kafka/KafkaSerdeFactory.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/kafka/KafkaSerdeFactory.java index bbc98e570bd0..b63c5d28e26c 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/kafka/KafkaSerdeFactory.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/kafka/KafkaSerdeFactory.java @@ -18,131 +18,125 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableMap; import io.confluent.ksql.schema.ksql.PersistenceSchema; +import io.confluent.ksql.schema.ksql.SchemaConverters; import io.confluent.ksql.schema.ksql.SimpleColumn; -import io.confluent.ksql.schema.ksql.types.SqlType; -import io.confluent.ksql.schema.ksql.types.SqlTypes; +import io.confluent.ksql.schema.ksql.types.SqlBaseType; import io.confluent.ksql.serde.FormatFactory; -import io.confluent.ksql.serde.connect.ConnectSchemas; +import io.confluent.ksql.serde.SerdeUtils; import io.confluent.ksql.serde.voids.KsqlVoidSerde; import io.confluent.ksql.util.KsqlException; +import java.util.Collections; import java.util.List; import java.util.Objects; -import java.util.Optional; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.connect.data.ConnectSchema; -import org.apache.kafka.connect.data.Field; -import org.apache.kafka.connect.data.Struct; public final class KafkaSerdeFactory { // Note: If supporting new types here, add new type of PRINT TOPIC support too - private static final ImmutableMap> SERDE = ImmutableMap.of( - SqlTypes.INTEGER, Serdes.Integer(), - SqlTypes.BIGINT, Serdes.Long(), - SqlTypes.DOUBLE, Serdes.Double(), - SqlTypes.STRING, Serdes.String() + private static final ImmutableMap, Serde> SERDE = ImmutableMap.of( + Integer.class, Serdes.Integer(), + Long.class, Serdes.Long(), + Double.class, Serdes.Double(), + String.class, Serdes.String() ); private KafkaSerdeFactory() { } - static Serde createSerde(final PersistenceSchema schema) { - final Serde primitiveSerde = getPrimitiveSerde(schema); + static Serde> createSerde(final PersistenceSchema schema) { + final List columns = schema.columns(); + if (columns.isEmpty()) { + // No columns: + return new KsqlVoidSerde<>(); + } - final ConnectSchema connectSchema = ConnectSchemas.columnsToConnectSchema(schema.columns()); + if (columns.size() != 1) { + throw new KsqlException("The '" + FormatFactory.KAFKA.name() + + "' format only supports a single field. Got: " + columns); + } - final Serializer serializer = new RowSerializer( + final SimpleColumn singleColumn = columns.get(0); + + final Class javaType = SchemaConverters.sqlToJavaConverter() + .toJavaType(singleColumn.type()); + + return createSerde(singleColumn, javaType); + } + + private static Serde> createSerde( + final SimpleColumn singleColumn, + final Class javaType + ) { + final Serde primitiveSerde = getPrimitiveSerde(singleColumn.type().baseType(), javaType); + + final Serializer> serializer = new RowSerializer<>( primitiveSerde.serializer(), - connectSchema + javaType ); - final Deserializer deserializer = new RowDeserializer( - primitiveSerde.deserializer(), - connectSchema + final Deserializer> deserializer = new RowDeserializer( + primitiveSerde.deserializer() ); return Serdes.serdeFrom(serializer, deserializer); } - @VisibleForTesting @SuppressWarnings({"unchecked", "rawtypes"}) - public static Serde getPrimitiveSerde(final PersistenceSchema schema) { - - final List columns = schema.columns(); - if (columns.isEmpty()) { - // No columns: - return (Serde) new KsqlVoidSerde(); - } - - if (columns.size() != 1) { - throw new KsqlException("The '" + FormatFactory.KAFKA.name() - + "' format only supports a single field. Got: " + schema.columns()); - } - - final SqlType type = columns.get(0).type(); - final Serde serde = SERDE.get(type); + @VisibleForTesting + public static Serde getPrimitiveSerde(final SqlBaseType sqlType, final Class javaType) { + final Serde serde = SERDE.get(javaType); if (serde == null) { throw new KsqlException("The '" + FormatFactory.KAFKA.name() - + "' format does not support type '" + type.baseType() + "'"); + + "' format does not support type '" + sqlType + "'"); } return (Serde) serde; } - private static final class RowSerializer implements Serializer { + private static final class RowSerializer implements Serializer> { - private final Serializer delegate; - private final Optional field; + private final Serializer delegate; + private final Class javaType; - RowSerializer(final Serializer delegate, final ConnectSchema schema) { + RowSerializer(final Serializer delegate, final Class javaType) { this.delegate = Objects.requireNonNull(delegate, "delegate"); - this.field = schema.fields().isEmpty() - ? Optional.empty() - : Optional.of(schema.fields().get(0)); + this.javaType = Objects.requireNonNull(javaType, "javaType"); } @Override - public byte[] serialize(final String topic, final Struct struct) { - final Object value = struct == null || !field.isPresent() - ? null - : struct.get(field.get()); + public byte[] serialize(final String topic, final List values) { + if (values == null) { + return null; + } + + SerdeUtils.throwOnColumnCountMismatch(1, values.size(), true, topic); + final T value = SerdeUtils.safeCast(values.get(0), javaType); return delegate.serialize(topic, value); } } - private static final class RowDeserializer implements Deserializer { + private static final class RowDeserializer implements Deserializer> { - private final Deserializer delegate; - private final ConnectSchema schema; - private final Optional field; + private final Deserializer delegate; - RowDeserializer( - final Deserializer delegate, - final ConnectSchema schema - ) { + RowDeserializer(final Deserializer delegate) { this.delegate = Objects.requireNonNull(delegate, "delegate"); - this.schema = Objects.requireNonNull(schema, "schema"); - this.field = schema.fields().isEmpty() - ? Optional.empty() - : Optional.of(schema.fields().get(0)); } @Override - public Struct deserialize(final String topic, final byte[] bytes) { + public List deserialize(final String topic, final byte[] bytes) { try { final Object primitive = delegate.deserialize(topic, bytes); if (primitive == null) { return null; } - final Struct struct = new Struct(schema); - struct.put(field.orElseThrow(IllegalStateException::new), primitive); - return struct; + return Collections.singletonList(primitive); } catch (final Exception e) { throw new SerializationException( "Error deserializing KAFKA message from topic: " + topic, e); diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/unwrapped/UnwrappedDeserializer.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/unwrapped/UnwrappedDeserializer.java index 6f97db71162e..996b6c80c33b 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/unwrapped/UnwrappedDeserializer.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/unwrapped/UnwrappedDeserializer.java @@ -17,30 +17,20 @@ import static java.util.Objects.requireNonNull; -import io.confluent.ksql.serde.SerdeUtils; +import java.util.Collections; +import java.util.List; import java.util.Map; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.connect.data.ConnectSchema; -import org.apache.kafka.connect.data.Field; -import org.apache.kafka.connect.data.Struct; -public class UnwrappedDeserializer implements Deserializer { +public class UnwrappedDeserializer implements Deserializer> { - private final Field field; - private final ConnectSchema schema; - private final Deserializer inner; + private final Deserializer inner; public UnwrappedDeserializer( - final ConnectSchema schema, - final Deserializer inner, - final Class colType + final Deserializer inner ) { - this.schema = requireNonNull(schema, "schema"); this.inner = requireNonNull(inner, "inner"); - this.field = Unwrapped.getOnlyField(schema); - - SerdeUtils.throwOnSchemaJavaTypeMismatch(field.schema(), colType); } @Override @@ -49,32 +39,28 @@ public void configure(final Map configs, final boolean isKey) { } @Override - public Struct deserialize(final String topic, final byte[] bytes) { + public List deserialize(final String topic, final byte[] bytes) { if (bytes == null) { return null; } - final T single = inner.deserialize(topic, bytes); - return wrapSingle(single); + final Object single = inner.deserialize(topic, bytes); + return Collections.singletonList(single); } @Override - public Struct deserialize(final String topic, final Headers headers, final byte[] bytes) { + public List deserialize(final String topic, final Headers headers, final byte[] bytes) { if (bytes == null) { return null; } final Object single = inner.deserialize(topic, headers, bytes); - return wrapSingle(single); + return Collections.singletonList(single); } @Override public void close() { inner.close(); } - - private Struct wrapSingle(final Object single) { - return new Struct(schema).put(field, single); - } } diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/unwrapped/UnwrappedSerializer.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/unwrapped/UnwrappedSerializer.java index d113b6674bca..527df5e64fef 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/unwrapped/UnwrappedSerializer.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/unwrapped/UnwrappedSerializer.java @@ -18,31 +18,27 @@ import static java.util.Objects.requireNonNull; import io.confluent.ksql.serde.SerdeUtils; +import java.util.List; import java.util.Map; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.connect.data.ConnectSchema; -import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.Struct; /** * Serializer that extracts the single column within a {@link Struct} and passes this to an inner * serializer. */ -public class UnwrappedSerializer implements Serializer { +public class UnwrappedSerializer implements Serializer> { private final Serializer inner; - private final Field field; + private final Class colType; public UnwrappedSerializer( - final ConnectSchema schema, final Serializer inner, final Class colType ) { this.inner = requireNonNull(inner, "inner"); - this.field = Unwrapped.getOnlyField(schema); - - SerdeUtils.throwOnSchemaJavaTypeMismatch(field.schema(), colType); + this.colType = requireNonNull(colType, "colType"); } @Override @@ -51,22 +47,22 @@ public void configure(final Map configs, final boolean isKey) { } @Override - public byte[] serialize(final String topic, final Struct struct) { - if (struct == null) { + public byte[] serialize(final String topic, final List values) { + if (values == null) { return null; } - final T single = extractOnlyColumn(struct); + final T single = extractOnlyColumn(values, topic); return inner.serialize(topic, single); } @Override - public byte[] serialize(final String topic, final Headers headers, final Struct struct) { + public byte[] serialize(final String topic, final Headers headers, final List struct) { if (struct == null) { return null; } - final T single = extractOnlyColumn(struct); + final T single = extractOnlyColumn(struct, topic); return inner.serialize(topic, headers, single); } @@ -75,16 +71,11 @@ public void close() { inner.close(); } - private T extractOnlyColumn(final Struct struct) { - final Object val = struct.get(field); - return castColValue(val); - } + private T extractOnlyColumn(final List values, final String topic) { + SerdeUtils.throwOnColumnCountMismatch(1, values.size(), true, topic); - @SuppressWarnings("unchecked") - private T castColValue(final Object val) { - // Cast is safe as constructor has confirmed the Java type of the field matches T. - // And ksqlDB ensures only struct's with correct schema are passed. - return (T) val; + final Object val = values.get(0); + return SerdeUtils.safeCast(val, colType); } } diff --git a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/voids/KsqlVoidSerde.java b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/voids/KsqlVoidSerde.java index 8c602546cd89..f7e7df84d0d7 100644 --- a/ksqldb-serde/src/main/java/io/confluent/ksql/serde/voids/KsqlVoidSerde.java +++ b/ksqldb-serde/src/main/java/io/confluent/ksql/serde/voids/KsqlVoidSerde.java @@ -18,7 +18,7 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serdes.WrapperSerde; -import org.apache.kafka.common.serialization.VoidSerializer; +import org.apache.kafka.common.serialization.Serializer; /** * Serde for handling voids. @@ -27,16 +27,24 @@ * throw if it encounters non-null data to deserialize. This means the deserializer can be used * where the source record contains a key or value, but the user does not want to deserialize it. */ -public final class KsqlVoidSerde extends WrapperSerde { +public final class KsqlVoidSerde extends WrapperSerde { public KsqlVoidSerde() { - super(new VoidSerializer(), new LaxVoidDeserializer()); + super(new LaxVoidSerializer<>(), new LaxVoidDeserializer<>()); } - public static final class LaxVoidDeserializer implements Deserializer { + public static final class LaxVoidSerializer implements Serializer { @Override - public Void deserialize(final String topic, final byte[] data) { + public byte[] serialize(final String s, final T t) { + return null; + } + } + + public static final class LaxVoidDeserializer implements Deserializer { + + @Override + public T deserialize(final String topic, final byte[] data) { return null; } } diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/logging/processing/LoggingDeserializerTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/logging/processing/LoggingDeserializerTest.java index dcae437e5eec..f67cf71784b1 100644 --- a/ksqldb-serde/src/test/java/io/confluent/ksql/logging/processing/LoggingDeserializerTest.java +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/logging/processing/LoggingDeserializerTest.java @@ -17,7 +17,7 @@ import static io.confluent.ksql.GenericRow.genericRow; import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.*; import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.any; @@ -35,6 +35,7 @@ import java.util.function.Function; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.connect.data.SchemaAndValue; +import org.hamcrest.Matchers; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/schema/ksql/PhysicalSchemaTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/schema/ksql/PhysicalSchemaTest.java index b618b545d3a1..13c3c9f023b2 100644 --- a/ksqldb-serde/src/test/java/io/confluent/ksql/schema/ksql/PhysicalSchemaTest.java +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/schema/ksql/PhysicalSchemaTest.java @@ -73,12 +73,12 @@ public void shouldImplementEquals() { public void shouldReturnKeySchema() { // When: final PhysicalSchema result = PhysicalSchema - .from(SCHEMA, SerdeOptions.of()); + .from(SCHEMA, SerdeOptions.of(SerdeOption.UNWRAP_SINGLE_KEYS)); // Then: assertThat(result.keySchema(), is(PersistenceSchema.from( SCHEMA.key(), - EnabledSerdeFeatures.of() + EnabledSerdeFeatures.of(SerdeFeature.UNWRAP_SINGLES) ))); } diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/FormatFactoryTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/FormatFactoryTest.java index 14bb8254b78d..8c213bcd4841 100644 --- a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/FormatFactoryTest.java +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/FormatFactoryTest.java @@ -93,4 +93,13 @@ public void shouldThrowWhenAttemptingToUseValueDelimiterWithJsonFormat() { } + @Test + public void shouldCreateFromNameWithCaseInsensitivity() { + // When: + final Format format = FormatFactory.fromName("aVrO"); + + // Then: + assertThat(format, is(FormatFactory.AVRO)); + } + } \ No newline at end of file diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/GenericKeySerDeTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/GenericKeySerDeTest.java index c52f54cbce38..10853cbdf22a 100644 --- a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/GenericKeySerDeTest.java +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/GenericKeySerDeTest.java @@ -20,6 +20,7 @@ import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.sameInstance; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -30,9 +31,12 @@ import io.confluent.ksql.schema.ksql.PersistenceSchema; import io.confluent.ksql.util.KsqlConfig; import java.time.Duration; +import java.util.List; import java.util.Optional; import java.util.function.Supplier; +import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.WindowedSerdes.SessionWindowedSerde; @@ -65,7 +69,11 @@ public class GenericKeySerDeTest { @Mock private ProcessingLogContext processingLogCxt; @Mock - private Serde innerSerde; + private Serde> innerSerde; + @Mock + private Serializer> innerSerializer; + @Mock + private Deserializer> innerDeserializer; @Mock private Serde loggingSerde; @@ -77,6 +85,9 @@ public void setUp() { when(innerFactory.createFormatSerde(any(), any(), any(), any(), any())).thenReturn(innerSerde); when(innerFactory.wrapInLoggingSerde(any(), any(), any())).thenReturn(loggingSerde); + + when(innerSerde.serializer()).thenReturn(innerSerializer); + when(innerSerde.deserializer()).thenReturn(innerDeserializer); } @Test @@ -104,7 +115,7 @@ public void shouldWrapInLoggingSerdeNonWindowed() { factory.create(format, schema, config, srClientFactory, LOGGER_PREFIX, processingLogCxt); // Then: - verify(innerFactory).wrapInLoggingSerde(innerSerde, LOGGER_PREFIX, processingLogCxt); + verify(innerFactory).wrapInLoggingSerde(any(), eq(LOGGER_PREFIX), eq(processingLogCxt)); } @Test @@ -114,7 +125,7 @@ public void shouldWrapInLoggingSerdeWindowed() { .create(format, TIMED_WND, schema, config, srClientFactory, LOGGER_PREFIX, processingLogCxt); // Then: - verify(innerFactory).wrapInLoggingSerde(innerSerde, LOGGER_PREFIX, processingLogCxt); + verify(innerFactory).wrapInLoggingSerde(any(), eq(LOGGER_PREFIX), eq(processingLogCxt)); } @Test diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/GenericRowSerDeTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/GenericRowSerDeTest.java index 64288d91b299..57bbc77695c9 100644 --- a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/GenericRowSerDeTest.java +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/GenericRowSerDeTest.java @@ -35,20 +35,14 @@ import io.confluent.ksql.serde.GenericRowSerDe.GenericRowDeserializer; import io.confluent.ksql.serde.GenericRowSerDe.GenericRowSerializer; import io.confluent.ksql.util.KsqlConfig; -import io.confluent.ksql.util.KsqlException; import java.nio.charset.StandardCharsets; +import java.util.List; import java.util.Map; import java.util.function.Supplier; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.connect.data.ConnectSchema; -import org.apache.kafka.connect.data.Field; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.connect.errors.DataException; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -64,11 +58,6 @@ public class GenericRowSerDeTest { private static final Map SOME_CONFIG = ImmutableMap.of("some", "thing"); private static final byte[] SERIALIZED = "serialized".getBytes(StandardCharsets.UTF_8); - private static final ConnectSchema CONNECT_SCHEMA = (ConnectSchema) SchemaBuilder.struct() - .field("vic", Schema.OPTIONAL_STRING_SCHEMA) - .field("bob", Schema.OPTIONAL_INT32_SCHEMA) - .build(); - @Mock private GenericSerdeFactory innerFactory; @Mock @@ -82,15 +71,11 @@ public class GenericRowSerDeTest { @Mock private ProcessingLogContext processingLogCxt; @Mock - private Serde innerSerde; - @Mock - private Serializer innerSerializer; - @Mock - private Deserializer innerDeserializer; + private Serde> innerSerde; @Mock - private Struct deserialized; + private Serializer> innerSerializer; @Mock - private Schema deserializedSchema; + private Deserializer> innerDeserializer; @Mock private Serde loggingSerde; @Captor @@ -99,22 +84,19 @@ public class GenericRowSerDeTest { private ValueSerdeFactory factory; private GenericRowSerializer serializer; private GenericRowDeserializer deserializer; - private final Struct struct = new Struct(CONNECT_SCHEMA); @Before public void setUp() { factory = new GenericRowSerDe(innerFactory); - serializer = new GenericRowSerializer(innerSerializer, CONNECT_SCHEMA); - deserializer = new GenericRowDeserializer(innerDeserializer, CONNECT_SCHEMA); + serializer = new GenericRowSerializer(innerSerializer, 2); + deserializer = new GenericRowDeserializer(innerDeserializer, 2); when(innerFactory.createFormatSerde(any(), any(), any(), any(), any())).thenReturn(innerSerde); when(innerFactory.wrapInLoggingSerde(any(), any(), any())).thenReturn(loggingSerde); when(innerSerde.serializer()).thenReturn(innerSerializer); when(innerSerde.deserializer()).thenReturn(innerDeserializer); when(innerSerializer.serialize(any(), any())).thenReturn(SERIALIZED); - when(innerDeserializer.deserialize(any(), any())).thenReturn(deserialized); - when(deserialized.schema()).thenReturn(deserializedSchema); } @Test @@ -238,7 +220,7 @@ public void shouldThrowOnSerializeOnColumnCountMismatch() { ); // Then: - assertThat(e.getMessage(), is("Field count mismatch on serialization." + assertThat(e.getMessage(), is("Column count mismatch on serialization." + " topic: topicName" + ", expected: 2" + ", got: 3" @@ -248,11 +230,7 @@ public void shouldThrowOnSerializeOnColumnCountMismatch() { @Test public void shouldThrowOnDeserializeOnColumnCountMismatch() { // Given: - when(deserializedSchema.fields()).thenReturn(ImmutableList.of( - new Field("too", 0, Schema.OPTIONAL_INT64_SCHEMA), - new Field("many", 0, Schema.OPTIONAL_INT64_SCHEMA), - new Field("columns", 0, Schema.OPTIONAL_INT64_SCHEMA) - )); + givenInnerDeserializerReturns(ImmutableList.of("too", "many", "columns")); // When: final Exception e = assertThrows( @@ -261,7 +239,7 @@ public void shouldThrowOnDeserializeOnColumnCountMismatch() { ); // Then: - assertThat(e.getMessage(), is("Field count mismatch on deserialization." + assertThat(e.getMessage(), is("Column count mismatch on deserialization." + " topic: topicName" + ", expected: 2" + ", got: 3" @@ -269,7 +247,7 @@ public void shouldThrowOnDeserializeOnColumnCountMismatch() { } @Test - public void shouldConvertRowToStructWhenSerializing() { + public void shouldConvertRowToListWhenSerializing() { // Given: final GenericRow row = GenericRow.genericRow("hello", 10); @@ -277,20 +255,13 @@ public void shouldConvertRowToStructWhenSerializing() { serializer.serialize("topicName", row); // Then: - verify(innerSerializer).serialize("topicName", struct - .put("vic", "hello") - .put("bob", 10) - ); + verify(innerSerializer).serialize("topicName", row.values()); } @Test - public void shouldConvertStructToRowWhenDeserializing() { + public void shouldConvertListToRowWhenDeserializing() { // Given: - struct - .put("vic", "world") - .put("bob", -10); - - when(innerDeserializer.deserialize(any(), any())).thenReturn(struct); + givenInnerDeserializerReturns(ImmutableList.of("world", -10)); // When: final GenericRow row = deserializer.deserialize("topicName", SERIALIZED); @@ -299,36 +270,8 @@ public void shouldConvertStructToRowWhenDeserializing() { assertThat(row, is(GenericRow.genericRow("world", -10))); } - @Test - public void shouldThrowOnSerializationIfColumnValueDoesNotMatchSchema() { - // Given: - final GenericRow row = GenericRow.genericRow("hello", "Not a number"); - - // When: - assertThrows( - DataException.class, - () -> serializer.serialize("topicName", row) - ); - } - - @Test - public void shouldThrowOnSerializationIfStructColumnValueDoesNotMatchSchema() { - // Given: - final GenericRow row = GenericRow.genericRow(struct, 10); - - // When: - final Exception e = assertThrows( - KsqlException.class, - () -> serializer.serialize("topicName", row) - ); - - // Then: - assertThat(e.getMessage(), is( - "Failed to prepare Struct value field 'vic' for serialization. " - + "This could happen if the value was produced by a user-defined function " - + "where the schema has non-optional return types. ksqlDB requires all " - + "schemas to be optional at all levels of the Struct: the Struct itself, " - + "schemas for all fields within the Struct, and so on." - )); + @SuppressWarnings({"unchecked", "rawtypes"}) + private void givenInnerDeserializerReturns(final List values) { + when(innerDeserializer.deserialize(any(), any())).thenReturn((List)values); } } \ No newline at end of file diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/connect/ConnectFormatTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/connect/ConnectFormatTest.java index 63066ec9af1a..4e3110075504 100644 --- a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/connect/ConnectFormatTest.java +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/connect/ConnectFormatTest.java @@ -49,6 +49,7 @@ import java.util.Set; import java.util.function.Function; import java.util.function.Supplier; +import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serializer; @@ -113,6 +114,8 @@ public void setUp() { when(serde.serializer()).thenReturn(serializer); when(serde.deserializer()).thenReturn(deserializer); + + when(persistenceSchema.features()).thenReturn(EnabledSerdeFeatures.of()); } @Test @@ -220,14 +223,11 @@ public void shouldCallSubclassToCreateOuterWhenWrapped() { // Given: final SimpleColumn singleColumn = createColumn("bob", SqlTypes.INTEGER); when(persistenceSchema.columns()).thenReturn(ImmutableList.of(singleColumn)); - when(persistenceSchema.features()).thenReturn(EnabledSerdeFeatures.of()); // When: - final Serde result = format.getSerde(persistenceSchema, formatProps, config, srFactory); + format.getSerde(persistenceSchema, formatProps, config, srFactory); // Then: - assertThat(result, is(serde)); - verify(format) .getConnectSerde(SINGLE_FIELD_SCHEMA, formatProps, config, srFactory, Struct.class); } @@ -243,7 +243,8 @@ public void shouldCallSubclassToCreateInnerWhenUnwrapped() { final ConnectSchema fieldSchema = (ConnectSchema) SINGLE_FIELD_SCHEMA.fields().get(0).schema(); // When: - final Serde result = format.getSerde(persistenceSchema, formatProps, config, srFactory); + final Serde> result = format + .getSerde(persistenceSchema, formatProps, config, srFactory); // Then: verify(format) @@ -253,6 +254,43 @@ public void shouldCallSubclassToCreateInnerWhenUnwrapped() { assertThat(result.deserializer(), instanceOf(UnwrappedDeserializer.class)); } + @Test + public void shouldThrowOnSerializationIfStructColumnValueDoesNotMatchSchema() { + // Given: + final SimpleColumn singleColumn = createColumn( + "bob", + SqlTypes.struct() + .field("vic", SqlTypes.STRING) + .build() + ); + when(persistenceSchema.columns()).thenReturn(ImmutableList.of(singleColumn)); + + final ConnectSchema connectSchema = (ConnectSchema) SchemaBuilder.struct() + .field("vic", Schema.STRING_SCHEMA) + .build(); + + final Serializer> serializer = format + .getSerde(persistenceSchema, formatProps, config, srFactory) + .serializer(); + + final List values = ImmutableList.of(new Struct(connectSchema)); + + // When: + final Exception e = assertThrows( + SerializationException.class, + () -> serializer.serialize("topicName", values) + ); + + // Then: + assertThat(e.getMessage(), is( + "Failed to prepare Struct value field 'bob' for serialization. " + + "This could happen if the value was produced by a user-defined function " + + "where the schema has non-optional return types. ksqlDB requires all " + + "schemas to be optional at all levels of the Struct: the Struct itself, " + + "schemas for all fields within the Struct, and so on." + )); + } + private static SimpleColumn createColumn(final String name, final SqlType sqlType) { final SimpleColumn column = mock(SimpleColumn.class); when(column.name()).thenReturn(ColumnName.of(name)); diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java index 3cde0bf5fac5..d0a255f089bb 100644 --- a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedDeserializerTest.java @@ -15,14 +15,11 @@ package io.confluent.ksql.serde.delimited; -import static java.nio.charset.StandardCharsets.UTF_8; -import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThrows; -import static org.junit.internal.matchers.ThrowableMessageMatcher.hasMessage; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -32,16 +29,13 @@ import io.confluent.ksql.schema.ksql.types.SqlType; import io.confluent.ksql.schema.ksql.types.SqlTypes; import io.confluent.ksql.serde.EnabledSerdeFeatures; -import io.confluent.ksql.serde.connect.ConnectSchemas; import io.confluent.ksql.util.KsqlException; import java.math.BigDecimal; import java.nio.charset.StandardCharsets; import java.util.Arrays; +import java.util.List; import org.apache.commons.csv.CSVFormat; import org.apache.kafka.common.errors.SerializationException; -import org.apache.kafka.connect.data.ConnectSchema; -import org.apache.kafka.connect.data.Struct; -import org.hamcrest.CoreMatchers; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -58,9 +52,6 @@ public class KsqlDelimitedDeserializerTest { column("COST", SqlTypes.decimal(4, 2)) ); - private static final ConnectSchema ORDER_CONNECT_SCHEMA = ConnectSchemas - .columnsToConnectSchema(ORDER_SCHEMA.columns()); - private KsqlDelimitedDeserializer deserializer; @Before @@ -74,15 +65,10 @@ public void shouldDeserializeDelimitedCorrectly() { final byte[] bytes = "1511897796092,1,item_1,10.0,10.10\r\n".getBytes(StandardCharsets.UTF_8); // When: - final Struct struct = deserializer.deserialize("", bytes); + final List result = deserializer.deserialize("", bytes); // Then: - assertThat(struct.schema(), is(ORDER_CONNECT_SCHEMA)); - 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"))); + assertThat(result, contains(1511897796092L, 1L, "item_1", 10.0, new BigDecimal("10.10"))); } @Test @@ -91,15 +77,10 @@ public void shouldDeserializeJsonCorrectlyWithEmptyFields() { final byte[] bytes = "1511897796092,1,item_1,,\r\n".getBytes(StandardCharsets.UTF_8); // When: - final Struct struct = deserializer.deserialize("", bytes); + final List result = deserializer.deserialize("", bytes); // Then: - assertThat(struct.schema(), is(ORDER_CONNECT_SCHEMA)); - assertThat(struct.get("ORDERTIME"), is(1511897796092L)); - assertThat(struct.get("ORDERID"), is(1L)); - assertThat(struct.get("ITEMID"), is("item_1")); - assertThat(struct.get("ORDERUNITS"), is(nullValue())); - assertThat(struct.get("COST"), is(nullValue())); + assertThat(result, contains(1511897796092L, 1L, "item_1", null, null)); } @Test @@ -110,26 +91,29 @@ public void shouldThrowIfRowHasTooFewColumns() { // When: final Exception e = assertThrows( SerializationException.class, - () -> deserializer.deserialize("", bytes) + () -> deserializer.deserialize("t", bytes) ); // Then: - assertThat(e.getCause(), (hasMessage(is("Unexpected field count, csvFields:4 schemaFields:5")))); + assertThat(e.getCause().getMessage(), + is("Column count mismatch on deserialization. topic: t, expected: 5, got: 4")); } @Test public void shouldThrowIfRowHasTooMayColumns() { // Given: - final byte[] bytes = "1511897796092,1,item_1,10.0,10.10,extra\r\n".getBytes(StandardCharsets.UTF_8); + final byte[] bytes = "1511897796092,1,item_1,10.0,10.10,extra\r\n" + .getBytes(StandardCharsets.UTF_8); // When: final Exception e = assertThrows( SerializationException.class, - () -> deserializer.deserialize("", bytes) + () -> deserializer.deserialize("t", bytes) ); // Then: - assertThat(e.getCause(), (hasMessage(is("Unexpected field count, csvFields:6 schemaFields:5")))); + assertThat(e.getCause().getMessage(), + is("Column count mismatch on deserialization. topic: t, expected: 5, got: 6")); } @Test @@ -145,10 +129,10 @@ public void shouldDeserializedTopLevelPrimitiveTypeIfSchemaHasOnlySingleField() final byte[] bytes = "10".getBytes(StandardCharsets.UTF_8); // When: - final Struct result = deserializer.deserialize("", bytes); + final List result = deserializer.deserialize("", bytes); // Then: - assertThat(result.get("id"), CoreMatchers.is(10)); + assertThat(result, contains(10)); } @Test @@ -164,10 +148,10 @@ public void shouldDeserializeDecimal() { final byte[] bytes = "01.12".getBytes(StandardCharsets.UTF_8); // When: - final Struct result = deserializer.deserialize("", bytes); + final List result = deserializer.deserialize("", bytes); // Then: - assertThat(result.get("cost"), is(new BigDecimal("01.12"))); + assertThat(result, contains(new BigDecimal("01.12"))); } @Test @@ -183,10 +167,10 @@ public void shouldDeserializeDecimalWithoutLeadingZeros() { final byte[] bytes = "1.12".getBytes(StandardCharsets.UTF_8); // When: - final Struct result = deserializer.deserialize("", bytes); + final List result = deserializer.deserialize("", bytes); // Then: - assertThat(result.get("cost"), is(new BigDecimal("01.12"))); + assertThat(result, contains(new BigDecimal("01.12"))); } @Test @@ -199,15 +183,10 @@ public void shouldDeserializeDelimitedCorrectlyWithTabDelimiter() { new KsqlDelimitedDeserializer(ORDER_SCHEMA, CSVFormat.DEFAULT.withDelimiter('\t')); // When: - final Struct struct = deserializer.deserialize("", bytes); + final List result = deserializer.deserialize("", bytes); // Then: - assertThat(struct.schema(), is(ORDER_CONNECT_SCHEMA)); - 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"))); + assertThat(result, contains(1511897796092L, 1L, "item_1", 10.0, new BigDecimal("10.10"))); } @Test @@ -219,41 +198,10 @@ public void shouldDeserializeDelimitedCorrectlyWithBarDelimiter() { new KsqlDelimitedDeserializer(ORDER_SCHEMA, CSVFormat.DEFAULT.withDelimiter('|')); // When: - final Struct struct = deserializer.deserialize("", bytes); + final List result = deserializer.deserialize("", bytes); // Then: - assertThat(struct.schema(), is(ORDER_CONNECT_SCHEMA)); - 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: - final PersistenceSchema schema = persistenceSchema( - column("id", SqlTypes.INTEGER), - column("id2", SqlTypes.INTEGER) - ); - - final KsqlDelimitedDeserializer deserializer = - createDeserializer(schema); - - final byte[] bytes = "10".getBytes(UTF_8); - - // When: - final Exception e = assertThrows( - SerializationException.class, - () -> deserializer.deserialize("", bytes) - ); - - // Then: - assertThat(e.getCause(), - (instanceOf(KsqlException.class))); - assertThat(e.getCause(), - (hasMessage(CoreMatchers.is("Unexpected field count, csvFields:1 schemaFields:2")))); + assertThat(result, contains(1511897796092L, 1L, "item_1", 10.0d, new BigDecimal("10.10"))); } @Test @@ -265,12 +213,13 @@ public void shouldThrowOnArrayTypes() { // When: final Exception e = assertThrows( - UnsupportedOperationException.class, + KsqlException.class, () -> createDeserializer(schema) ); // Then: - assertThat(e.getMessage(), containsString("DELIMITED does not support type: ARRAY, field: ids")); + assertThat(e.getMessage(), + containsString("The 'DELIMITED' format does not support type 'ARRAY', column: `ids`")); } @Test @@ -282,12 +231,13 @@ public void shouldThrowOnMapTypes() { // When: final Exception e = assertThrows( - UnsupportedOperationException.class, + KsqlException.class, () -> createDeserializer(schema) ); // Then: - assertThat(e.getMessage(), containsString("DELIMITED does not support type: MAP, field: ids")); + assertThat(e.getMessage(), + containsString("The 'DELIMITED' format does not support type 'MAP', column: `ids`")); } @Test @@ -304,13 +254,13 @@ public void shouldThrowOnStructTypes() { // When: final Exception e = assertThrows( - UnsupportedOperationException.class, + KsqlException.class, () -> createDeserializer(schema) ); // Then: assertThat(e.getMessage(), - containsString("DELIMITED does not support type: STRUCT, field: ids")); + containsString("The 'DELIMITED' format does not support type 'STRUCT', column: `ids`")); } private static SimpleColumn column(final String name, final SqlType type) { diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactoryTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactoryTest.java deleted file mode 100644 index a7ea09acec54..000000000000 --- a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerdeFactoryTest.java +++ /dev/null @@ -1,116 +0,0 @@ -/* - * 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.delimited; - -import static io.confluent.ksql.schema.ksql.types.SqlTypes.STRING; -import static io.confluent.ksql.schema.ksql.types.SqlTypes.array; -import static io.confluent.ksql.schema.ksql.types.SqlTypes.map; -import static io.confluent.ksql.schema.ksql.types.SqlTypes.struct; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.containsString; -import static org.junit.Assert.assertThrows; - -import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; -import io.confluent.ksql.name.ColumnName; -import io.confluent.ksql.schema.ksql.LogicalSchema; -import io.confluent.ksql.schema.ksql.PersistenceSchema; -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.SerdeOptions; -import io.confluent.ksql.util.KsqlConfig; -import io.confluent.ksql.util.KsqlException; -import java.util.function.Supplier; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; - -@RunWith(MockitoJUnitRunner.class) -public class KsqlDelimitedSerdeFactoryTest { - - @Mock - private KsqlConfig config; - @Mock - private Supplier srClientFactory; - - private KsqlDelimitedSerdeFactory factory; - - @Before - public void setUp() { - factory = new KsqlDelimitedSerdeFactory(Delimiter.of(',')); - } - - @Test - public void shouldThrowIfArray() { - // Given: - final PersistenceSchema schema = schemaWithFieldOfType(array(STRING)); - - // When: - final Exception e = assertThrows( - KsqlException.class, - () -> factory.createSerde(schema, config, srClientFactory) - ); - - // Then: - assertThat(e.getMessage(), containsString("The 'DELIMITED' format does not support type 'ARRAY'")); - } - - @Test - public void shouldThrowIfMap() { - // Given: - final PersistenceSchema schema = schemaWithFieldOfType(map(SqlTypes.STRING, STRING)); - - // When: - final Exception e = assertThrows( - KsqlException.class, - () -> factory.createSerde(schema, config, srClientFactory) - ); - - // Then: - assertThat(e.getMessage(), containsString("The 'DELIMITED' format does not support type 'MAP'")); - } - - @Test - public void shouldThrowIfStruct() { - // Given: - final PersistenceSchema schema = schemaWithFieldOfType(struct() - .field("f0", STRING) - .build() - ); - - // When: - final Exception e = assertThrows( - KsqlException.class, - () -> factory.createSerde(schema, config, srClientFactory) - ); - - // Then: - assertThat(e.getMessage(), containsString("The 'DELIMITED' format does not support type 'STRUCT'")); - } - - private static PersistenceSchema schemaWithFieldOfType(final SqlType fieldSchema) { - final LogicalSchema schema = LogicalSchema.builder() - .keyColumn(ColumnName.of("k0"), fieldSchema) - .valueColumn(ColumnName.of("v0"), fieldSchema) - .build(); - - final PhysicalSchema physicalSchema = PhysicalSchema.from(schema, SerdeOptions.of()); - return physicalSchema.valueSchema(); - } -} \ No newline at end of file diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java index 7597adc279af..2522e50250b1 100644 --- a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/delimited/KsqlDelimitedSerializerTest.java @@ -18,48 +18,49 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.is; -import static org.junit.Assert.assertThrows; -import static org.junit.internal.matchers.ThrowableMessageMatcher.hasMessage; -import io.confluent.ksql.util.DecimalUtil; +import io.confluent.ksql.name.ColumnName; +import io.confluent.ksql.schema.ksql.LogicalSchema; +import io.confluent.ksql.schema.ksql.PersistenceSchema; +import io.confluent.ksql.schema.ksql.types.SqlType; +import io.confluent.ksql.schema.ksql.types.SqlTypes; +import io.confluent.ksql.serde.EnabledSerdeFeatures; import java.math.BigDecimal; import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; import org.apache.commons.csv.CSVFormat; -import org.apache.kafka.common.errors.SerializationException; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.data.Struct; import org.junit.Before; import org.junit.Test; public class KsqlDelimitedSerializerTest { - private static final Schema SCHEMA = SchemaBuilder.struct() - .field("ORDERTIME", Schema.OPTIONAL_INT64_SCHEMA) - .field("ORDERID", Schema.OPTIONAL_INT64_SCHEMA) - .field("ITEMID", Schema.OPTIONAL_STRING_SCHEMA) - .field("ORDERUNITS", Schema.OPTIONAL_FLOAT64_SCHEMA) - .optional() - .build(); + private static final PersistenceSchema SCHEMA = PersistenceSchema.from( + LogicalSchema.builder() + .valueColumn(ColumnName.of("ORDERTIME"), SqlTypes.BIGINT) + .valueColumn(ColumnName.of("ORDERID"), SqlTypes.BIGINT) + .valueColumn(ColumnName.of("ITEMID"), SqlTypes.STRING) + .valueColumn(ColumnName.of("ORDERUNITS"), SqlTypes.DOUBLE) + .build().value(), + EnabledSerdeFeatures.of() + ); + protected static final CSVFormat CSV_FORMAT = CSVFormat.DEFAULT.withDelimiter(','); private KsqlDelimitedSerializer serializer; @Before public void setUp() { - serializer = new KsqlDelimitedSerializer(CSVFormat.DEFAULT.withDelimiter(',')); + serializer = new KsqlDelimitedSerializer(SCHEMA, CSV_FORMAT); } @Test public void shouldSerializeRowCorrectly() { // Given: - final Struct data = new Struct(SCHEMA) - .put("ORDERTIME", 1511897796092L) - .put("ORDERID", 1L) - .put("ITEMID", "item_1") - .put("ORDERUNITS", 10.0); + final List values = Arrays.asList(1511897796092L, 1L, "item_1", 10.0); // When: - final byte[] bytes = serializer.serialize("t1", data); + final byte[] bytes = serializer.serialize("t1", values); // Then: final String delimitedString = new String(bytes, StandardCharsets.UTF_8); @@ -69,49 +70,25 @@ public void shouldSerializeRowCorrectly() { @Test public void shouldSerializeRowWithNull() { // Given: - final Struct data = new Struct(SCHEMA) - .put("ORDERTIME", 1511897796092L) - .put("ORDERID", 1L) - .put("ITEMID", "item_1") - .put("ORDERUNITS", null); + final List values = Arrays.asList(1511897796092L, 1L, "item_1", null); // When: - final byte[] bytes = serializer.serialize("t1", data); + final byte[] bytes = serializer.serialize("t1", values); // Then: final String delimitedString = new String(bytes, StandardCharsets.UTF_8); assertThat(delimitedString, equalTo("1511897796092,1,item_1,")); } - @Test - public void shouldSerializedTopLevelPrimitiveIfValueHasOneField() { - // Given: - final Schema schema = SchemaBuilder.struct() - .field("id", Schema.OPTIONAL_INT64_SCHEMA) - .build(); - - final Struct value = new Struct(schema) - .put("id", 10L); - - // When: - final byte[] bytes = serializer.serialize("", value); - - // Then: - assertThat(new String(bytes, StandardCharsets.UTF_8), is("10")); - } - @Test public void shouldSerializeDecimal() { // Given: - final Schema schema = SchemaBuilder.struct() - .field("id", DecimalUtil.builder(4, 2).build()) - .build(); + givenSingleColumnSerializer(SqlTypes.decimal(4, 2)); - final Struct value = new Struct(schema) - .put("id", new BigDecimal("11.12")); + final List values = Collections.singletonList(new BigDecimal("11.12")); // When: - final byte[] bytes = serializer.serialize("", value); + final byte[] bytes = serializer.serialize("", values); // Then: assertThat(new String(bytes, StandardCharsets.UTF_8), is("11.12")); @@ -120,15 +97,12 @@ public void shouldSerializeDecimal() { @Test public void shouldSerializeDecimalWithPaddedZeros() { // Given: - final Schema schema = SchemaBuilder.struct() - .field("id", DecimalUtil.builder(4, 2).build()) - .build(); + givenSingleColumnSerializer(SqlTypes.decimal(4, 2)); - final Struct value = new Struct(schema) - .put("id", new BigDecimal("1.12")); + final List values = Collections.singletonList(new BigDecimal("1.12")); // When: - final byte[] bytes = serializer.serialize("", value); + final byte[] bytes = serializer.serialize("", values); // Then: assertThat(new String(bytes, StandardCharsets.UTF_8), is("1.12")); @@ -137,15 +111,12 @@ public void shouldSerializeDecimalWithPaddedZeros() { @Test public void shouldSerializeZeroDecimalWithPaddedZeros() { // Given: - final Schema schema = SchemaBuilder.struct() - .field("id", DecimalUtil.builder(4, 2).build()) - .build(); + givenSingleColumnSerializer(SqlTypes.decimal(4, 2)); - final Struct value = new Struct(schema) - .put("id", BigDecimal.ZERO); + final List values = Collections.singletonList(BigDecimal.ZERO); // When: - final byte[] bytes = serializer.serialize("", value); + final byte[] bytes = serializer.serialize("", values); // Then: assertThat(new String(bytes, StandardCharsets.UTF_8), is("0.00")); @@ -154,15 +125,12 @@ public void shouldSerializeZeroDecimalWithPaddedZeros() { @Test public void shouldSerializeOneHalfDecimalWithPaddedZeros() { // Given: - final Schema schema = SchemaBuilder.struct() - .field("id", DecimalUtil.builder(4, 2).build()) - .build(); + givenSingleColumnSerializer(SqlTypes.decimal(4, 2)); - final Struct value = new Struct(schema) - .put("id", new BigDecimal(0.5)); + final List values = Collections.singletonList(new BigDecimal("0.5")); // When: - final byte[] bytes = serializer.serialize("", value); + final byte[] bytes = serializer.serialize("", values); // Then: assertThat(new String(bytes, StandardCharsets.UTF_8), is("0.50")); @@ -171,15 +139,12 @@ public void shouldSerializeOneHalfDecimalWithPaddedZeros() { @Test public void shouldSerializeNegativeOneHalfDecimalWithPaddedZeros() { // Given: - final Schema schema = SchemaBuilder.struct() - .field("id", DecimalUtil.builder(4, 2).build()) - .build(); + givenSingleColumnSerializer(SqlTypes.decimal(4, 2)); - final Struct value = new Struct(schema) - .put("id", new BigDecimal(-0.5)); + final List values = Collections.singletonList(new BigDecimal("-0.5")); // When: - final byte[] bytes = serializer.serialize("", value); + final byte[] bytes = serializer.serialize("", values); // Then: assertThat(new String(bytes, StandardCharsets.UTF_8), is("\"-0.50\"")); @@ -188,121 +153,44 @@ public void shouldSerializeNegativeOneHalfDecimalWithPaddedZeros() { @Test public void shouldSerializeNegativeDecimalWithPaddedZeros() { // Given: - final Schema schema = SchemaBuilder.struct() - .field("id", DecimalUtil.builder(4, 2).build()) - .build(); + givenSingleColumnSerializer(SqlTypes.decimal(4, 2)); - final Struct value = new Struct(schema) - .put("id", new BigDecimal("-1.12")); + final List values = Collections.singletonList(new BigDecimal("-1.12")); // When: - final byte[] bytes = serializer.serialize("", value); + final byte[] bytes = serializer.serialize("", values); // Then: assertThat(new String(bytes, StandardCharsets.UTF_8), is("\"-1.12\"")); } @Test - public void shouldSerializeRowCorrectlyWithTabDelimiter() { - shouldSerializeRowCorrectlyWithNonDefaultDelimiter('\t'); - } - - @Test - public void shouldSerializeRowCorrectlyWithBarDelimiter() { - shouldSerializeRowCorrectlyWithNonDefaultDelimiter('|'); - } - - private void shouldSerializeRowCorrectlyWithNonDefaultDelimiter(final char delimiter) { + public void shouldSerializeRowCorrectlyWithDifferentDelimiter() { // Given: - final Struct data = new Struct(SCHEMA) - .put("ORDERTIME", 1511897796092L) - .put("ORDERID", 1L) - .put("ITEMID", "item_1") - .put("ORDERUNITS", 10.0); + final List values = Arrays.asList(1511897796092L, 1L, "item_1", 10.0); - final KsqlDelimitedSerializer serializer = - new KsqlDelimitedSerializer(CSVFormat.DEFAULT.withDelimiter(delimiter)); + final KsqlDelimitedSerializer serializer1 = + new KsqlDelimitedSerializer(SCHEMA, CSVFormat.DEFAULT.withDelimiter('\t')); // When: - final byte[] bytes = serializer.serialize("t1", data); + final byte[] bytes = serializer1.serialize("t1", values); // Then: - final String delimitedString = new String(bytes, StandardCharsets.UTF_8); - assertThat(delimitedString, equalTo( - "1511897796092" + delimiter + "1" + delimiter + "item_1" + delimiter + "10.0")); + assertThat(new String(bytes, StandardCharsets.UTF_8), is("1511897796092\t1\titem_1\t10.0")); } - @Test - public void shouldThrowOnArrayField() { - // Given: - final Schema schemaWithArray = SchemaBuilder.struct() - .field("f0", SchemaBuilder - .array(Schema.OPTIONAL_STRING_SCHEMA) - .optional() - .build()) - .optional() - .build(); - - final Struct data = new Struct(schemaWithArray) - .put("f0", null); + private void givenSingleColumnSerializer(final SqlType columnType) { + final PersistenceSchema schema = givenSingleColumnPersistenceSchema(columnType); - // When: - final Exception e = assertThrows( - SerializationException.class, - () -> serializer.serialize("t1", data) - ); - - // Then: - assertThat(e.getCause(), (hasMessage(is("DELIMITED does not support type: ARRAY")))); + serializer = new KsqlDelimitedSerializer(schema, CSV_FORMAT); } - @Test - public void shouldThrowOnMapField() { - // Given: - final Schema schemaWithMap = SchemaBuilder.struct() - .field("f0", SchemaBuilder - .map(Schema.OPTIONAL_STRING_SCHEMA, Schema.OPTIONAL_STRING_SCHEMA) - .optional() - .build()) - .optional() - .build(); - - final Struct data = new Struct(schemaWithMap) - .put("f0", null); - - // When: - final Exception e = assertThrows( - SerializationException.class, - () -> serializer.serialize("t1", data) + private static PersistenceSchema givenSingleColumnPersistenceSchema(final SqlType columnType) { + return PersistenceSchema.from( + LogicalSchema.builder() + .valueColumn(ColumnName.of("id"), columnType) + .build().value(), + EnabledSerdeFeatures.of() ); - - // Then: - assertThat(e.getCause(), (hasMessage(is("DELIMITED does not support type: MAP")))); } - - @Test - public void shouldThrowOnStructField() { - // Given: - final Schema schemaWithStruct = SchemaBuilder.struct() - .field("f0", SchemaBuilder - .struct() - .field("f0", Schema.OPTIONAL_STRING_SCHEMA) - .optional() - .build()) - .optional() - .build(); - - final Struct data = new Struct(schemaWithStruct) - .put("f0", null); - - // When: - final Exception e = assertThrows( - SerializationException.class, - () -> serializer.serialize("t1", data) - ); - - // Then: - assertThat(e.getCause(), (hasMessage(is("DELIMITED does not support type: STRUCT")))); - } - } diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/kafka/KafkaSerdeFactoryTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/kafka/KafkaSerdeFactoryTest.java index f515c0bcb71c..eee13207c957 100644 --- a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/kafka/KafkaSerdeFactoryTest.java +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/kafka/KafkaSerdeFactoryTest.java @@ -21,7 +21,7 @@ import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertThrows; -import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient; +import com.google.common.collect.ImmutableList; import io.confluent.ksql.name.ColumnName; import io.confluent.ksql.schema.ksql.LogicalSchema; import io.confluent.ksql.schema.ksql.PersistenceSchema; @@ -30,25 +30,14 @@ import io.confluent.ksql.schema.ksql.types.SqlType; import io.confluent.ksql.schema.ksql.types.SqlTypes; import io.confluent.ksql.serde.SerdeOptions; -import io.confluent.ksql.serde.connect.ConnectSchemas; -import io.confluent.ksql.util.KsqlConfig; import io.confluent.ksql.util.KsqlException; -import java.util.function.Supplier; +import java.util.Collections; +import java.util.List; import org.apache.kafka.common.serialization.Serde; -import org.apache.kafka.connect.data.Struct; import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; -@RunWith(MockitoJUnitRunner.class) public class KafkaSerdeFactoryTest { - @Mock - private KsqlConfig ksqlConfig; - @Mock - private Supplier srClientFactory; - @Test public void shouldThroIfMultipleFields() { // Given: @@ -85,7 +74,7 @@ public void shouldThroIfBoolean() { } @Test - public void shouldThroIfDecimal() { + public void shouldThrowIfDecimal() { // Given: final PersistenceSchema schema = schemaWithFieldOfType(SqlTypes.decimal(1, 1)); @@ -100,7 +89,7 @@ public void shouldThroIfDecimal() { } @Test - public void shouldThroIfArray() { + public void shouldThrowIfArray() { // Given: final PersistenceSchema schema = schemaWithFieldOfType(SqlTypes.array(SqlTypes.STRING)); @@ -115,10 +104,11 @@ public void shouldThroIfArray() { } @Test - public void shouldThroIfMap() { + public void shouldThrowIfMap() { // Given: - final PersistenceSchema schema = schemaWithFieldOfType(SqlTypes.map(SqlTypes.STRING, SqlTypes.STRING - )); + final PersistenceSchema schema = schemaWithFieldOfType( + SqlTypes.map(SqlTypes.STRING, SqlTypes.STRING + )); // When: final Exception e = assertThrows( @@ -131,7 +121,7 @@ public void shouldThroIfMap() { } @Test - public void shouldThroIfStruct() { + public void shouldThrowIfStruct() { // Given: final PersistenceSchema schema = schemaWithFieldOfType(SqlTypes.struct() .field("f0", SqlTypes.STRING) @@ -152,7 +142,7 @@ public void shouldSerializeNullAsNull() { // Given: final PersistenceSchema schema = schemaWithFieldOfType(SqlTypes.INTEGER); - final Serde serde = KafkaSerdeFactory.createSerde(schema); + final Serde> serde = KafkaSerdeFactory.createSerde(schema); // When: final byte[] result = serde.serializer().serialize("topic", null); @@ -166,7 +156,7 @@ public void shouldDeserializeNullAsNull() { // Given: final PersistenceSchema schema = schemaWithFieldOfType(SqlTypes.INTEGER); - final Serde serde = KafkaSerdeFactory.createSerde(schema); + final Serde> serde = KafkaSerdeFactory.createSerde(schema); // When: final Object result = serde.deserializer().deserialize("topic", null); @@ -175,7 +165,6 @@ public void shouldDeserializeNullAsNull() { assertThat(result, is(nullValue())); } - @Test public void shouldHandleNullKeyColumn() { // Given: @@ -185,7 +174,7 @@ public void shouldHandleNullKeyColumn() { final PersistenceSchema schema = PhysicalSchema.from(logical, SerdeOptions.of()).keySchema(); - final Serde serde = KafkaSerdeFactory.createSerde(schema); + final Serde> serde = KafkaSerdeFactory.createSerde(schema); // When: final byte[] bytes = serde.serializer().serialize("topic", null); @@ -205,12 +194,10 @@ public void shouldHandleEmptyKey() { final PersistenceSchema schema = PhysicalSchema.from(logical, SerdeOptions.of()).keySchema(); - final Serde serde = KafkaSerdeFactory.createSerde(schema); - - final Struct struct = new Struct(ConnectSchemas.columnsToConnectSchema(logical.key())); + final Serde> serde = KafkaSerdeFactory.createSerde(schema); // When: - final byte[] bytes = serde.serializer().serialize("topic", struct); + final byte[] bytes = serde.serializer().serialize("topic", ImmutableList.of()); final Object result = serde.deserializer().deserialize("topic", null); // Then: @@ -249,17 +236,16 @@ private static void shouldHandle(final SqlType fieldSchema, final Object value) // Given: final PersistenceSchema schema = schemaWithFieldOfType(fieldSchema); - final Serde serde = KafkaSerdeFactory.createSerde(schema); + final Serde> serde = KafkaSerdeFactory.createSerde(schema); - final Struct struct = new Struct(ConnectSchemas.columnsToConnectSchema(schema.columns())); - struct.put("f0", value); + final List values = Collections.singletonList(value); // When: - final byte[] bytes = serde.serializer().serialize("topic", struct); + final byte[] bytes = serde.serializer().serialize("topic", values); final Object result = serde.deserializer().deserialize("topic", bytes); // Then: - assertThat(result, is(struct)); + assertThat(result, is(values)); } private static PersistenceSchema schemaWithFieldOfType(final SqlType fieldSchema) { diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/unwrapped/UnwrappedDeserializerTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/unwrapped/UnwrappedDeserializerTest.java index 0e84d00187fe..ca97129f05ce 100644 --- a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/unwrapped/UnwrappedDeserializerTest.java +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/unwrapped/UnwrappedDeserializerTest.java @@ -1,23 +1,20 @@ package io.confluent.ksql.serde.unwrapped; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.nullValue; -import static org.junit.Assert.assertThrows; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import com.google.common.collect.ImmutableMap; import java.nio.charset.StandardCharsets; +import java.util.List; import java.util.Map; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.LongDeserializer; -import org.apache.kafka.connect.data.ConnectSchema; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.data.Struct; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -27,10 +24,6 @@ @RunWith(MockitoJUnitRunner.class) public class UnwrappedDeserializerTest { - private static final ConnectSchema SCHEMA = (ConnectSchema) SchemaBuilder.struct() - .field("bob", SchemaBuilder.OPTIONAL_STRING_SCHEMA) - .build(); - private static final String TOPIC = "some-topic"; private static final Headers HEADERS = new RecordHeaders(); private static final byte[] SERIALIZED = "data".getBytes(StandardCharsets.UTF_8); @@ -38,54 +31,16 @@ public class UnwrappedDeserializerTest { @Mock private Deserializer inner; - private UnwrappedDeserializer deserializer; + private UnwrappedDeserializer deserializer; @Before public void setUp() { - deserializer = new UnwrappedDeserializer<>(SCHEMA, inner, String.class); + deserializer = new UnwrappedDeserializer(inner); when(inner.deserialize(any(), any())).thenReturn(DESERIALIZED); when(inner.deserialize(any(), any(), any())).thenReturn(DESERIALIZED); } - @Test - public void shouldThrowIfNotSingleField() { - // Given: - final ConnectSchema schema = (ConnectSchema) SchemaBuilder.struct() - .field("bob", SchemaBuilder.OPTIONAL_STRING_SCHEMA) - .field("vic", SchemaBuilder.OPTIONAL_STRING_SCHEMA) - .build(); - - // When: - assertThrows( - IllegalArgumentException.class, - () -> new UnwrappedDeserializer<>(schema, inner, String.class) - ); - } - - @Test - public void shouldThrowIfNotStruct() { - // Given: - final ConnectSchema wrongSchema = (ConnectSchema) SchemaBuilder.OPTIONAL_STRING_SCHEMA; - - assertThrows( - IllegalArgumentException.class, - () -> new UnwrappedDeserializer<>(wrongSchema, inner, String.class) - ); - } - - @Test - public void shouldThrowIfSchemaDoesNotMatchTargetType() { - // Given: - final Deserializer inner = new LongDeserializer(); - - // Then: - assertThrows( - IllegalArgumentException.class, - () -> new UnwrappedDeserializer<>(SCHEMA, inner, Long.class) - ); - } - @Test public void shouldConfigureInner() { // Given: @@ -110,7 +65,7 @@ public void shouldCloseInner() { @Test public void shouldDeserializeOldStyleNulls() { // When: - final Struct result = deserializer.deserialize(TOPIC, null); + final List result = deserializer.deserialize(TOPIC, null); // Then: assertThat(result, is(nullValue())); @@ -119,7 +74,7 @@ public void shouldDeserializeOldStyleNulls() { @Test public void shouldDeserializeNewStyleNulls() { // When: - final Struct result = deserializer.deserialize(TOPIC, HEADERS, null); + final List result = deserializer.deserialize(TOPIC, HEADERS, null); // Then: assertThat(result, is(nullValue())); @@ -128,20 +83,20 @@ public void shouldDeserializeNewStyleNulls() { @Test public void shouldDeserializeOldStyle() { // When: - final Struct result = deserializer.deserialize(TOPIC, SERIALIZED); + final List result = deserializer.deserialize(TOPIC, SERIALIZED); // Then: verify(inner).deserialize(TOPIC, SERIALIZED); - assertThat(result, is(new Struct(SCHEMA).put("bob", DESERIALIZED))); + assertThat(result, contains(DESERIALIZED)); } @Test public void shouldDeserializeNewStyle() { // When: - final Struct result = deserializer.deserialize(TOPIC, HEADERS, SERIALIZED); + final List result = deserializer.deserialize(TOPIC, HEADERS, SERIALIZED); // Then: verify(inner).deserialize(TOPIC, HEADERS, SERIALIZED); - assertThat(result, is(new Struct(SCHEMA).put("bob", DESERIALIZED))); + assertThat(result, contains(DESERIALIZED)); } } \ No newline at end of file diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/unwrapped/UnwrappedSerializerTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/unwrapped/UnwrappedSerializerTest.java index b960cccfff16..94c93d10b4e4 100644 --- a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/unwrapped/UnwrappedSerializerTest.java +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/unwrapped/UnwrappedSerializerTest.java @@ -8,16 +8,15 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import java.nio.charset.StandardCharsets; +import java.util.List; import java.util.Map; +import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.internals.RecordHeaders; -import org.apache.kafka.common.serialization.LongSerializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.connect.data.ConnectSchema; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.data.Struct; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -27,10 +26,6 @@ @RunWith(MockitoJUnitRunner.class) public class UnwrappedSerializerTest { - private static final ConnectSchema SCHEMA = (ConnectSchema) SchemaBuilder.struct() - .field("bob", SchemaBuilder.OPTIONAL_STRING_SCHEMA) - .build(); - private static final String TOPIC = "some-topic"; private static final Headers HEADERS = new RecordHeaders(); private static final byte[] SERIALIZED = "serialized".getBytes(StandardCharsets.UTF_8); @@ -42,49 +37,47 @@ public class UnwrappedSerializerTest { @Before public void setUp() { - serializer = new UnwrappedSerializer<>(SCHEMA, inner, String.class); + serializer = new UnwrappedSerializer<>(inner, String.class); when(inner.serialize(any(), any())).thenReturn(SERIALIZED); when(inner.serialize(any(), any(), any())).thenReturn(SERIALIZED); } @Test - public void shouldThrowIfNotSingleField() { - // Given: - final ConnectSchema schema = (ConnectSchema) SchemaBuilder.struct() - .field("bob", SchemaBuilder.OPTIONAL_STRING_SCHEMA) - .field("vic", SchemaBuilder.OPTIONAL_STRING_SCHEMA) - .build(); - + public void shouldThrowIfLessThanOneValue() { // When: - assertThrows( - IllegalArgumentException.class, - () -> new UnwrappedSerializer<>(schema, inner, String.class) + final Exception e = assertThrows( + SerializationException.class, + () -> serializer.serialize("t", ImmutableList.of()) ); + + // Then: + assertThat(e.getMessage(), is("Column count mismatch on serialization. topic: t, expected: 1, got: 0")); } @Test - public void shouldThrowIfNotStruct() { - // Given: - final ConnectSchema wrongSchema = (ConnectSchema) SchemaBuilder.OPTIONAL_STRING_SCHEMA; + public void shouldThrowIfMoreThanOneValue() { + // When: + final Exception e = assertThrows( + SerializationException.class, + () -> serializer.serialize("t", ImmutableList.of("too", "many")) + ); // Then: - assertThrows( - IllegalArgumentException.class, - () -> new UnwrappedSerializer<>(wrongSchema, inner, String.class) - ); + assertThat(e.getMessage(), is("Column count mismatch on serialization. topic: t, expected: 1, got: 2")); } @Test - public void shouldThrowIfSchemaDoesNotMatchTargetType() { - // Given: - final Serializer inner = new LongSerializer(); - + public void shouldThrowIfWrongType() { // Then: - assertThrows( - IllegalArgumentException.class, - () -> new UnwrappedSerializer<>(SCHEMA, inner, Long.class) + final Exception e = assertThrows( + SerializationException.class, + () -> serializer.serialize("t", ImmutableList.of(12)) ); + + // Then: + assertThat(e.getMessage(), is("value does not match expected type. " + + "expected: String, but got: Integer")); } @Test @@ -129,10 +122,10 @@ public void shouldSerializeNewStyleNulls() { @Test public void shouldSerializeOldStyle() { // Given: - final Struct struct = new Struct(SCHEMA).put("bob", DATA); + final List values = ImmutableList.of(DATA); // When: - final byte[] result = serializer.serialize(TOPIC, struct); + final byte[] result = serializer.serialize(TOPIC, values); // Then: verify(inner).serialize(TOPIC, DATA); @@ -142,10 +135,10 @@ public void shouldSerializeOldStyle() { @Test public void shouldSerializeNewStyle() { // Given: - final Struct struct = new Struct(SCHEMA).put("bob", DATA); + final List values = ImmutableList.of(DATA); // When: - final byte[] result = serializer.serialize(TOPIC, HEADERS, struct); + final byte[] result = serializer.serialize(TOPIC, HEADERS, values); // Then: verify(inner).serialize(TOPIC, HEADERS, DATA); diff --git a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/voids/KsqlVoidSerdeTest.java b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/voids/KsqlVoidSerdeTest.java index acef6b7f4107..2bad48570db6 100644 --- a/ksqldb-serde/src/test/java/io/confluent/ksql/serde/voids/KsqlVoidSerdeTest.java +++ b/ksqldb-serde/src/test/java/io/confluent/ksql/serde/voids/KsqlVoidSerdeTest.java @@ -32,12 +32,12 @@ public class KsqlVoidSerdeTest { @Before public void setUp() { - serializer = new KsqlVoidSerde().serializer(); - deserializer = new KsqlVoidSerde().deserializer(); + serializer = new KsqlVoidSerde().serializer(); + deserializer = new KsqlVoidSerde().deserializer(); } @Test - public void shouldSerializeDataToNull() { + public void shouldSerializeNullDataToNull() { // When: final byte[] result = serializer.serialize("t", null); @@ -45,6 +45,16 @@ public void shouldSerializeDataToNull() { assertThat(result, is(nullValue())); } + @SuppressWarnings({"unchecked", "rawtypes"}) + @Test + public void shouldSerializeAnyDataToNull() { + // When: + final byte[] result = ((Serializer)serializer).serialize("t", "not null"); + + // Then: did not throw and + assertThat(result, is(nullValue())); + } + @Test public void shouldNotThrowOnDeserializationIfDataNotNull() { // When: diff --git a/ksqldb-streams/src/test/java/io/confluent/ksql/execution/streams/SourceBuilderTest.java b/ksqldb-streams/src/test/java/io/confluent/ksql/execution/streams/SourceBuilderTest.java index cf9d77260b19..32ac13861a89 100644 --- a/ksqldb-streams/src/test/java/io/confluent/ksql/execution/streams/SourceBuilderTest.java +++ b/ksqldb-streams/src/test/java/io/confluent/ksql/execution/streams/SourceBuilderTest.java @@ -21,6 +21,7 @@ import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; diff --git a/ksqldb-udf-quickstart/src/main/resources/archetype-resources/src/test/java/SummaryStatsUdafTests.java b/ksqldb-udf-quickstart/src/main/resources/archetype-resources/src/test/java/SummaryStatsUdafTests.java index 10fa73e911ae..8ba4dc53bda9 100644 --- a/ksqldb-udf-quickstart/src/main/resources/archetype-resources/src/test/java/SummaryStatsUdafTests.java +++ b/ksqldb-udf-quickstart/src/main/resources/archetype-resources/src/test/java/SummaryStatsUdafTests.java @@ -18,7 +18,6 @@ import io.confluent.ksql.function.udaf.Udaf; import java.util.HashMap; import java.util.Map; -import java.util.stream.Stream; import org.junit.Assert; import org.junit.Test; @@ -53,7 +52,7 @@ public void shouldComputeNewAggregate() { SummaryStatsUdaf.createUdaf(); // When: - Map newAggregate = udaf.aggregate(900.0, aggregate(1.0, 400.0, 400.0)); + final Map newAggregate = udaf.aggregate(900.0, aggregate(1.0, 400.0, 400.0)); // Then: Assert.assertEquals( diff --git a/ksqldb-version-metrics-client/src/test/java/io/confluent/support/metrics/submitters/ConfluentSubmitterTest.java b/ksqldb-version-metrics-client/src/test/java/io/confluent/support/metrics/submitters/ConfluentSubmitterTest.java index ba1a40e8a91c..de95c1f30af7 100644 --- a/ksqldb-version-metrics-client/src/test/java/io/confluent/support/metrics/submitters/ConfluentSubmitterTest.java +++ b/ksqldb-version-metrics-client/src/test/java/io/confluent/support/metrics/submitters/ConfluentSubmitterTest.java @@ -23,6 +23,7 @@ import io.confluent.support.metrics.utils.CustomerIdExamples; import io.confluent.support.metrics.utils.StringUtils; import org.apache.hc.core5.http.ClassicHttpResponse; +import org.apache.hc.core5.http.HttpResponse; import org.junit.Test; public class ConfluentSubmitterTest {