diff --git a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNode.java b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNode.java index 39e0bab607e9..392d3096d8eb 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNode.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNode.java @@ -119,7 +119,8 @@ public SchemaKStream buildStream(final KsqlQueryBuilder builder) { getKsqlTopic().getKafkaTopicName(), getKsqlTopic().getValueFormat(), serdeOptions, - contextStacker + contextStacker, + getTimestampColumn() ); } } diff --git a/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKStream.java b/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKStream.java index d6cc5cc0e74a..a4b9c5bf07dd 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKStream.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKStream.java @@ -40,6 +40,7 @@ import io.confluent.ksql.execution.plan.StreamTableJoin; import io.confluent.ksql.execution.streams.ExecutionStepFactory; import io.confluent.ksql.execution.streams.StepSchemaResolver; +import io.confluent.ksql.execution.timestamp.TimestampColumn; import io.confluent.ksql.function.FunctionRegistry; import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; @@ -97,13 +98,15 @@ public SchemaKStream into( final String kafkaTopicName, final ValueFormat valueFormat, final Set options, - final QueryContext.Stacker contextStacker + final QueryContext.Stacker contextStacker, + final Optional timestampColumn ) { final StreamSink step = ExecutionStepFactory.streamSink( contextStacker, Formats.of(keyFormat, valueFormat, options), sourceStep, - kafkaTopicName + kafkaTopicName, + timestampColumn ); return new SchemaKStream<>( step, diff --git a/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKTable.java b/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKTable.java index 86312cdc1148..83f2262f0213 100644 --- a/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKTable.java +++ b/ksql-engine/src/main/java/io/confluent/ksql/structured/SchemaKTable.java @@ -30,6 +30,7 @@ import io.confluent.ksql.execution.plan.TableSink; import io.confluent.ksql.execution.plan.TableTableJoin; import io.confluent.ksql.execution.streams.ExecutionStepFactory; +import io.confluent.ksql.execution.timestamp.TimestampColumn; import io.confluent.ksql.function.FunctionRegistry; import io.confluent.ksql.metastore.model.KeyField; import io.confluent.ksql.name.ColumnName; @@ -73,13 +74,15 @@ public SchemaKTable into( final String kafkaTopicName, final ValueFormat valueFormat, final Set options, - final QueryContext.Stacker contextStacker + final QueryContext.Stacker contextStacker, + final Optional timestampColumn ) { final TableSink step = ExecutionStepFactory.tableSink( contextStacker, sourceTableStep, Formats.of(keyFormat, valueFormat, options), - kafkaTopicName + kafkaTopicName, + timestampColumn ); return new SchemaKTable<>( step, diff --git a/ksql-engine/src/test/java/io/confluent/ksql/integration/StreamsSelectAndProjectIntTest.java b/ksql-engine/src/test/java/io/confluent/ksql/integration/StreamsSelectAndProjectIntTest.java index d9e2e1ffb3e4..cc2b4182d270 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/integration/StreamsSelectAndProjectIntTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/integration/StreamsSelectAndProjectIntTest.java @@ -184,9 +184,8 @@ public void shouldUseTimestampExtractedFromDDLStatement() throws Exception { final List> records = TEST_HARNESS.verifyAvailableRecords(resultStream.toUpperCase(), 1); - final SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd"); final long timestamp = records.get(0).timestamp(); - assertThat(timestamp, equalTo(dateFormat.parse("2018-01-04").getTime())); + assertThat(timestamp, is(4L)); } private void testTimestampColumnSelection( diff --git a/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNodeTest.java b/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNodeTest.java index bcc0edd4cddf..fee6dfa2b9c2 100644 --- a/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNodeTest.java +++ b/ksql-engine/src/test/java/io/confluent/ksql/planner/plan/KsqlStructuredDataOutputNodeTest.java @@ -117,7 +117,7 @@ public void before() { when(sourceNode.getNodeOutputType()).thenReturn(DataSourceType.KSTREAM); when(sourceNode.buildStream(ksqlStreamBuilder)).thenReturn((SchemaKStream) sourceStream); - when(sourceStream.into(any(), any(), any(), any())) + when(sourceStream.into(any(), any(), any(), any(), any())) .thenReturn((SchemaKStream) sinkStream); when(ksqlStreamBuilder.buildNodeContext(any())).thenAnswer(inv -> @@ -189,7 +189,7 @@ public void shouldBuildOutputNodeForInsertIntoAvroFromNonAvro() { outputNode.buildStream(ksqlStreamBuilder); // Then: - verify(sourceStream).into(any(), eq(valueFormat), any(), any()); + verify(sourceStream).into(any(), eq(valueFormat), any(), any(), any()); } @Test @@ -202,7 +202,8 @@ public void shouldCallInto() { eq(SINK_KAFKA_TOPIC_NAME), eq(JSON_FORMAT), eq(SerdeOption.none()), - stackerCaptor.capture() + stackerCaptor.capture(), + eq(outputNode.getTimestampColumn()) ); assertThat( stackerCaptor.getValue().getQueryContext().getContext(), diff --git a/ksql-execution/src/main/java/io/confluent/ksql/execution/plan/StreamSink.java b/ksql-execution/src/main/java/io/confluent/ksql/execution/plan/StreamSink.java index 66382321d624..955d60e36e39 100644 --- a/ksql-execution/src/main/java/io/confluent/ksql/execution/plan/StreamSink.java +++ b/ksql-execution/src/main/java/io/confluent/ksql/execution/plan/StreamSink.java @@ -16,9 +16,12 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.errorprone.annotations.Immutable; +import io.confluent.ksql.execution.timestamp.TimestampColumn; + import java.util.Collections; import java.util.List; import java.util.Objects; +import java.util.Optional; @Immutable public class StreamSink implements ExecutionStep> { @@ -26,16 +29,20 @@ public class StreamSink implements ExecutionStep> { private final ExecutionStep> source; private final Formats formats; private final String topicName; + private final Optional timestampColumn; public StreamSink( @JsonProperty(value = "properties", required = true) final ExecutionStepPropertiesV1 props, @JsonProperty(value = "source", required = true) final ExecutionStep> source, @JsonProperty(value = "formats", required = true) final Formats formats, - @JsonProperty(value = "topicName", required = true) final String topicName) { + @JsonProperty(value = "topicName", required = true) final String topicName, + @JsonProperty(value = "timestampColumn") final Optional timestampColumn + ) { this.properties = Objects.requireNonNull(props, "props"); this.formats = Objects.requireNonNull(formats, "formats"); this.source = Objects.requireNonNull(source, "source"); this.topicName = Objects.requireNonNull(topicName, "topicName"); + this.timestampColumn = Objects.requireNonNull(timestampColumn, "timestampColumn"); } public String getTopicName() { @@ -60,6 +67,10 @@ public ExecutionStep> getSource() { return source; } + public Optional getTimestampColumn() { + return timestampColumn; + } + @Override public KStreamHolder build(final PlanBuilder builder) { return builder.visitStreamSink(this); @@ -77,12 +88,13 @@ public boolean equals(final Object o) { return Objects.equals(properties, that.properties) && Objects.equals(source, that.source) && Objects.equals(formats, that.formats) - && Objects.equals(topicName, that.topicName); + && Objects.equals(topicName, that.topicName) + && Objects.equals(timestampColumn, that.timestampColumn); } @Override public int hashCode() { - return Objects.hash(properties, source, formats, topicName); + return Objects.hash(properties, source, formats, topicName, timestampColumn); } } diff --git a/ksql-execution/src/main/java/io/confluent/ksql/execution/plan/TableSink.java b/ksql-execution/src/main/java/io/confluent/ksql/execution/plan/TableSink.java index 9456f1e3e12e..744911c88e46 100644 --- a/ksql-execution/src/main/java/io/confluent/ksql/execution/plan/TableSink.java +++ b/ksql-execution/src/main/java/io/confluent/ksql/execution/plan/TableSink.java @@ -17,9 +17,12 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.errorprone.annotations.Immutable; +import io.confluent.ksql.execution.timestamp.TimestampColumn; + import java.util.Collections; import java.util.List; import java.util.Objects; +import java.util.Optional; @Immutable public class TableSink implements ExecutionStep> { @@ -27,17 +30,20 @@ public class TableSink implements ExecutionStep> { private final ExecutionStep> source; private final Formats formats; private final String topicName; + private final Optional timestampColumn; public TableSink( @JsonProperty(value = "properties", required = true) final ExecutionStepPropertiesV1 props, @JsonProperty(value = "source", required = true) final ExecutionStep> source, @JsonProperty(value = "formats", required = true) final Formats formats, - @JsonProperty(value = "topicName", required = true) final String topicName + @JsonProperty(value = "topicName", required = true) final String topicName, + @JsonProperty(value = "timestampColumn") final Optional timestampColumn ) { this.properties = Objects.requireNonNull(props, "props"); this.source = Objects.requireNonNull(source, "source"); this.formats = Objects.requireNonNull(formats, "formats"); this.topicName = Objects.requireNonNull(topicName, "topicName"); + this.timestampColumn = Objects.requireNonNull(timestampColumn, "timestampColumn"); } @Override @@ -63,6 +69,10 @@ public ExecutionStep> getSource() { return source; } + public Optional getTimestampColumn() { + return timestampColumn; + } + @Override public KTableHolder build(final PlanBuilder builder) { return builder.visitTableSink(this); @@ -80,12 +90,13 @@ public boolean equals(final Object o) { return Objects.equals(properties, tableSink.properties) && Objects.equals(source, tableSink.source) && Objects.equals(formats, tableSink.formats) - && Objects.equals(topicName, tableSink.topicName); + && Objects.equals(topicName, tableSink.topicName) + && Objects.equals(timestampColumn, tableSink.timestampColumn); } @Override public int hashCode() { - return Objects.hash(properties, source, formats, topicName); + return Objects.hash(properties, source, formats, topicName, timestampColumn); } } diff --git a/ksql-execution/src/test/java/io/confluent/ksql/execution/plan/StreamSinkTest.java b/ksql-execution/src/test/java/io/confluent/ksql/execution/plan/StreamSinkTest.java index 19ce8b3c73cb..71db6fba3de3 100644 --- a/ksql-execution/src/test/java/io/confluent/ksql/execution/plan/StreamSinkTest.java +++ b/ksql-execution/src/test/java/io/confluent/ksql/execution/plan/StreamSinkTest.java @@ -15,12 +15,16 @@ package io.confluent.ksql.execution.plan; import com.google.common.testing.EqualsTester; +import io.confluent.ksql.execution.timestamp.TimestampColumn; +import io.confluent.ksql.name.ColumnName; 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; +import java.util.Optional; + @RunWith(MockitoJUnitRunner.class) public class StreamSinkTest { @Mock @@ -40,11 +44,13 @@ public class StreamSinkTest { public void shouldImplementEquals() { new EqualsTester() .addEqualityGroup( - new StreamSink<>(properties1, source1, formats1, "topic1"), - new StreamSink<>(properties1, source1, formats1, "topic1")) - .addEqualityGroup(new StreamSink<>(properties2, source1, formats1, "topic1")) - .addEqualityGroup(new StreamSink<>(properties1, source2, formats1, "topic1")) - .addEqualityGroup(new StreamSink<>(properties1, source1, formats2, "topic1")) - .addEqualityGroup(new StreamSink<>(properties1, source1, formats1, "topic2")); + new StreamSink<>(properties1, source1, formats1, "topic1", Optional.empty()), + new StreamSink<>(properties1, source1, formats1, "topic1", Optional.empty())) + .addEqualityGroup(new StreamSink<>(properties2, source1, formats1, "topic1", Optional.empty())) + .addEqualityGroup(new StreamSink<>(properties1, source2, formats1, "topic1", Optional.empty())) + .addEqualityGroup(new StreamSink<>(properties1, source1, formats2, "topic1", Optional.empty())) + .addEqualityGroup(new StreamSink<>(properties1, source1, formats1, "topic2", Optional.empty())) + .addEqualityGroup(new StreamSink<>(properties1, source1, formats1, "topic1", + Optional.of(new TimestampColumn(ColumnName.of("c1"), Optional.of("BIGINT"))))); } } \ No newline at end of file diff --git a/ksql-execution/src/test/java/io/confluent/ksql/execution/plan/TableSinkTest.java b/ksql-execution/src/test/java/io/confluent/ksql/execution/plan/TableSinkTest.java index 091b0c7e7116..01f418fe2525 100644 --- a/ksql-execution/src/test/java/io/confluent/ksql/execution/plan/TableSinkTest.java +++ b/ksql-execution/src/test/java/io/confluent/ksql/execution/plan/TableSinkTest.java @@ -15,12 +15,16 @@ package io.confluent.ksql.execution.plan; import com.google.common.testing.EqualsTester; +import io.confluent.ksql.execution.timestamp.TimestampColumn; +import io.confluent.ksql.name.ColumnName; 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; +import java.util.Optional; + @RunWith(MockitoJUnitRunner.class) public class TableSinkTest { @Mock @@ -40,11 +44,13 @@ public class TableSinkTest { public void shouldImplementEquals() { new EqualsTester() .addEqualityGroup( - new TableSink<>(properties1, source1, formats1, "topic1"), - new TableSink<>(properties1, source1, formats1, "topic1")) - .addEqualityGroup(new TableSink<>(properties2, source1, formats1, "topic1")) - .addEqualityGroup(new TableSink<>(properties1, source2, formats1, "topic1")) - .addEqualityGroup(new TableSink<>(properties1, source1, formats2, "topic1")) - .addEqualityGroup(new TableSink<>(properties1, source1, formats1, "topic2")); + new TableSink<>(properties1, source1, formats1, "topic1", Optional.empty()), + new TableSink<>(properties1, source1, formats1, "topic1", Optional.empty())) + .addEqualityGroup(new TableSink<>(properties2, source1, formats1, "topic1", Optional.empty())) + .addEqualityGroup(new TableSink<>(properties1, source2, formats1, "topic1", Optional.empty())) + .addEqualityGroup(new TableSink<>(properties1, source1, formats2, "topic1", Optional.empty())) + .addEqualityGroup(new TableSink<>(properties1, source1, formats1, "topic2", Optional.empty())) + .addEqualityGroup(new TableSink<>(properties1, source1, formats1, "topic1", + Optional.of(new TimestampColumn(ColumnName.of("c1"), Optional.of("BIGINT")))));; } } \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts b/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts index e92dd1103ba5..948f628efbe0 100644 --- a/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts +++ b/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts @@ -79,8 +79,11 @@ Topologies: --> Project <-- Join-this-join, Join-other-join Processor: Project (stores: []) - --> KSTREAM-SINK-0000000012 + --> ApplyTimestampTransform-S1_JOIN_S2 <-- Join-merge - Sink: KSTREAM-SINK-0000000012 (topic: S1_JOIN_S2) + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000012 <-- Project + Sink: KSTREAM-SINK-0000000012 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 diff --git a/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides b/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides index bfe1cd14b5a5..bc803854f43a 100644 --- a/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides +++ b/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides @@ -79,8 +79,11 @@ Topologies: --> Project <-- Join-this-join, Join-other-join Processor: Project (stores: []) - --> KSTREAM-SINK-0000000012 + --> ApplyTimestampTransform-S1_JOIN_S2 <-- Join-merge - Sink: KSTREAM-SINK-0000000012 (topic: S1_JOIN_S2) + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000012 <-- Project + Sink: KSTREAM-SINK-0000000012 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 diff --git a/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides b/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides index 36385430dd35..ccf7b3a24588 100644 --- a/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides +++ b/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides @@ -51,25 +51,28 @@ Topologies: Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) --> PrependAliasLeft <-- KSTREAM-SOURCE-0000000004 - Source: KSTREAM-SOURCE-0000000000 (topics: [t1]) - --> KTABLE-SOURCE-0000000001 Processor: PrependAliasLeft (stores: []) --> Join <-- KSTREAM-TRANSFORMVALUES-0000000005 Processor: Join (stores: [KafkaTopic_Right-Reduce]) --> Project <-- PrependAliasLeft + Source: KSTREAM-SOURCE-0000000000 (topics: [t1]) + --> KTABLE-SOURCE-0000000001 Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Right-Reduce]) --> KTABLE-TRANSFORMVALUES-0000000002 <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> ApplyTimestampTransform-S1_JOIN_T1 + <-- Join + Processor: ApplyTimestampTransform-S1_JOIN_T1 (stores: []) + --> KSTREAM-SINK-0000000009 + <-- Project Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) --> PrependAliasRight <-- KTABLE-SOURCE-0000000001 - Processor: Project (stores: []) - --> KSTREAM-SINK-0000000009 - <-- Join Sink: KSTREAM-SINK-0000000009 (topic: S1_JOIN_T1) - <-- Project + <-- ApplyTimestampTransform-S1_JOIN_T1 Processor: PrependAliasRight (stores: []) --> none <-- KTABLE-TRANSFORMVALUES-0000000002 diff --git a/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/join-with-custom-timestamp_-_table_table_inner_join_with_ts b/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/join-with-custom-timestamp_-_table_table_inner_join_with_ts index 24b7d1f71d17..f52d57f88a0c 100644 --- a/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/join-with-custom-timestamp_-_table_table_inner_join_with_ts +++ b/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/join-with-custom-timestamp_-_table_table_inner_join_with_ts @@ -80,8 +80,11 @@ Topologies: --> KTABLE-TOSTREAM-0000000012 <-- KTABLE-MERGE-0000000008 Processor: KTABLE-TOSTREAM-0000000012 (stores: []) - --> KSTREAM-SINK-0000000013 + --> ApplyTimestampTransform-S1_JOIN_S2 <-- Project - Sink: KSTREAM-SINK-0000000013 (topic: S1_JOIN_S2) + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000013 <-- KTABLE-TOSTREAM-0000000012 + Sink: KSTREAM-SINK-0000000013 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 diff --git a/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides b/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides index d8eee4042d93..0e860d0a08a3 100644 --- a/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides +++ b/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides @@ -80,8 +80,11 @@ Topologies: --> KTABLE-TOSTREAM-0000000012 <-- KTABLE-MERGE-0000000008 Processor: KTABLE-TOSTREAM-0000000012 (stores: []) - --> KSTREAM-SINK-0000000013 + --> ApplyTimestampTransform-S1_JOIN_S2 <-- Project - Sink: KSTREAM-SINK-0000000013 (topic: S1_JOIN_S2) + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000013 <-- KTABLE-TOSTREAM-0000000012 + Sink: KSTREAM-SINK-0000000013 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 diff --git a/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/rowtime_-_test_ROWTIME b/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/rowtime_-_test_ROWTIME deleted file mode 100644 index 6c58a4d93f60..000000000000 --- a/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/rowtime_-_test_ROWTIME +++ /dev/null @@ -1,60 +0,0 @@ -{ - "ksql.extension.dir" : "ext", - "ksql.streams.cache.max.bytes.buffering" : "0", - "ksql.security.extension.class" : null, - "ksql.transient.prefix" : "transient_", - "ksql.persistence.wrap.single.values" : "true", - "ksql.authorization.cache.expiry.time.secs" : "30", - "ksql.schema.registry.url" : "", - "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", - "ksql.output.topic.name.prefix" : "", - "ksql.streams.auto.offset.reset" : "earliest", - "ksql.connect.url" : "http://localhost:8083", - "ksql.service.id" : "some.ksql.service.id", - "ksql.internal.topic.min.insync.replicas" : "1", - "ksql.streams.shutdown.timeout.ms" : "300000", - "ksql.internal.topic.replicas" : "1", - "ksql.insert.into.values.enabled" : "true", - "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", - "ksql.access.validator.enable" : "auto", - "ksql.streams.bootstrap.servers" : "localhost:0", - "ksql.streams.commit.interval.ms" : "2000", - "ksql.metric.reporters" : "", - "ksql.streams.auto.commit.interval.ms" : "0", - "ksql.metrics.extension" : null, - "ksql.streams.topology.optimization" : "all", - "ksql.execution.plan.enable" : "false", - "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", - "ksql.streams.num.stream.threads" : "4", - "ksql.authorization.cache.max.entries" : "10000", - "ksql.metrics.tags.custom" : "", - "ksql.pull.queries.enable" : "true", - "ksql.udfs.enabled" : "true", - "ksql.udf.enable.security.manager" : "true", - "ksql.connect.worker.config" : "", - "ksql.query.pull.routing.timeout.ms" : "30000", - "ksql.sink.window.change.log.additional.retention" : "1000000", - "ksql.udf.collect.metrics" : "false", - "ksql.persistent.prefix" : "query_", - "ksql.query.persistent.active.limit" : "2147483647" -} -CONFIGS_END -CSAS_OUTPUT_0.KsqlTopic.Source = STRUCT NOT NULL -CSAS_OUTPUT_0.OUTPUT = STRUCT NOT NULL -SCHEMAS_END -Topologies: - Sub-topology: 0 - Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) - --> KSTREAM-TRANSFORMVALUES-0000000001 - Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) - --> WhereFilter - <-- KSTREAM-SOURCE-0000000000 - Processor: WhereFilter (stores: []) - --> Project - <-- KSTREAM-TRANSFORMVALUES-0000000001 - Processor: Project (stores: []) - --> KSTREAM-SINK-0000000004 - <-- WhereFilter - Sink: KSTREAM-SINK-0000000004 (topic: OUTPUT) - <-- Project - diff --git a/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/rowtime_-_test_ROWTIME_with_AND b/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/rowtime_-_test_ROWTIME_with_AND deleted file mode 100644 index 6c58a4d93f60..000000000000 --- a/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/rowtime_-_test_ROWTIME_with_AND +++ /dev/null @@ -1,60 +0,0 @@ -{ - "ksql.extension.dir" : "ext", - "ksql.streams.cache.max.bytes.buffering" : "0", - "ksql.security.extension.class" : null, - "ksql.transient.prefix" : "transient_", - "ksql.persistence.wrap.single.values" : "true", - "ksql.authorization.cache.expiry.time.secs" : "30", - "ksql.schema.registry.url" : "", - "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", - "ksql.output.topic.name.prefix" : "", - "ksql.streams.auto.offset.reset" : "earliest", - "ksql.connect.url" : "http://localhost:8083", - "ksql.service.id" : "some.ksql.service.id", - "ksql.internal.topic.min.insync.replicas" : "1", - "ksql.streams.shutdown.timeout.ms" : "300000", - "ksql.internal.topic.replicas" : "1", - "ksql.insert.into.values.enabled" : "true", - "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", - "ksql.access.validator.enable" : "auto", - "ksql.streams.bootstrap.servers" : "localhost:0", - "ksql.streams.commit.interval.ms" : "2000", - "ksql.metric.reporters" : "", - "ksql.streams.auto.commit.interval.ms" : "0", - "ksql.metrics.extension" : null, - "ksql.streams.topology.optimization" : "all", - "ksql.execution.plan.enable" : "false", - "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", - "ksql.streams.num.stream.threads" : "4", - "ksql.authorization.cache.max.entries" : "10000", - "ksql.metrics.tags.custom" : "", - "ksql.pull.queries.enable" : "true", - "ksql.udfs.enabled" : "true", - "ksql.udf.enable.security.manager" : "true", - "ksql.connect.worker.config" : "", - "ksql.query.pull.routing.timeout.ms" : "30000", - "ksql.sink.window.change.log.additional.retention" : "1000000", - "ksql.udf.collect.metrics" : "false", - "ksql.persistent.prefix" : "query_", - "ksql.query.persistent.active.limit" : "2147483647" -} -CONFIGS_END -CSAS_OUTPUT_0.KsqlTopic.Source = STRUCT NOT NULL -CSAS_OUTPUT_0.OUTPUT = STRUCT NOT NULL -SCHEMAS_END -Topologies: - Sub-topology: 0 - Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) - --> KSTREAM-TRANSFORMVALUES-0000000001 - Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) - --> WhereFilter - <-- KSTREAM-SOURCE-0000000000 - Processor: WhereFilter (stores: []) - --> Project - <-- KSTREAM-TRANSFORMVALUES-0000000001 - Processor: Project (stores: []) - --> KSTREAM-SINK-0000000004 - <-- WhereFilter - Sink: KSTREAM-SINK-0000000004 (topic: OUTPUT) - <-- Project - diff --git a/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/rowtime_-_test_ROWTIME_with_BETWEEN b/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/rowtime_-_test_ROWTIME_with_BETWEEN deleted file mode 100644 index 6c58a4d93f60..000000000000 --- a/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/rowtime_-_test_ROWTIME_with_BETWEEN +++ /dev/null @@ -1,60 +0,0 @@ -{ - "ksql.extension.dir" : "ext", - "ksql.streams.cache.max.bytes.buffering" : "0", - "ksql.security.extension.class" : null, - "ksql.transient.prefix" : "transient_", - "ksql.persistence.wrap.single.values" : "true", - "ksql.authorization.cache.expiry.time.secs" : "30", - "ksql.schema.registry.url" : "", - "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", - "ksql.output.topic.name.prefix" : "", - "ksql.streams.auto.offset.reset" : "earliest", - "ksql.connect.url" : "http://localhost:8083", - "ksql.service.id" : "some.ksql.service.id", - "ksql.internal.topic.min.insync.replicas" : "1", - "ksql.streams.shutdown.timeout.ms" : "300000", - "ksql.internal.topic.replicas" : "1", - "ksql.insert.into.values.enabled" : "true", - "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", - "ksql.access.validator.enable" : "auto", - "ksql.streams.bootstrap.servers" : "localhost:0", - "ksql.streams.commit.interval.ms" : "2000", - "ksql.metric.reporters" : "", - "ksql.streams.auto.commit.interval.ms" : "0", - "ksql.metrics.extension" : null, - "ksql.streams.topology.optimization" : "all", - "ksql.execution.plan.enable" : "false", - "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", - "ksql.streams.num.stream.threads" : "4", - "ksql.authorization.cache.max.entries" : "10000", - "ksql.metrics.tags.custom" : "", - "ksql.pull.queries.enable" : "true", - "ksql.udfs.enabled" : "true", - "ksql.udf.enable.security.manager" : "true", - "ksql.connect.worker.config" : "", - "ksql.query.pull.routing.timeout.ms" : "30000", - "ksql.sink.window.change.log.additional.retention" : "1000000", - "ksql.udf.collect.metrics" : "false", - "ksql.persistent.prefix" : "query_", - "ksql.query.persistent.active.limit" : "2147483647" -} -CONFIGS_END -CSAS_OUTPUT_0.KsqlTopic.Source = STRUCT NOT NULL -CSAS_OUTPUT_0.OUTPUT = STRUCT NOT NULL -SCHEMAS_END -Topologies: - Sub-topology: 0 - Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) - --> KSTREAM-TRANSFORMVALUES-0000000001 - Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) - --> WhereFilter - <-- KSTREAM-SOURCE-0000000000 - Processor: WhereFilter (stores: []) - --> Project - <-- KSTREAM-TRANSFORMVALUES-0000000001 - Processor: Project (stores: []) - --> KSTREAM-SINK-0000000004 - <-- WhereFilter - Sink: KSTREAM-SINK-0000000004 (topic: OUTPUT) - <-- Project - diff --git a/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/rowtime_-_test_ROWTIME_with_inexact_timestring b/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/rowtime_-_test_ROWTIME_with_inexact_timestring deleted file mode 100644 index 6c58a4d93f60..000000000000 --- a/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/rowtime_-_test_ROWTIME_with_inexact_timestring +++ /dev/null @@ -1,60 +0,0 @@ -{ - "ksql.extension.dir" : "ext", - "ksql.streams.cache.max.bytes.buffering" : "0", - "ksql.security.extension.class" : null, - "ksql.transient.prefix" : "transient_", - "ksql.persistence.wrap.single.values" : "true", - "ksql.authorization.cache.expiry.time.secs" : "30", - "ksql.schema.registry.url" : "", - "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", - "ksql.output.topic.name.prefix" : "", - "ksql.streams.auto.offset.reset" : "earliest", - "ksql.connect.url" : "http://localhost:8083", - "ksql.service.id" : "some.ksql.service.id", - "ksql.internal.topic.min.insync.replicas" : "1", - "ksql.streams.shutdown.timeout.ms" : "300000", - "ksql.internal.topic.replicas" : "1", - "ksql.insert.into.values.enabled" : "true", - "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", - "ksql.access.validator.enable" : "auto", - "ksql.streams.bootstrap.servers" : "localhost:0", - "ksql.streams.commit.interval.ms" : "2000", - "ksql.metric.reporters" : "", - "ksql.streams.auto.commit.interval.ms" : "0", - "ksql.metrics.extension" : null, - "ksql.streams.topology.optimization" : "all", - "ksql.execution.plan.enable" : "false", - "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", - "ksql.streams.num.stream.threads" : "4", - "ksql.authorization.cache.max.entries" : "10000", - "ksql.metrics.tags.custom" : "", - "ksql.pull.queries.enable" : "true", - "ksql.udfs.enabled" : "true", - "ksql.udf.enable.security.manager" : "true", - "ksql.connect.worker.config" : "", - "ksql.query.pull.routing.timeout.ms" : "30000", - "ksql.sink.window.change.log.additional.retention" : "1000000", - "ksql.udf.collect.metrics" : "false", - "ksql.persistent.prefix" : "query_", - "ksql.query.persistent.active.limit" : "2147483647" -} -CONFIGS_END -CSAS_OUTPUT_0.KsqlTopic.Source = STRUCT NOT NULL -CSAS_OUTPUT_0.OUTPUT = STRUCT NOT NULL -SCHEMAS_END -Topologies: - Sub-topology: 0 - Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) - --> KSTREAM-TRANSFORMVALUES-0000000001 - Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) - --> WhereFilter - <-- KSTREAM-SOURCE-0000000000 - Processor: WhereFilter (stores: []) - --> Project - <-- KSTREAM-TRANSFORMVALUES-0000000001 - Processor: Project (stores: []) - --> KSTREAM-SINK-0000000004 - <-- WhereFilter - Sink: KSTREAM-SINK-0000000004 (topic: OUTPUT) - <-- Project - diff --git a/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/rowtime_-_test_ROWTIME_with_timezone b/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/rowtime_-_test_ROWTIME_with_timezone deleted file mode 100644 index 6c58a4d93f60..000000000000 --- a/ksql-functional-tests/src/test/resources/expected_topology/0_6_0-pre/rowtime_-_test_ROWTIME_with_timezone +++ /dev/null @@ -1,60 +0,0 @@ -{ - "ksql.extension.dir" : "ext", - "ksql.streams.cache.max.bytes.buffering" : "0", - "ksql.security.extension.class" : null, - "ksql.transient.prefix" : "transient_", - "ksql.persistence.wrap.single.values" : "true", - "ksql.authorization.cache.expiry.time.secs" : "30", - "ksql.schema.registry.url" : "", - "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", - "ksql.output.topic.name.prefix" : "", - "ksql.streams.auto.offset.reset" : "earliest", - "ksql.connect.url" : "http://localhost:8083", - "ksql.service.id" : "some.ksql.service.id", - "ksql.internal.topic.min.insync.replicas" : "1", - "ksql.streams.shutdown.timeout.ms" : "300000", - "ksql.internal.topic.replicas" : "1", - "ksql.insert.into.values.enabled" : "true", - "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", - "ksql.access.validator.enable" : "auto", - "ksql.streams.bootstrap.servers" : "localhost:0", - "ksql.streams.commit.interval.ms" : "2000", - "ksql.metric.reporters" : "", - "ksql.streams.auto.commit.interval.ms" : "0", - "ksql.metrics.extension" : null, - "ksql.streams.topology.optimization" : "all", - "ksql.execution.plan.enable" : "false", - "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", - "ksql.streams.num.stream.threads" : "4", - "ksql.authorization.cache.max.entries" : "10000", - "ksql.metrics.tags.custom" : "", - "ksql.pull.queries.enable" : "true", - "ksql.udfs.enabled" : "true", - "ksql.udf.enable.security.manager" : "true", - "ksql.connect.worker.config" : "", - "ksql.query.pull.routing.timeout.ms" : "30000", - "ksql.sink.window.change.log.additional.retention" : "1000000", - "ksql.udf.collect.metrics" : "false", - "ksql.persistent.prefix" : "query_", - "ksql.query.persistent.active.limit" : "2147483647" -} -CONFIGS_END -CSAS_OUTPUT_0.KsqlTopic.Source = STRUCT NOT NULL -CSAS_OUTPUT_0.OUTPUT = STRUCT NOT NULL -SCHEMAS_END -Topologies: - Sub-topology: 0 - Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) - --> KSTREAM-TRANSFORMVALUES-0000000001 - Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) - --> WhereFilter - <-- KSTREAM-SOURCE-0000000000 - Processor: WhereFilter (stores: []) - --> Project - <-- KSTREAM-TRANSFORMVALUES-0000000001 - Processor: Project (stores: []) - --> KSTREAM-SINK-0000000004 - <-- WhereFilter - Sink: KSTREAM-SINK-0000000004 (topic: OUTPUT) - <-- Project - diff --git a/ksql-functional-tests/src/test/resources/expected_topology/0_7_0/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts b/ksql-functional-tests/src/test/resources/expected_topology/0_7_0/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts index 685120acf10e..8bb791449d65 100644 --- a/ksql-functional-tests/src/test/resources/expected_topology/0_7_0/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts +++ b/ksql-functional-tests/src/test/resources/expected_topology/0_7_0/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts @@ -83,8 +83,11 @@ Topologies: --> Project <-- Join-this-join, Join-other-join Processor: Project (stores: []) - --> KSTREAM-SINK-0000000012 + --> ApplyTimestampTransform-S1_JOIN_S2 <-- Join-merge - Sink: KSTREAM-SINK-0000000012 (topic: S1_JOIN_S2) + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000012 <-- Project + Sink: KSTREAM-SINK-0000000012 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 diff --git a/ksql-functional-tests/src/test/resources/expected_topology/0_7_0/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides b/ksql-functional-tests/src/test/resources/expected_topology/0_7_0/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides index 7e9084cf3f16..c4e5ad38a934 100644 --- a/ksql-functional-tests/src/test/resources/expected_topology/0_7_0/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides +++ b/ksql-functional-tests/src/test/resources/expected_topology/0_7_0/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides @@ -83,8 +83,11 @@ Topologies: --> Project <-- Join-this-join, Join-other-join Processor: Project (stores: []) - --> KSTREAM-SINK-0000000012 + --> ApplyTimestampTransform-S1_JOIN_S2 <-- Join-merge - Sink: KSTREAM-SINK-0000000012 (topic: S1_JOIN_S2) + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000012 <-- Project + Sink: KSTREAM-SINK-0000000012 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 diff --git a/ksql-functional-tests/src/test/resources/expected_topology/0_7_0/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides b/ksql-functional-tests/src/test/resources/expected_topology/0_7_0/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides index 9f0645c8d8bc..d6c8e84e8390 100644 --- a/ksql-functional-tests/src/test/resources/expected_topology/0_7_0/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides +++ b/ksql-functional-tests/src/test/resources/expected_topology/0_7_0/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides @@ -55,25 +55,28 @@ Topologies: Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) --> PrependAliasLeft <-- KSTREAM-SOURCE-0000000004 - Source: KSTREAM-SOURCE-0000000000 (topics: [t1]) - --> KTABLE-SOURCE-0000000001 Processor: PrependAliasLeft (stores: []) --> Join <-- KSTREAM-TRANSFORMVALUES-0000000005 Processor: Join (stores: [KafkaTopic_Right-Reduce]) --> Project <-- PrependAliasLeft + Source: KSTREAM-SOURCE-0000000000 (topics: [t1]) + --> KTABLE-SOURCE-0000000001 Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Right-Reduce]) --> KTABLE-TRANSFORMVALUES-0000000002 <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> ApplyTimestampTransform-S1_JOIN_T1 + <-- Join + Processor: ApplyTimestampTransform-S1_JOIN_T1 (stores: []) + --> KSTREAM-SINK-0000000009 + <-- Project Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) --> PrependAliasRight <-- KTABLE-SOURCE-0000000001 - Processor: Project (stores: []) - --> KSTREAM-SINK-0000000009 - <-- Join Sink: KSTREAM-SINK-0000000009 (topic: S1_JOIN_T1) - <-- Project + <-- ApplyTimestampTransform-S1_JOIN_T1 Processor: PrependAliasRight (stores: []) --> none <-- KTABLE-TRANSFORMVALUES-0000000002 diff --git a/ksql-functional-tests/src/test/resources/expected_topology/0_7_0/join-with-custom-timestamp_-_table_table_inner_join_with_ts b/ksql-functional-tests/src/test/resources/expected_topology/0_7_0/join-with-custom-timestamp_-_table_table_inner_join_with_ts index d3b8d2fd3040..299df6917609 100644 --- a/ksql-functional-tests/src/test/resources/expected_topology/0_7_0/join-with-custom-timestamp_-_table_table_inner_join_with_ts +++ b/ksql-functional-tests/src/test/resources/expected_topology/0_7_0/join-with-custom-timestamp_-_table_table_inner_join_with_ts @@ -84,8 +84,11 @@ Topologies: --> KTABLE-TOSTREAM-0000000012 <-- KTABLE-MERGE-0000000008 Processor: KTABLE-TOSTREAM-0000000012 (stores: []) - --> KSTREAM-SINK-0000000013 + --> ApplyTimestampTransform-S1_JOIN_S2 <-- Project - Sink: KSTREAM-SINK-0000000013 (topic: S1_JOIN_S2) + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000013 <-- KTABLE-TOSTREAM-0000000012 + Sink: KSTREAM-SINK-0000000013 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 diff --git a/ksql-functional-tests/src/test/resources/expected_topology/0_7_0/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides b/ksql-functional-tests/src/test/resources/expected_topology/0_7_0/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides index eb2b0f4a8955..55c22f8cf659 100644 --- a/ksql-functional-tests/src/test/resources/expected_topology/0_7_0/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides +++ b/ksql-functional-tests/src/test/resources/expected_topology/0_7_0/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides @@ -84,8 +84,11 @@ Topologies: --> KTABLE-TOSTREAM-0000000012 <-- KTABLE-MERGE-0000000008 Processor: KTABLE-TOSTREAM-0000000012 (stores: []) - --> KSTREAM-SINK-0000000013 + --> ApplyTimestampTransform-S1_JOIN_S2 <-- Project - Sink: KSTREAM-SINK-0000000013 (topic: S1_JOIN_S2) + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000013 <-- KTABLE-TOSTREAM-0000000012 + Sink: KSTREAM-SINK-0000000013 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_AVRO/6.0.0_1582734336704/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_AVRO/6.0.0_1582734336704/plan.json new file mode 100644 index 000000000000..b611bc7eb5ea --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_AVRO/6.0.0_1582734336704/plan.json @@ -0,0 +1,246 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1 (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, TS BIGINT) WITH (KAFKA_TOPIC='s1', KEY='ID', TIMESTAMP='TS', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S2 (ROWKEY BIGINT KEY, ID BIGINT, F1 STRING, F2 STRING) WITH (KAFKA_TOPIC='s2', KEY='ID', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1_JOIN_S2 WITH (TIMESTAMP='TS') AS SELECT\n S1.ID ID,\n S1.NAME NAME,\n S1.TS TS,\n S2.F1 F1,\n S2.F2 F2\nFROM S1 S1\nINNER JOIN S2 S2 WITHIN 11 SECONDS ON ((S1.ID = S2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1_JOIN_S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "S1_JOIN_S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "S1", "S2" ], + "sink" : "S1_JOIN_S2", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "S1_JOIN_S2" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT" + }, + "selectExpressions" : [ "ID AS S1_ID", "NAME AS S1_NAME", "TS AS S1_TS", "ROWTIME AS S1_ROWTIME", "ROWKEY AS S1_ROWKEY" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING" + }, + "selectExpressions" : [ "ID AS S2_ID", "F1 AS S2_F1", "F2 AS S2_F2", "ROWTIME AS S2_ROWTIME", "ROWKEY AS S2_ROWKEY" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "S1_ID AS ID", "S1_NAME AS NAME", "S1_TS AS TS", "S2_F1 AS F1", "S2_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S1_JOIN_S2", + "timestampColumn" : { + "column" : "TS", + "format" : null + } + }, + "queryId" : "CSAS_S1_JOIN_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/tmp/confluent8609378800848380873", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_AVRO/6.0.0_1582734336704/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_AVRO/6.0.0_1582734336704/spec.json new file mode 100644 index 000000000000..eabdb79b4383 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_AVRO/6.0.0_1582734336704/spec.json @@ -0,0 +1,91 @@ +{ + "version" : "6.0.0", + "timestamp" : 1582734336704, + "schemas" : { + "CSAS_S1_JOIN_S2_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.S1_JOIN_S2" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 0, + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 10000 + }, { + "topic" : "s2", + "key" : 10, + "value" : { + "ID" : 10, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 13000 + }, { + "topic" : "s1", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000 + }, + "timestamp" : 22000 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000 + }, + "timestamp" : 33000 + } ], + "outputs" : [ { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 0 + }, { + "topic" : "S1_JOIN_S2", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 11000 + }, { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 8000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_AVRO/6.0.0_1582734336704/topology b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_AVRO/6.0.0_1582734336704/topology new file mode 100644 index 000000000000..f74bc03af76a --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_AVRO/6.0.0_1582734336704/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [s1]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [s2]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> ApplyTimestampTransform-S1_JOIN_S2 + <-- Join-merge + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Project + Sink: KSTREAM-SINK-0000000012 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_JSON/6.0.0_1582734336763/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_JSON/6.0.0_1582734336763/plan.json new file mode 100644 index 000000000000..ba6acc7623d0 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_JSON/6.0.0_1582734336763/plan.json @@ -0,0 +1,246 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1 (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, TS BIGINT) WITH (KAFKA_TOPIC='s1', KEY='ID', TIMESTAMP='TS', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S2 (ROWKEY BIGINT KEY, ID BIGINT, F1 STRING, F2 STRING) WITH (KAFKA_TOPIC='s2', KEY='ID', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1_JOIN_S2 WITH (TIMESTAMP='TS') AS SELECT\n S1.ID ID,\n S1.NAME NAME,\n S1.TS TS,\n S2.F1 F1,\n S2.F2 F2\nFROM S1 S1\nINNER JOIN S2 S2 WITHIN 11 SECONDS ON ((S1.ID = S2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1_JOIN_S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "S1_JOIN_S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "S1", "S2" ], + "sink" : "S1_JOIN_S2", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "S1_JOIN_S2" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT" + }, + "selectExpressions" : [ "ID AS S1_ID", "NAME AS S1_NAME", "TS AS S1_TS", "ROWTIME AS S1_ROWTIME", "ROWKEY AS S1_ROWKEY" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING" + }, + "selectExpressions" : [ "ID AS S2_ID", "F1 AS S2_F1", "F2 AS S2_F2", "ROWTIME AS S2_ROWTIME", "ROWKEY AS S2_ROWKEY" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "S1_ID AS ID", "S1_NAME AS NAME", "S1_TS AS TS", "S2_F1 AS F1", "S2_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S1_JOIN_S2", + "timestampColumn" : { + "column" : "TS", + "format" : null + } + }, + "queryId" : "CSAS_S1_JOIN_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/tmp/confluent8609378800848380873", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_JSON/6.0.0_1582734336763/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_JSON/6.0.0_1582734336763/spec.json new file mode 100644 index 000000000000..519ecf7f5989 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_JSON/6.0.0_1582734336763/spec.json @@ -0,0 +1,91 @@ +{ + "version" : "6.0.0", + "timestamp" : 1582734336763, + "schemas" : { + "CSAS_S1_JOIN_S2_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.S1_JOIN_S2" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 0, + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 10000 + }, { + "topic" : "s2", + "key" : 10, + "value" : { + "ID" : 10, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 13000 + }, { + "topic" : "s1", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000 + }, + "timestamp" : 22000 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000 + }, + "timestamp" : 33000 + } ], + "outputs" : [ { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 0 + }, { + "topic" : "S1_JOIN_S2", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 11000 + }, { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 8000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_JSON/6.0.0_1582734336763/topology b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_JSON/6.0.0_1582734336763/topology new file mode 100644 index 000000000000..f74bc03af76a --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_JSON/6.0.0_1582734336763/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [s1]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [s2]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> ApplyTimestampTransform-S1_JOIN_S2 + <-- Join-merge + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Project + Sink: KSTREAM-SINK-0000000012 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_PROTOBUF/6.0.0_1582734336880/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_PROTOBUF/6.0.0_1582734336880/plan.json new file mode 100644 index 000000000000..333c666462d1 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_PROTOBUF/6.0.0_1582734336880/plan.json @@ -0,0 +1,246 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1 (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, TS BIGINT) WITH (KAFKA_TOPIC='s1', KEY='ID', TIMESTAMP='TS', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S2 (ROWKEY BIGINT KEY, ID BIGINT, F1 STRING, F2 STRING) WITH (KAFKA_TOPIC='s2', KEY='ID', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1_JOIN_S2 WITH (TIMESTAMP='TS') AS SELECT\n S1.ID ID,\n S1.NAME NAME,\n S1.TS TS,\n S2.F1 F1,\n S2.F2 F2\nFROM S1 S1\nINNER JOIN S2 S2 WITHIN 11 SECONDS ON ((S1.ID = S2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1_JOIN_S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "S1_JOIN_S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "S1", "S2" ], + "sink" : "S1_JOIN_S2", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "S1_JOIN_S2" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT" + }, + "selectExpressions" : [ "ID AS S1_ID", "NAME AS S1_NAME", "TS AS S1_TS", "ROWTIME AS S1_ROWTIME", "ROWKEY AS S1_ROWKEY" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING" + }, + "selectExpressions" : [ "ID AS S2_ID", "F1 AS S2_F1", "F2 AS S2_F2", "ROWTIME AS S2_ROWTIME", "ROWKEY AS S2_ROWKEY" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "S1_ID AS ID", "S1_NAME AS NAME", "S1_TS AS TS", "S2_F1 AS F1", "S2_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S1_JOIN_S2", + "timestampColumn" : { + "column" : "TS", + "format" : null + } + }, + "queryId" : "CSAS_S1_JOIN_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/tmp/confluent8609378800848380873", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_PROTOBUF/6.0.0_1582734336880/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_PROTOBUF/6.0.0_1582734336880/spec.json new file mode 100644 index 000000000000..6ce815b43f16 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_PROTOBUF/6.0.0_1582734336880/spec.json @@ -0,0 +1,91 @@ +{ + "version" : "6.0.0", + "timestamp" : 1582734336880, + "schemas" : { + "CSAS_S1_JOIN_S2_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.S1_JOIN_S2" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 0, + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 10000 + }, { + "topic" : "s2", + "key" : 10, + "value" : { + "ID" : 10, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 13000 + }, { + "topic" : "s1", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000 + }, + "timestamp" : 22000 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000 + }, + "timestamp" : 33000 + } ], + "outputs" : [ { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 0 + }, { + "topic" : "S1_JOIN_S2", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 11000 + }, { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 8000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_PROTOBUF/6.0.0_1582734336880/topology b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_PROTOBUF/6.0.0_1582734336880/topology new file mode 100644 index 000000000000..f74bc03af76a --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_-_PROTOBUF/6.0.0_1582734336880/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [s1]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [s2]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> ApplyTimestampTransform-S1_JOIN_S2 + <-- Join-merge + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Project + Sink: KSTREAM-SINK-0000000012 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734336922/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734336922/plan.json new file mode 100644 index 000000000000..185676b53631 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734336922/plan.json @@ -0,0 +1,252 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1 (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, TS BIGINT) WITH (KAFKA_TOPIC='s1', KEY='ID', TIMESTAMP='TS', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S2 (ROWKEY BIGINT KEY, ID BIGINT, F1 STRING, F2 STRING, RTS BIGINT) WITH (KAFKA_TOPIC='s2', KEY='ID', TIMESTAMP='RTS', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING, `RTS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1_JOIN_S2 WITH (TIMESTAMP='TS') AS SELECT\n S1.ID ID,\n S1.NAME NAME,\n S1.TS TS,\n S2.F1 F1,\n S2.F2 F2\nFROM S1 S1\nINNER JOIN S2 S2 WITHIN 11 SECONDS ON ((S1.ID = S2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1_JOIN_S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "S1_JOIN_S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "S1", "S2" ], + "sink" : "S1_JOIN_S2", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "S1_JOIN_S2" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT" + }, + "selectExpressions" : [ "ID AS S1_ID", "NAME AS S1_NAME", "TS AS S1_TS", "ROWTIME AS S1_ROWTIME", "ROWKEY AS S1_ROWKEY" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING, `RTS` BIGINT" + }, + "selectExpressions" : [ "ID AS S2_ID", "F1 AS S2_F1", "F2 AS S2_F2", "RTS AS S2_RTS", "ROWTIME AS S2_ROWTIME", "ROWKEY AS S2_ROWKEY" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "S1_ID AS ID", "S1_NAME AS NAME", "S1_TS AS TS", "S2_F1 AS F1", "S2_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S1_JOIN_S2", + "timestampColumn" : { + "column" : "TS", + "format" : null + } + }, + "queryId" : "CSAS_S1_JOIN_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/tmp/confluent8609378800848380873", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734336922/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734336922/spec.json new file mode 100644 index 000000000000..e9fc0f11ed2a --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734336922/spec.json @@ -0,0 +1,93 @@ +{ + "version" : "6.0.0", + "timestamp" : 1582734336922, + "schemas" : { + "CSAS_S1_JOIN_S2_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.S1_JOIN_S2" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 0, + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : "foo", + "RTS" : 10000 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 10, + "value" : { + "ID" : 10, + "F1" : "foo", + "F2" : "bar", + "RTS" : 13000 + }, + "timestamp" : 0 + }, { + "topic" : "s1", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000 + }, + "timestamp" : 0 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000 + }, + "timestamp" : 0 + } ], + "outputs" : [ { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 0 + }, { + "topic" : "S1_JOIN_S2", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 11000 + }, { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 8000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734336922/topology b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734336922/topology new file mode 100644 index 000000000000..f74bc03af76a --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734336922/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [s1]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [s2]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> ApplyTimestampTransform-S1_JOIN_S2 + <-- Join-merge + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Project + Sink: KSTREAM-SINK-0000000012 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734336984/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734336984/plan.json new file mode 100644 index 000000000000..b9266faca5d9 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734336984/plan.json @@ -0,0 +1,252 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1 (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, TS BIGINT) WITH (KAFKA_TOPIC='s1', KEY='ID', TIMESTAMP='TS', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S2 (ROWKEY BIGINT KEY, ID BIGINT, F1 STRING, F2 STRING, RTS BIGINT) WITH (KAFKA_TOPIC='s2', KEY='ID', TIMESTAMP='RTS', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING, `RTS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1_JOIN_S2 WITH (TIMESTAMP='TS') AS SELECT\n S1.ID ID,\n S1.NAME NAME,\n S1.TS TS,\n S2.F1 F1,\n S2.F2 F2\nFROM S1 S1\nINNER JOIN S2 S2 WITHIN 11 SECONDS ON ((S1.ID = S2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1_JOIN_S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "S1_JOIN_S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "S1", "S2" ], + "sink" : "S1_JOIN_S2", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "S1_JOIN_S2" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT" + }, + "selectExpressions" : [ "ID AS S1_ID", "NAME AS S1_NAME", "TS AS S1_TS", "ROWTIME AS S1_ROWTIME", "ROWKEY AS S1_ROWKEY" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING, `RTS` BIGINT" + }, + "selectExpressions" : [ "ID AS S2_ID", "F1 AS S2_F1", "F2 AS S2_F2", "RTS AS S2_RTS", "ROWTIME AS S2_ROWTIME", "ROWKEY AS S2_ROWKEY" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "S1_ID AS ID", "S1_NAME AS NAME", "S1_TS AS TS", "S2_F1 AS F1", "S2_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S1_JOIN_S2", + "timestampColumn" : { + "column" : "TS", + "format" : null + } + }, + "queryId" : "CSAS_S1_JOIN_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/tmp/confluent8609378800848380873", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734336984/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734336984/spec.json new file mode 100644 index 000000000000..e25bf17e7907 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734336984/spec.json @@ -0,0 +1,93 @@ +{ + "version" : "6.0.0", + "timestamp" : 1582734336984, + "schemas" : { + "CSAS_S1_JOIN_S2_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.S1_JOIN_S2" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 0, + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : "foo", + "RTS" : 10000 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 10, + "value" : { + "ID" : 10, + "F1" : "foo", + "F2" : "bar", + "RTS" : 13000 + }, + "timestamp" : 0 + }, { + "topic" : "s1", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000 + }, + "timestamp" : 0 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000 + }, + "timestamp" : 0 + } ], + "outputs" : [ { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 0 + }, { + "topic" : "S1_JOIN_S2", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 11000 + }, { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 8000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734336984/topology b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734336984/topology new file mode 100644 index 000000000000..f74bc03af76a --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734336984/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [s1]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [s2]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> ApplyTimestampTransform-S1_JOIN_S2 + <-- Join-merge + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Project + Sink: KSTREAM-SINK-0000000012 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337023/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337023/plan.json new file mode 100644 index 000000000000..7e79c738f433 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337023/plan.json @@ -0,0 +1,252 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1 (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, TS BIGINT) WITH (KAFKA_TOPIC='s1', KEY='ID', TIMESTAMP='TS', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S2 (ROWKEY BIGINT KEY, ID BIGINT, F1 STRING, F2 STRING, RTS BIGINT) WITH (KAFKA_TOPIC='s2', KEY='ID', TIMESTAMP='RTS', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING, `RTS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1_JOIN_S2 WITH (TIMESTAMP='TS') AS SELECT\n S1.ID ID,\n S1.NAME NAME,\n S1.TS TS,\n S2.F1 F1,\n S2.F2 F2\nFROM S1 S1\nINNER JOIN S2 S2 WITHIN 11 SECONDS ON ((S1.ID = S2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1_JOIN_S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "S1_JOIN_S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "S1", "S2" ], + "sink" : "S1_JOIN_S2", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "S1_JOIN_S2" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamStreamJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "rightInternalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT" + }, + "selectExpressions" : [ "ID AS S1_ID", "NAME AS S1_NAME", "TS AS S1_TS", "ROWTIME AS S1_ROWTIME", "ROWKEY AS S1_ROWKEY" ] + }, + "rightSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING, `RTS` BIGINT" + }, + "selectExpressions" : [ "ID AS S2_ID", "F1 AS S2_F1", "F2 AS S2_F2", "RTS AS S2_RTS", "ROWTIME AS S2_ROWTIME", "ROWKEY AS S2_ROWKEY" ] + }, + "beforeMillis" : 11.000000000, + "afterMillis" : 11.000000000 + }, + "selectExpressions" : [ "S1_ID AS ID", "S1_NAME AS NAME", "S1_TS AS TS", "S2_F1 AS F1", "S2_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S1_JOIN_S2", + "timestampColumn" : { + "column" : "TS", + "format" : null + } + }, + "queryId" : "CSAS_S1_JOIN_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/tmp/confluent8609378800848380873", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337023/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337023/spec.json new file mode 100644 index 000000000000..85f5a8a9e9dc --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337023/spec.json @@ -0,0 +1,93 @@ +{ + "version" : "6.0.0", + "timestamp" : 1582734337023, + "schemas" : { + "CSAS_S1_JOIN_S2_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.Join.Right" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_S2_0.S1_JOIN_S2" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 0, + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : "foo", + "RTS" : 10000 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 10, + "value" : { + "ID" : 10, + "F1" : "foo", + "F2" : "bar", + "RTS" : 13000 + }, + "timestamp" : 0 + }, { + "topic" : "s1", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000 + }, + "timestamp" : 0 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000 + }, + "timestamp" : 0 + } ], + "outputs" : [ { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 0 + }, { + "topic" : "S1_JOIN_S2", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 11000 + }, { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 8000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337023/topology b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337023/topology new file mode 100644 index 000000000000..f74bc03af76a --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_stream_inner_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337023/topology @@ -0,0 +1,42 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [s1]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Source: KSTREAM-SOURCE-0000000003 (topics: [s2]) + --> KSTREAM-TRANSFORMVALUES-0000000004 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000000 + Processor: KSTREAM-TRANSFORMVALUES-0000000004 (stores: []) + --> PrependAliasRight + <-- KSTREAM-SOURCE-0000000003 + Processor: PrependAliasLeft (stores: []) + --> Join-this-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: PrependAliasRight (stores: []) + --> Join-other-windowed + <-- KSTREAM-TRANSFORMVALUES-0000000004 + Processor: Join-other-windowed (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-other-join + <-- PrependAliasRight + Processor: Join-this-windowed (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-this-join + <-- PrependAliasLeft + Processor: Join-other-join (stores: [KSTREAM-JOINTHIS-0000000008-store]) + --> Join-merge + <-- Join-other-windowed + Processor: Join-this-join (stores: [KSTREAM-JOINOTHER-0000000009-store]) + --> Join-merge + <-- Join-this-windowed + Processor: Join-merge (stores: []) + --> Project + <-- Join-this-join, Join-other-join + Processor: Project (stores: []) + --> ApplyTimestampTransform-S1_JOIN_S2 + <-- Join-merge + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000012 + <-- Project + Sink: KSTREAM-SINK-0000000012 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734337067/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734337067/plan.json new file mode 100644 index 000000000000..dfdefef210e2 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734337067/plan.json @@ -0,0 +1,239 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1 (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, TS BIGINT) WITH (KAFKA_TOPIC='s1', KEY='ID', TIMESTAMP='TS', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE T1 (ROWKEY BIGINT KEY, ID BIGINT, F1 STRING, F2 STRING, RTS BIGINT) WITH (KAFKA_TOPIC='t1', KEY='ID', TIMESTAMP='RTS', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "T1", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING, `RTS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "topicName" : "t1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1_JOIN_T1 WITH (TIMESTAMP='TS') AS SELECT\n S1.ID ID,\n S1.NAME NAME,\n S1.TS TS,\n T1.F1 F1,\n T1.F2 F2\nFROM S1 S1\nINNER JOIN T1 T1 ON ((S1.ID = T1.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1_JOIN_T1", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "S1_JOIN_T1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "S1", "T1" ], + "sink" : "S1_JOIN_T1", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "S1_JOIN_T1" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT" + }, + "selectExpressions" : [ "ID AS S1_ID", "NAME AS S1_NAME", "TS AS S1_TS", "ROWTIME AS S1_ROWTIME", "ROWKEY AS S1_ROWKEY" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "t1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING, `RTS` BIGINT" + }, + "selectExpressions" : [ "ID AS T1_ID", "F1 AS T1_F1", "F2 AS T1_F2", "RTS AS T1_RTS", "ROWTIME AS T1_ROWTIME", "ROWKEY AS T1_ROWKEY" ] + } + }, + "selectExpressions" : [ "S1_ID AS ID", "S1_NAME AS NAME", "S1_TS AS TS", "T1_F1 AS F1", "T1_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S1_JOIN_T1", + "timestampColumn" : { + "column" : "TS", + "format" : null + } + }, + "queryId" : "CSAS_S1_JOIN_T1_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/tmp/confluent8609378800848380873", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734337067/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734337067/spec.json new file mode 100644 index 000000000000..d8c04cdf326a --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734337067/spec.json @@ -0,0 +1,92 @@ +{ + "version" : "6.0.0", + "timestamp" : 1582734337067, + "schemas" : { + "CSAS_S1_JOIN_T1_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_T1_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_T1_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_T1_0.S1_JOIN_T1" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "t1", + "key" : 0, + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : "foo", + "RTS" : 10000 + }, + "timestamp" : 10000 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "t1", + "key" : 10, + "value" : { + "ID" : 10, + "F1" : "foo", + "F2" : "bar", + "RTS" : 13000 + }, + "timestamp" : 90000 + }, { + "topic" : "s1", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000 + }, + "timestamp" : 800000 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000 + }, + "timestamp" : 0 + } ], + "outputs" : [ { + "topic" : "S1_JOIN_T1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 0 + }, { + "topic" : "S1_JOIN_T1", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 11000 + }, { + "topic" : "S1_JOIN_T1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 8000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734337067/topology b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734337067/topology new file mode 100644 index 000000000000..9834b690229a --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734337067/topology @@ -0,0 +1,33 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000004 (topics: [s1]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000004 + Processor: PrependAliasLeft (stores: []) + --> Join + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: Join (stores: [KafkaTopic_Right-Reduce]) + --> Project + <-- PrependAliasLeft + Source: KSTREAM-SOURCE-0000000000 (topics: [t1]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> ApplyTimestampTransform-S1_JOIN_T1 + <-- Join + Processor: ApplyTimestampTransform-S1_JOIN_T1 (stores: []) + --> KSTREAM-SINK-0000000009 + <-- Project + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000001 + Sink: KSTREAM-SINK-0000000009 (topic: S1_JOIN_T1) + <-- ApplyTimestampTransform-S1_JOIN_T1 + Processor: PrependAliasRight (stores: []) + --> none + <-- KTABLE-TRANSFORMVALUES-0000000002 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734337146/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734337146/plan.json new file mode 100644 index 000000000000..38e8a4ed4bc9 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734337146/plan.json @@ -0,0 +1,239 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1 (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, TS BIGINT) WITH (KAFKA_TOPIC='s1', KEY='ID', TIMESTAMP='TS', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE T1 (ROWKEY BIGINT KEY, ID BIGINT, F1 STRING, F2 STRING, RTS BIGINT) WITH (KAFKA_TOPIC='t1', KEY='ID', TIMESTAMP='RTS', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "T1", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING, `RTS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "topicName" : "t1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1_JOIN_T1 WITH (TIMESTAMP='TS') AS SELECT\n S1.ID ID,\n S1.NAME NAME,\n S1.TS TS,\n T1.F1 F1,\n T1.F2 F2\nFROM S1 S1\nINNER JOIN T1 T1 ON ((S1.ID = T1.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1_JOIN_T1", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "S1_JOIN_T1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "S1", "T1" ], + "sink" : "S1_JOIN_T1", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "S1_JOIN_T1" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT" + }, + "selectExpressions" : [ "ID AS S1_ID", "NAME AS S1_NAME", "TS AS S1_TS", "ROWTIME AS S1_ROWTIME", "ROWKEY AS S1_ROWKEY" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "t1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING, `RTS` BIGINT" + }, + "selectExpressions" : [ "ID AS T1_ID", "F1 AS T1_F1", "F2 AS T1_F2", "RTS AS T1_RTS", "ROWTIME AS T1_ROWTIME", "ROWKEY AS T1_ROWKEY" ] + } + }, + "selectExpressions" : [ "S1_ID AS ID", "S1_NAME AS NAME", "S1_TS AS TS", "T1_F1 AS F1", "T1_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S1_JOIN_T1", + "timestampColumn" : { + "column" : "TS", + "format" : null + } + }, + "queryId" : "CSAS_S1_JOIN_T1_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/tmp/confluent8609378800848380873", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734337146/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734337146/spec.json new file mode 100644 index 000000000000..844bdcadd54e --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734337146/spec.json @@ -0,0 +1,92 @@ +{ + "version" : "6.0.0", + "timestamp" : 1582734337146, + "schemas" : { + "CSAS_S1_JOIN_T1_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_T1_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_T1_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_T1_0.S1_JOIN_T1" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "t1", + "key" : 0, + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : "foo", + "RTS" : 10000 + }, + "timestamp" : 10000 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "t1", + "key" : 10, + "value" : { + "ID" : 10, + "F1" : "foo", + "F2" : "bar", + "RTS" : 13000 + }, + "timestamp" : 90000 + }, { + "topic" : "s1", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000 + }, + "timestamp" : 800000 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000 + }, + "timestamp" : 0 + } ], + "outputs" : [ { + "topic" : "S1_JOIN_T1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 0 + }, { + "topic" : "S1_JOIN_T1", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 11000 + }, { + "topic" : "S1_JOIN_T1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 8000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734337146/topology b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734337146/topology new file mode 100644 index 000000000000..9834b690229a --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734337146/topology @@ -0,0 +1,33 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000004 (topics: [s1]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000004 + Processor: PrependAliasLeft (stores: []) + --> Join + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: Join (stores: [KafkaTopic_Right-Reduce]) + --> Project + <-- PrependAliasLeft + Source: KSTREAM-SOURCE-0000000000 (topics: [t1]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> ApplyTimestampTransform-S1_JOIN_T1 + <-- Join + Processor: ApplyTimestampTransform-S1_JOIN_T1 (stores: []) + --> KSTREAM-SINK-0000000009 + <-- Project + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000001 + Sink: KSTREAM-SINK-0000000009 (topic: S1_JOIN_T1) + <-- ApplyTimestampTransform-S1_JOIN_T1 + Processor: PrependAliasRight (stores: []) + --> none + <-- KTABLE-TRANSFORMVALUES-0000000002 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337225/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337225/plan.json new file mode 100644 index 000000000000..2b68f8fb2005 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337225/plan.json @@ -0,0 +1,239 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1 (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, TS BIGINT) WITH (KAFKA_TOPIC='s1', KEY='ID', TIMESTAMP='TS', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE T1 (ROWKEY BIGINT KEY, ID BIGINT, F1 STRING, F2 STRING, RTS BIGINT) WITH (KAFKA_TOPIC='t1', KEY='ID', TIMESTAMP='RTS', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "T1", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING, `RTS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "topicName" : "t1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM S1_JOIN_T1 WITH (TIMESTAMP='TS') AS SELECT\n S1.ID ID,\n S1.NAME NAME,\n S1.TS TS,\n T1.F1 F1,\n T1.F2 F2\nFROM S1 S1\nINNER JOIN T1 T1 ON ((S1.ID = T1.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "S1_JOIN_T1", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "S1_JOIN_T1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "S1", "T1" ], + "sink" : "S1_JOIN_T1", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "S1_JOIN_T1" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "internalFormats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "leftSource" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT" + }, + "selectExpressions" : [ "ID AS S1_ID", "NAME AS S1_NAME", "TS AS S1_TS", "ROWTIME AS S1_ROWTIME", "ROWKEY AS S1_ROWKEY" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "t1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING, `RTS` BIGINT" + }, + "selectExpressions" : [ "ID AS T1_ID", "F1 AS T1_F1", "F2 AS T1_F2", "RTS AS T1_RTS", "ROWTIME AS T1_ROWTIME", "ROWKEY AS T1_ROWKEY" ] + } + }, + "selectExpressions" : [ "S1_ID AS ID", "S1_NAME AS NAME", "S1_TS AS TS", "T1_F1 AS F1", "T1_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S1_JOIN_T1", + "timestampColumn" : { + "column" : "TS", + "format" : null + } + }, + "queryId" : "CSAS_S1_JOIN_T1_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/tmp/confluent8609378800848380873", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337225/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337225/spec.json new file mode 100644 index 000000000000..fb9f481de5b8 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337225/spec.json @@ -0,0 +1,92 @@ +{ + "version" : "6.0.0", + "timestamp" : 1582734337225, + "schemas" : { + "CSAS_S1_JOIN_T1_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_T1_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_T1_0.Join.Left" : "STRUCT NOT NULL", + "CSAS_S1_JOIN_T1_0.S1_JOIN_T1" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "t1", + "key" : 0, + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : "foo", + "RTS" : 10000 + }, + "timestamp" : 10000 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0 + }, + "timestamp" : 10000 + }, { + "topic" : "t1", + "key" : 10, + "value" : { + "ID" : 10, + "F1" : "foo", + "F2" : "bar", + "RTS" : 13000 + }, + "timestamp" : 90000 + }, { + "topic" : "s1", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000 + }, + "timestamp" : 800000 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000 + }, + "timestamp" : 0 + } ], + "outputs" : [ { + "topic" : "S1_JOIN_T1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 0 + }, { + "topic" : "S1_JOIN_T1", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 11000 + }, { + "topic" : "S1_JOIN_T1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 8000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337225/topology b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337225/topology new file mode 100644 index 000000000000..9834b690229a --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_stream_table_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337225/topology @@ -0,0 +1,33 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000004 (topics: [s1]) + --> KSTREAM-TRANSFORMVALUES-0000000005 + Processor: KSTREAM-TRANSFORMVALUES-0000000005 (stores: []) + --> PrependAliasLeft + <-- KSTREAM-SOURCE-0000000004 + Processor: PrependAliasLeft (stores: []) + --> Join + <-- KSTREAM-TRANSFORMVALUES-0000000005 + Processor: Join (stores: [KafkaTopic_Right-Reduce]) + --> Project + <-- PrependAliasLeft + Source: KSTREAM-SOURCE-0000000000 (topics: [t1]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> ApplyTimestampTransform-S1_JOIN_T1 + <-- Join + Processor: ApplyTimestampTransform-S1_JOIN_T1 (stores: []) + --> KSTREAM-SINK-0000000009 + <-- Project + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000001 + Sink: KSTREAM-SINK-0000000009 (topic: S1_JOIN_T1) + <-- ApplyTimestampTransform-S1_JOIN_T1 + Processor: PrependAliasRight (stores: []) + --> none + <-- KTABLE-TRANSFORMVALUES-0000000002 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_AVRO/6.0.0_1582734337274/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_AVRO/6.0.0_1582734337274/plan.json new file mode 100644 index 000000000000..569b26768230 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_AVRO/6.0.0_1582734337274/plan.json @@ -0,0 +1,222 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S1 (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, TS BIGINT) WITH (KAFKA_TOPIC='s1', KEY='ID', TIMESTAMP='TS', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S1", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 (ROWKEY BIGINT KEY, ID BIGINT, F1 STRING, F2 STRING) WITH (KAFKA_TOPIC='s2', KEY='ID', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S1_JOIN_S2 WITH (TIMESTAMP='TS') AS SELECT\n S1.ID ID,\n S1.NAME NAME,\n S1.TS TS,\n S2.F1 F1,\n S2.F2 F2\nFROM S1 S1\nINNER JOIN S2 S2 ON ((S1.ID = S2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S1_JOIN_S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "S1_JOIN_S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "S1", "S2" ], + "sink" : "S1_JOIN_S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S1_JOIN_S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT" + }, + "selectExpressions" : [ "ID AS S1_ID", "NAME AS S1_NAME", "TS AS S1_TS", "ROWTIME AS S1_ROWTIME", "ROWKEY AS S1_ROWKEY" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING" + }, + "selectExpressions" : [ "ID AS S2_ID", "F1 AS S2_F1", "F2 AS S2_F2", "ROWTIME AS S2_ROWTIME", "ROWKEY AS S2_ROWKEY" ] + } + }, + "selectExpressions" : [ "S1_ID AS ID", "S1_NAME AS NAME", "S1_TS AS TS", "S2_F1 AS F1", "S2_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S1_JOIN_S2", + "timestampColumn" : { + "column" : "TS", + "format" : null + } + }, + "queryId" : "CTAS_S1_JOIN_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/tmp/confluent8609378800848380873", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_AVRO/6.0.0_1582734337274/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_AVRO/6.0.0_1582734337274/spec.json new file mode 100644 index 000000000000..c5d76337b755 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_AVRO/6.0.0_1582734337274/spec.json @@ -0,0 +1,89 @@ +{ + "version" : "6.0.0", + "timestamp" : 1582734337274, + "schemas" : { + "CTAS_S1_JOIN_S2_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CTAS_S1_JOIN_S2_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CTAS_S1_JOIN_S2_0.S1_JOIN_S2" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 0, + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 10000 + }, { + "topic" : "s2", + "key" : 10, + "value" : { + "ID" : 10, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 13000 + }, { + "topic" : "s1", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 19000 + }, + "timestamp" : 22000 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 18000 + }, + "timestamp" : 33000 + } ], + "outputs" : [ { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 0 + }, { + "topic" : "S1_JOIN_S2", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 19000, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 19000 + }, { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 18000, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 18000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_AVRO/6.0.0_1582734337274/topology b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_AVRO/6.0.0_1582734337274/topology new file mode 100644 index 000000000000..3f5c3b5480d2 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_AVRO/6.0.0_1582734337274/topology @@ -0,0 +1,45 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [s1]) + --> KTABLE-SOURCE-0000000001 + Source: KSTREAM-SOURCE-0000000004 (topics: [s2]) + --> KTABLE-SOURCE-0000000005 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-SOURCE-0000000005 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000006 + <-- KSTREAM-SOURCE-0000000004 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasLeft + <-- KTABLE-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000006 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000005 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000009 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000010 + <-- KTABLE-TRANSFORMVALUES-0000000006 + Processor: KTABLE-JOINOTHER-0000000010 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000008 (stores: []) + --> Project + <-- KTABLE-JOINTHIS-0000000009, KTABLE-JOINOTHER-0000000010 + Processor: Project (stores: []) + --> KTABLE-TOSTREAM-0000000012 + <-- KTABLE-MERGE-0000000008 + Processor: KTABLE-TOSTREAM-0000000012 (stores: []) + --> ApplyTimestampTransform-S1_JOIN_S2 + <-- Project + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000013 + <-- KTABLE-TOSTREAM-0000000012 + Sink: KSTREAM-SINK-0000000013 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_JSON/6.0.0_1582734337315/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_JSON/6.0.0_1582734337315/plan.json new file mode 100644 index 000000000000..8e42a0d6c27f --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_JSON/6.0.0_1582734337315/plan.json @@ -0,0 +1,222 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S1 (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, TS BIGINT) WITH (KAFKA_TOPIC='s1', KEY='ID', TIMESTAMP='TS', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S1", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 (ROWKEY BIGINT KEY, ID BIGINT, F1 STRING, F2 STRING) WITH (KAFKA_TOPIC='s2', KEY='ID', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S1_JOIN_S2 WITH (TIMESTAMP='TS') AS SELECT\n S1.ID ID,\n S1.NAME NAME,\n S1.TS TS,\n S2.F1 F1,\n S2.F2 F2\nFROM S1 S1\nINNER JOIN S2 S2 ON ((S1.ID = S2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S1_JOIN_S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "S1_JOIN_S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "S1", "S2" ], + "sink" : "S1_JOIN_S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S1_JOIN_S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT" + }, + "selectExpressions" : [ "ID AS S1_ID", "NAME AS S1_NAME", "TS AS S1_TS", "ROWTIME AS S1_ROWTIME", "ROWKEY AS S1_ROWKEY" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING" + }, + "selectExpressions" : [ "ID AS S2_ID", "F1 AS S2_F1", "F2 AS S2_F2", "ROWTIME AS S2_ROWTIME", "ROWKEY AS S2_ROWKEY" ] + } + }, + "selectExpressions" : [ "S1_ID AS ID", "S1_NAME AS NAME", "S1_TS AS TS", "S2_F1 AS F1", "S2_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S1_JOIN_S2", + "timestampColumn" : { + "column" : "TS", + "format" : null + } + }, + "queryId" : "CTAS_S1_JOIN_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/tmp/confluent8609378800848380873", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_JSON/6.0.0_1582734337315/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_JSON/6.0.0_1582734337315/spec.json new file mode 100644 index 000000000000..6889e338a501 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_JSON/6.0.0_1582734337315/spec.json @@ -0,0 +1,89 @@ +{ + "version" : "6.0.0", + "timestamp" : 1582734337315, + "schemas" : { + "CTAS_S1_JOIN_S2_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CTAS_S1_JOIN_S2_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CTAS_S1_JOIN_S2_0.S1_JOIN_S2" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 0, + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 10000 + }, { + "topic" : "s2", + "key" : 10, + "value" : { + "ID" : 10, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 13000 + }, { + "topic" : "s1", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 19000 + }, + "timestamp" : 22000 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 18000 + }, + "timestamp" : 33000 + } ], + "outputs" : [ { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 0 + }, { + "topic" : "S1_JOIN_S2", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 19000, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 19000 + }, { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 18000, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 18000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_JSON/6.0.0_1582734337315/topology b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_JSON/6.0.0_1582734337315/topology new file mode 100644 index 000000000000..3f5c3b5480d2 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_JSON/6.0.0_1582734337315/topology @@ -0,0 +1,45 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [s1]) + --> KTABLE-SOURCE-0000000001 + Source: KSTREAM-SOURCE-0000000004 (topics: [s2]) + --> KTABLE-SOURCE-0000000005 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-SOURCE-0000000005 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000006 + <-- KSTREAM-SOURCE-0000000004 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasLeft + <-- KTABLE-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000006 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000005 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000009 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000010 + <-- KTABLE-TRANSFORMVALUES-0000000006 + Processor: KTABLE-JOINOTHER-0000000010 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000008 (stores: []) + --> Project + <-- KTABLE-JOINTHIS-0000000009, KTABLE-JOINOTHER-0000000010 + Processor: Project (stores: []) + --> KTABLE-TOSTREAM-0000000012 + <-- KTABLE-MERGE-0000000008 + Processor: KTABLE-TOSTREAM-0000000012 (stores: []) + --> ApplyTimestampTransform-S1_JOIN_S2 + <-- Project + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000013 + <-- KTABLE-TOSTREAM-0000000012 + Sink: KSTREAM-SINK-0000000013 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_PROTOBUF/6.0.0_1582734337353/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_PROTOBUF/6.0.0_1582734337353/plan.json new file mode 100644 index 000000000000..0638f2888e8a --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_PROTOBUF/6.0.0_1582734337353/plan.json @@ -0,0 +1,222 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S1 (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, TS BIGINT) WITH (KAFKA_TOPIC='s1', KEY='ID', TIMESTAMP='TS', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S1", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 (ROWKEY BIGINT KEY, ID BIGINT, F1 STRING, F2 STRING) WITH (KAFKA_TOPIC='s2', KEY='ID', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING", + "keyField" : "ID", + "timestampColumn" : null, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S1_JOIN_S2 WITH (TIMESTAMP='TS') AS SELECT\n S1.ID ID,\n S1.NAME NAME,\n S1.TS TS,\n S2.F1 F1,\n S2.F2 F2\nFROM S1 S1\nINNER JOIN S2 S2 ON ((S1.ID = S2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S1_JOIN_S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "S1_JOIN_S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "S1", "S2" ], + "sink" : "S1_JOIN_S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S1_JOIN_S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT" + }, + "selectExpressions" : [ "ID AS S1_ID", "NAME AS S1_NAME", "TS AS S1_TS", "ROWTIME AS S1_ROWTIME", "ROWKEY AS S1_ROWKEY" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING" + }, + "selectExpressions" : [ "ID AS S2_ID", "F1 AS S2_F1", "F2 AS S2_F2", "ROWTIME AS S2_ROWTIME", "ROWKEY AS S2_ROWKEY" ] + } + }, + "selectExpressions" : [ "S1_ID AS ID", "S1_NAME AS NAME", "S1_TS AS TS", "S2_F1 AS F1", "S2_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S1_JOIN_S2", + "timestampColumn" : { + "column" : "TS", + "format" : null + } + }, + "queryId" : "CTAS_S1_JOIN_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/tmp/confluent8609378800848380873", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_PROTOBUF/6.0.0_1582734337353/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_PROTOBUF/6.0.0_1582734337353/spec.json new file mode 100644 index 000000000000..285654b5e92d --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_PROTOBUF/6.0.0_1582734337353/spec.json @@ -0,0 +1,89 @@ +{ + "version" : "6.0.0", + "timestamp" : 1582734337353, + "schemas" : { + "CTAS_S1_JOIN_S2_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CTAS_S1_JOIN_S2_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CTAS_S1_JOIN_S2_0.S1_JOIN_S2" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 0, + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 10000 + }, { + "topic" : "s2", + "key" : 10, + "value" : { + "ID" : 10, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 13000 + }, { + "topic" : "s1", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 19000 + }, + "timestamp" : 22000 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 18000 + }, + "timestamp" : 33000 + } ], + "outputs" : [ { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 0 + }, { + "topic" : "S1_JOIN_S2", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 19000, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 19000 + }, { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 18000, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 18000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_PROTOBUF/6.0.0_1582734337353/topology b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_PROTOBUF/6.0.0_1582734337353/topology new file mode 100644 index 000000000000..3f5c3b5480d2 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_-_PROTOBUF/6.0.0_1582734337353/topology @@ -0,0 +1,45 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [s1]) + --> KTABLE-SOURCE-0000000001 + Source: KSTREAM-SOURCE-0000000004 (topics: [s2]) + --> KTABLE-SOURCE-0000000005 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-SOURCE-0000000005 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000006 + <-- KSTREAM-SOURCE-0000000004 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasLeft + <-- KTABLE-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000006 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000005 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000009 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000010 + <-- KTABLE-TRANSFORMVALUES-0000000006 + Processor: KTABLE-JOINOTHER-0000000010 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000008 (stores: []) + --> Project + <-- KTABLE-JOINTHIS-0000000009, KTABLE-JOINOTHER-0000000010 + Processor: Project (stores: []) + --> KTABLE-TOSTREAM-0000000012 + <-- KTABLE-MERGE-0000000008 + Processor: KTABLE-TOSTREAM-0000000012 (stores: []) + --> ApplyTimestampTransform-S1_JOIN_S2 + <-- Project + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000013 + <-- KTABLE-TOSTREAM-0000000012 + Sink: KSTREAM-SINK-0000000013 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734337396/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734337396/plan.json new file mode 100644 index 000000000000..d702ec41c4b8 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734337396/plan.json @@ -0,0 +1,228 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S1 (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, TS BIGINT) WITH (KAFKA_TOPIC='s1', KEY='ID', TIMESTAMP='TS', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S1", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 (ROWKEY BIGINT KEY, ID BIGINT, F1 STRING, F2 STRING, RTS BIGINT) WITH (KAFKA_TOPIC='s2', KEY='ID', TIMESTAMP='RTS', VALUE_FORMAT='AVRO');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING, `RTS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S1_JOIN_S2 WITH (TIMESTAMP='TS') AS SELECT\n S1.ID ID,\n S1.NAME NAME,\n S1.TS TS,\n S2.F1 F1,\n S2.F2 F2\nFROM S1 S1\nINNER JOIN S2 S2 ON ((S1.ID = S2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S1_JOIN_S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "S1_JOIN_S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "S1", "S2" ], + "sink" : "S1_JOIN_S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S1_JOIN_S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT" + }, + "selectExpressions" : [ "ID AS S1_ID", "NAME AS S1_NAME", "TS AS S1_TS", "ROWTIME AS S1_ROWTIME", "ROWKEY AS S1_ROWKEY" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING, `RTS` BIGINT" + }, + "selectExpressions" : [ "ID AS S2_ID", "F1 AS S2_F1", "F2 AS S2_F2", "RTS AS S2_RTS", "ROWTIME AS S2_ROWTIME", "ROWKEY AS S2_ROWKEY" ] + } + }, + "selectExpressions" : [ "S1_ID AS ID", "S1_NAME AS NAME", "S1_TS AS TS", "S2_F1 AS F1", "S2_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "AVRO", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S1_JOIN_S2", + "timestampColumn" : { + "column" : "TS", + "format" : null + } + }, + "queryId" : "CTAS_S1_JOIN_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/tmp/confluent8609378800848380873", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734337396/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734337396/spec.json new file mode 100644 index 000000000000..50c4031e53df --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734337396/spec.json @@ -0,0 +1,91 @@ +{ + "version" : "6.0.0", + "timestamp" : 1582734337396, + "schemas" : { + "CTAS_S1_JOIN_S2_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CTAS_S1_JOIN_S2_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CTAS_S1_JOIN_S2_0.S1_JOIN_S2" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 0, + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : "foo", + "RTS" : 10000 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 10, + "value" : { + "ID" : 10, + "F1" : "foo", + "F2" : "bar", + "RTS" : 13000 + }, + "timestamp" : 0 + }, { + "topic" : "s1", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000 + }, + "timestamp" : 0 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000 + }, + "timestamp" : 0 + } ], + "outputs" : [ { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 0 + }, { + "topic" : "S1_JOIN_S2", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 11000 + }, { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 8000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734337396/topology b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734337396/topology new file mode 100644 index 000000000000..3f5c3b5480d2 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_AVRO/6.0.0_1582734337396/topology @@ -0,0 +1,45 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [s1]) + --> KTABLE-SOURCE-0000000001 + Source: KSTREAM-SOURCE-0000000004 (topics: [s2]) + --> KTABLE-SOURCE-0000000005 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-SOURCE-0000000005 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000006 + <-- KSTREAM-SOURCE-0000000004 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasLeft + <-- KTABLE-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000006 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000005 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000009 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000010 + <-- KTABLE-TRANSFORMVALUES-0000000006 + Processor: KTABLE-JOINOTHER-0000000010 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000008 (stores: []) + --> Project + <-- KTABLE-JOINTHIS-0000000009, KTABLE-JOINOTHER-0000000010 + Processor: Project (stores: []) + --> KTABLE-TOSTREAM-0000000012 + <-- KTABLE-MERGE-0000000008 + Processor: KTABLE-TOSTREAM-0000000012 (stores: []) + --> ApplyTimestampTransform-S1_JOIN_S2 + <-- Project + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000013 + <-- KTABLE-TOSTREAM-0000000012 + Sink: KSTREAM-SINK-0000000013 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734337431/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734337431/plan.json new file mode 100644 index 000000000000..d12b44c04977 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734337431/plan.json @@ -0,0 +1,228 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S1 (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, TS BIGINT) WITH (KAFKA_TOPIC='s1', KEY='ID', TIMESTAMP='TS', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S1", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 (ROWKEY BIGINT KEY, ID BIGINT, F1 STRING, F2 STRING, RTS BIGINT) WITH (KAFKA_TOPIC='s2', KEY='ID', TIMESTAMP='RTS', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING, `RTS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S1_JOIN_S2 WITH (TIMESTAMP='TS') AS SELECT\n S1.ID ID,\n S1.NAME NAME,\n S1.TS TS,\n S2.F1 F1,\n S2.F2 F2\nFROM S1 S1\nINNER JOIN S2 S2 ON ((S1.ID = S2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S1_JOIN_S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "S1_JOIN_S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "S1", "S2" ], + "sink" : "S1_JOIN_S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S1_JOIN_S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT" + }, + "selectExpressions" : [ "ID AS S1_ID", "NAME AS S1_NAME", "TS AS S1_TS", "ROWTIME AS S1_ROWTIME", "ROWKEY AS S1_ROWKEY" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING, `RTS` BIGINT" + }, + "selectExpressions" : [ "ID AS S2_ID", "F1 AS S2_F1", "F2 AS S2_F2", "RTS AS S2_RTS", "ROWTIME AS S2_ROWTIME", "ROWKEY AS S2_ROWKEY" ] + } + }, + "selectExpressions" : [ "S1_ID AS ID", "S1_NAME AS NAME", "S1_TS AS TS", "S2_F1 AS F1", "S2_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S1_JOIN_S2", + "timestampColumn" : { + "column" : "TS", + "format" : null + } + }, + "queryId" : "CTAS_S1_JOIN_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/tmp/confluent8609378800848380873", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734337431/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734337431/spec.json new file mode 100644 index 000000000000..691024818fb2 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734337431/spec.json @@ -0,0 +1,91 @@ +{ + "version" : "6.0.0", + "timestamp" : 1582734337431, + "schemas" : { + "CTAS_S1_JOIN_S2_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CTAS_S1_JOIN_S2_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CTAS_S1_JOIN_S2_0.S1_JOIN_S2" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 0, + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : "foo", + "RTS" : 10000 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 10, + "value" : { + "ID" : 10, + "F1" : "foo", + "F2" : "bar", + "RTS" : 13000 + }, + "timestamp" : 0 + }, { + "topic" : "s1", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000 + }, + "timestamp" : 0 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000 + }, + "timestamp" : 0 + } ], + "outputs" : [ { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 0 + }, { + "topic" : "S1_JOIN_S2", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 11000 + }, { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 8000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734337431/topology b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734337431/topology new file mode 100644 index 000000000000..3f5c3b5480d2 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_JSON/6.0.0_1582734337431/topology @@ -0,0 +1,45 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [s1]) + --> KTABLE-SOURCE-0000000001 + Source: KSTREAM-SOURCE-0000000004 (topics: [s2]) + --> KTABLE-SOURCE-0000000005 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-SOURCE-0000000005 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000006 + <-- KSTREAM-SOURCE-0000000004 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasLeft + <-- KTABLE-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000006 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000005 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000009 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000010 + <-- KTABLE-TRANSFORMVALUES-0000000006 + Processor: KTABLE-JOINOTHER-0000000010 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000008 (stores: []) + --> Project + <-- KTABLE-JOINTHIS-0000000009, KTABLE-JOINOTHER-0000000010 + Processor: Project (stores: []) + --> KTABLE-TOSTREAM-0000000012 + <-- KTABLE-MERGE-0000000008 + Processor: KTABLE-TOSTREAM-0000000012 (stores: []) + --> ApplyTimestampTransform-S1_JOIN_S2 + <-- Project + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000013 + <-- KTABLE-TOSTREAM-0000000012 + Sink: KSTREAM-SINK-0000000013 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337502/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337502/plan.json new file mode 100644 index 000000000000..f09c08ac028d --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337502/plan.json @@ -0,0 +1,228 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S1 (ROWKEY BIGINT KEY, ID BIGINT, NAME STRING, TS BIGINT) WITH (KAFKA_TOPIC='s1', KEY='ID', TIMESTAMP='TS', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S1", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S2 (ROWKEY BIGINT KEY, ID BIGINT, F1 STRING, F2 STRING, RTS BIGINT) WITH (KAFKA_TOPIC='s2', KEY='ID', TIMESTAMP='RTS', VALUE_FORMAT='PROTOBUF');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING, `RTS` BIGINT", + "keyField" : "ID", + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE S1_JOIN_S2 WITH (TIMESTAMP='TS') AS SELECT\n S1.ID ID,\n S1.NAME NAME,\n S1.TS TS,\n S2.F1 F1,\n S2.F2 F2\nFROM S1 S1\nINNER JOIN S2 S2 ON ((S1.ID = S2.ID))\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "S1_JOIN_S2", + "schema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT, `F1` STRING, `F2` STRING", + "keyField" : "ID", + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "topicName" : "S1_JOIN_S2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "S1", "S2" ], + "sink" : "S1_JOIN_S2", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "S1_JOIN_S2" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableTableJoinV1", + "properties" : { + "queryContext" : "Join" + }, + "joinType" : "INNER", + "leftSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasLeft" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Left/Source" + }, + "topicName" : "s1", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "TS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `NAME` STRING, `TS` BIGINT" + }, + "selectExpressions" : [ "ID AS S1_ID", "NAME AS S1_NAME", "TS AS S1_TS", "ROWTIME AS S1_ROWTIME", "ROWKEY AS S1_ROWKEY" ] + }, + "rightSource" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "PrependAliasRight" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KafkaTopic_Right/Source" + }, + "topicName" : "s2", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "RTS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` BIGINT KEY, `ID` BIGINT, `F1` STRING, `F2` STRING, `RTS` BIGINT" + }, + "selectExpressions" : [ "ID AS S2_ID", "F1 AS S2_F1", "F2 AS S2_F2", "RTS AS S2_RTS", "ROWTIME AS S2_ROWTIME", "ROWKEY AS S2_ROWKEY" ] + } + }, + "selectExpressions" : [ "S1_ID AS ID", "S1_NAME AS NAME", "S1_TS AS TS", "S2_F1 AS F1", "S2_F2 AS F2" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "PROTOBUF", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "S1_JOIN_S2", + "timestampColumn" : { + "column" : "TS", + "format" : null + } + }, + "queryId" : "CTAS_S1_JOIN_S2_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/tmp/confluent8609378800848380873", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337502/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337502/spec.json new file mode 100644 index 000000000000..a5053dbe1ebb --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337502/spec.json @@ -0,0 +1,91 @@ +{ + "version" : "6.0.0", + "timestamp" : 1582734337502, + "schemas" : { + "CTAS_S1_JOIN_S2_0.KafkaTopic_Left.Source" : "STRUCT NOT NULL", + "CTAS_S1_JOIN_S2_0.KafkaTopic_Right.Source" : "STRUCT NOT NULL", + "CTAS_S1_JOIN_S2_0.S1_JOIN_S2" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 0, + "value" : { + "ID" : 0, + "F1" : "blah", + "F2" : "foo", + "RTS" : 10000 + }, + "timestamp" : 0 + }, { + "topic" : "s2", + "key" : 10, + "value" : { + "ID" : 10, + "F1" : "foo", + "F2" : "bar", + "RTS" : 13000 + }, + "timestamp" : 0 + }, { + "topic" : "s1", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000 + }, + "timestamp" : 0 + }, { + "topic" : "s1", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000 + }, + "timestamp" : 0 + } ], + "outputs" : [ { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "zero", + "TS" : 0, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 0 + }, { + "topic" : "S1_JOIN_S2", + "key" : 10, + "value" : { + "ID" : 10, + "NAME" : "100", + "TS" : 11000, + "F1" : "foo", + "F2" : "bar" + }, + "timestamp" : 11000 + }, { + "topic" : "S1_JOIN_S2", + "key" : 0, + "value" : { + "ID" : 0, + "NAME" : "jan", + "TS" : 8000, + "F1" : "blah", + "F2" : "foo" + }, + "timestamp" : 8000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337502/topology b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337502/topology new file mode 100644 index 000000000000..3f5c3b5480d2 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/join-with-custom-timestamp_-_table_table_inner_join_with_ts_extractor_both_sides_-_PROTOBUF/6.0.0_1582734337502/topology @@ -0,0 +1,45 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [s1]) + --> KTABLE-SOURCE-0000000001 + Source: KSTREAM-SOURCE-0000000004 (topics: [s2]) + --> KTABLE-SOURCE-0000000005 + Processor: KTABLE-SOURCE-0000000001 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-SOURCE-0000000005 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000006 + <-- KSTREAM-SOURCE-0000000004 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> PrependAliasLeft + <-- KTABLE-SOURCE-0000000001 + Processor: KTABLE-TRANSFORMVALUES-0000000006 (stores: []) + --> PrependAliasRight + <-- KTABLE-SOURCE-0000000005 + Processor: PrependAliasLeft (stores: []) + --> KTABLE-JOINTHIS-0000000009 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: PrependAliasRight (stores: []) + --> KTABLE-JOINOTHER-0000000010 + <-- KTABLE-TRANSFORMVALUES-0000000006 + Processor: KTABLE-JOINOTHER-0000000010 (stores: [KafkaTopic_Left-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasRight + Processor: KTABLE-JOINTHIS-0000000009 (stores: [KafkaTopic_Right-Reduce]) + --> KTABLE-MERGE-0000000008 + <-- PrependAliasLeft + Processor: KTABLE-MERGE-0000000008 (stores: []) + --> Project + <-- KTABLE-JOINTHIS-0000000009, KTABLE-JOINOTHER-0000000010 + Processor: Project (stores: []) + --> KTABLE-TOSTREAM-0000000012 + <-- KTABLE-MERGE-0000000008 + Processor: KTABLE-TOSTREAM-0000000012 (stores: []) + --> ApplyTimestampTransform-S1_JOIN_S2 + <-- Project + Processor: ApplyTimestampTransform-S1_JOIN_S2 (stores: []) + --> KSTREAM-SINK-0000000013 + <-- KTABLE-TOSTREAM-0000000012 + Sink: KSTREAM-SINK-0000000013 (topic: S1_JOIN_S2) + <-- ApplyTimestampTransform-S1_JOIN_S2 + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_Invalid_timestamp_value_should_throw_an_exception/6.0.0_1582734342043/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_Invalid_timestamp_value_should_throw_an_exception/6.0.0_1582734342043/plan.json new file mode 100644 index 000000000000..031024465e29 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_Invalid_timestamp_value_should_throw_an_exception/6.0.0_1582734342043/plan.json @@ -0,0 +1,151 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT, EVENT_TS STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `ID` BIGINT, `EVENT_TS` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TS WITH (TIMESTAMP='event_ts', TIMESTAMP_FORMAT='yyyy-MM-dd''T''HH:mm:ssX') AS SELECT\n TEST.ID ID,\n TEST.EVENT_TS EVENT_TS\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TS", + "schema" : "`ROWKEY` STRING KEY, `ID` BIGINT, `EVENT_TS` STRING", + "keyField" : null, + "timestampColumn" : { + "column" : "EVENT_TS", + "format" : "yyyy-MM-dd'T'HH:mm:ssX" + }, + "topicName" : "TS", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "TS", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "TS" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `ID` BIGINT, `EVENT_TS` STRING" + }, + "selectExpressions" : [ "ID AS ID", "EVENT_TS AS EVENT_TS" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "TS", + "timestampColumn" : { + "column" : "EVENT_TS", + "format" : "yyyy-MM-dd'T'HH:mm:ssX" + } + }, + "queryId" : "CSAS_TS_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/tmp/confluent8609378800848380873", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_Invalid_timestamp_value_should_throw_an_exception/6.0.0_1582734342043/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_Invalid_timestamp_value_should_throw_an_exception/6.0.0_1582734342043/spec.json new file mode 100644 index 000000000000..72e54279abda --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_Invalid_timestamp_value_should_throw_an_exception/6.0.0_1582734342043/spec.json @@ -0,0 +1,17 @@ +{ + "version" : "6.0.0", + "timestamp" : 1582734342043, + "schemas" : { + "CSAS_TS_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_TS_0.TS" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "", + "value" : { + "ID" : 1, + "EVENT_TS" : "not a timestamp" + } + } ], + "outputs" : [ ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_Invalid_timestamp_value_should_throw_an_exception/6.0.0_1582734342043/topology b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_Invalid_timestamp_value_should_throw_an_exception/6.0.0_1582734342043/topology new file mode 100644 index 000000000000..3b938b733ae6 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_Invalid_timestamp_value_should_throw_an_exception/6.0.0_1582734342043/topology @@ -0,0 +1,16 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> ApplyTimestampTransform-TS + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: ApplyTimestampTransform-TS (stores: []) + --> KSTREAM-SINK-0000000003 + <-- Project + Sink: KSTREAM-SINK-0000000003 (topic: TS) + <-- ApplyTimestampTransform-TS + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CSAS/6.0.0_1582734341981/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CSAS/6.0.0_1582734341981/plan.json new file mode 100644 index 000000000000..8e72067da1e6 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CSAS/6.0.0_1582734341981/plan.json @@ -0,0 +1,151 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `ID` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TS WITH (TIMESTAMP='sink_ts') AS SELECT\n TEST.ID SINK_TS,\n TEST.ID ID\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TS", + "schema" : "`ROWKEY` STRING KEY, `SINK_TS` BIGINT, `ID` BIGINT", + "keyField" : null, + "timestampColumn" : { + "column" : "SINK_TS", + "format" : null + }, + "topicName" : "TS", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "TS", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "TS" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `ID` BIGINT" + }, + "selectExpressions" : [ "ID AS SINK_TS", "ID AS ID" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "TS", + "timestampColumn" : { + "column" : "SINK_TS", + "format" : null + } + }, + "queryId" : "CSAS_TS_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/tmp/confluent8609378800848380873", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CSAS/6.0.0_1582734341981/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CSAS/6.0.0_1582734341981/spec.json new file mode 100644 index 000000000000..e7fcb2aa8b25 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CSAS/6.0.0_1582734341981/spec.json @@ -0,0 +1,47 @@ +{ + "version" : "6.0.0", + "timestamp" : 1582734341981, + "schemas" : { + "CSAS_TS_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_TS_0.TS" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "", + "value" : { + "ID" : 1 + }, + "timestamp" : 1526075913000 + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "ID" : -2 + }, + "timestamp" : 1526075913000 + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "ID" : 3 + }, + "timestamp" : 1589234313000 + } ], + "outputs" : [ { + "topic" : "TS", + "key" : "", + "value" : { + "SINK_TS" : 1, + "ID" : 1 + }, + "timestamp" : 1 + }, { + "topic" : "TS", + "key" : "", + "value" : { + "SINK_TS" : 3, + "ID" : 3 + }, + "timestamp" : 3 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CSAS/6.0.0_1582734341981/topology b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CSAS/6.0.0_1582734341981/topology new file mode 100644 index 000000000000..3b938b733ae6 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CSAS/6.0.0_1582734341981/topology @@ -0,0 +1,16 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> ApplyTimestampTransform-TS + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: ApplyTimestampTransform-TS (stores: []) + --> KSTREAM-SINK-0000000003 + <-- Project + Sink: KSTREAM-SINK-0000000003 (topic: TS) + <-- ApplyTimestampTransform-TS + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CSAS_using_a_string_TIMESTAMP_FORMAT/6.0.0_1582734341990/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CSAS_using_a_string_TIMESTAMP_FORMAT/6.0.0_1582734341990/plan.json new file mode 100644 index 000000000000..031024465e29 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CSAS_using_a_string_TIMESTAMP_FORMAT/6.0.0_1582734341990/plan.json @@ -0,0 +1,151 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT, EVENT_TS STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `ID` BIGINT, `EVENT_TS` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TS WITH (TIMESTAMP='event_ts', TIMESTAMP_FORMAT='yyyy-MM-dd''T''HH:mm:ssX') AS SELECT\n TEST.ID ID,\n TEST.EVENT_TS EVENT_TS\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TS", + "schema" : "`ROWKEY` STRING KEY, `ID` BIGINT, `EVENT_TS` STRING", + "keyField" : null, + "timestampColumn" : { + "column" : "EVENT_TS", + "format" : "yyyy-MM-dd'T'HH:mm:ssX" + }, + "topicName" : "TS", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "TS", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "TS" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `ID` BIGINT, `EVENT_TS` STRING" + }, + "selectExpressions" : [ "ID AS ID", "EVENT_TS AS EVENT_TS" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "TS", + "timestampColumn" : { + "column" : "EVENT_TS", + "format" : "yyyy-MM-dd'T'HH:mm:ssX" + } + }, + "queryId" : "CSAS_TS_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/tmp/confluent8609378800848380873", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CSAS_using_a_string_TIMESTAMP_FORMAT/6.0.0_1582734341990/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CSAS_using_a_string_TIMESTAMP_FORMAT/6.0.0_1582734341990/spec.json new file mode 100644 index 000000000000..aff39a0ebf67 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CSAS_using_a_string_TIMESTAMP_FORMAT/6.0.0_1582734341990/spec.json @@ -0,0 +1,50 @@ +{ + "version" : "6.0.0", + "timestamp" : 1582734341990, + "schemas" : { + "CSAS_TS_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_TS_0.TS" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "", + "value" : { + "ID" : 1, + "EVENT_TS" : "2018-05-11T21:58:33Z" + }, + "timestamp" : 10 + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "ID" : 2, + "EVENT_TS" : "not a timestamp" + }, + "timestamp" : 10 + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "ID" : 3, + "EVENT_TS" : "2019-05-11T21:58:33Z" + }, + "timestamp" : 10 + } ], + "outputs" : [ { + "topic" : "TS", + "key" : "", + "value" : { + "ID" : 1, + "EVENT_TS" : "2018-05-11T21:58:33Z" + }, + "timestamp" : 1526075913000 + }, { + "topic" : "TS", + "key" : "", + "value" : { + "ID" : 3, + "EVENT_TS" : "2019-05-11T21:58:33Z" + }, + "timestamp" : 1557611913000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CSAS_using_a_string_TIMESTAMP_FORMAT/6.0.0_1582734341990/topology b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CSAS_using_a_string_TIMESTAMP_FORMAT/6.0.0_1582734341990/topology new file mode 100644 index 000000000000..3b938b733ae6 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CSAS_using_a_string_TIMESTAMP_FORMAT/6.0.0_1582734341990/topology @@ -0,0 +1,16 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> ApplyTimestampTransform-TS + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: ApplyTimestampTransform-TS (stores: []) + --> KSTREAM-SINK-0000000003 + <-- Project + Sink: KSTREAM-SINK-0000000003 (topic: TS) + <-- ApplyTimestampTransform-TS + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CTAS/6.0.0_1582734341998/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CTAS/6.0.0_1582734341998/plan.json new file mode 100644 index 000000000000..bc17319d0937 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CTAS/6.0.0_1582734341998/plan.json @@ -0,0 +1,151 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TEST (ID BIGINT) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `ID` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE TABLE TS WITH (TIMESTAMP='sink_ts') AS SELECT\n TEST.ID SINK_TS,\n TEST.ID ID\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createTableV1", + "sourceName" : "TS", + "schema" : "`ROWKEY` STRING KEY, `SINK_TS` BIGINT, `ID` BIGINT", + "keyField" : null, + "timestampColumn" : { + "column" : "SINK_TS", + "format" : null + }, + "topicName" : "TS", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "TS", + "physicalPlan" : { + "@type" : "tableSinkV1", + "properties" : { + "queryContext" : "TS" + }, + "source" : { + "@type" : "tableSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "tableSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `ID` BIGINT" + }, + "selectExpressions" : [ "ID AS SINK_TS", "ID AS ID" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "TS", + "timestampColumn" : { + "column" : "SINK_TS", + "format" : null + } + }, + "queryId" : "CTAS_TS_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/tmp/confluent8609378800848380873", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CTAS/6.0.0_1582734341998/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CTAS/6.0.0_1582734341998/spec.json new file mode 100644 index 000000000000..523f8be0a7a2 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CTAS/6.0.0_1582734341998/spec.json @@ -0,0 +1,47 @@ +{ + "version" : "6.0.0", + "timestamp" : 1582734341998, + "schemas" : { + "CTAS_TS_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CTAS_TS_0.TS" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "", + "value" : { + "ID" : 1 + }, + "timestamp" : 1526075913000 + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "ID" : -2 + }, + "timestamp" : 1526075913000 + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "ID" : 3 + }, + "timestamp" : 1589234313000 + } ], + "outputs" : [ { + "topic" : "TS", + "key" : "", + "value" : { + "SINK_TS" : 1, + "ID" : 1 + }, + "timestamp" : 1 + }, { + "topic" : "TS", + "key" : "", + "value" : { + "SINK_TS" : 3, + "ID" : 3 + }, + "timestamp" : 3 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CTAS/6.0.0_1582734341998/topology b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CTAS/6.0.0_1582734341998/topology new file mode 100644 index 000000000000..d7fce4d8d45f --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CTAS/6.0.0_1582734341998/topology @@ -0,0 +1,22 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KTABLE-SOURCE-0000000001 + Processor: KTABLE-SOURCE-0000000001 (stores: [KsqlTopic-Reduce]) + --> KTABLE-TRANSFORMVALUES-0000000002 + <-- KSTREAM-SOURCE-0000000000 + Processor: KTABLE-TRANSFORMVALUES-0000000002 (stores: []) + --> Project + <-- KTABLE-SOURCE-0000000001 + Processor: Project (stores: []) + --> KTABLE-TOSTREAM-0000000004 + <-- KTABLE-TRANSFORMVALUES-0000000002 + Processor: KTABLE-TOSTREAM-0000000004 (stores: []) + --> ApplyTimestampTransform-TS + <-- Project + Processor: ApplyTimestampTransform-TS (stores: []) + --> KSTREAM-SINK-0000000005 + <-- KTABLE-TOSTREAM-0000000004 + Sink: KSTREAM-SINK-0000000005 (topic: TS) + <-- ApplyTimestampTransform-TS + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CTAS_using_a_string_TIMESTAMP_FORMAT/6.0.0_1582734342008/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CTAS_using_a_string_TIMESTAMP_FORMAT/6.0.0_1582734342008/plan.json new file mode 100644 index 000000000000..031024465e29 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CTAS_using_a_string_TIMESTAMP_FORMAT/6.0.0_1582734342008/plan.json @@ -0,0 +1,151 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TEST (ID BIGINT, EVENT_TS STRING) WITH (KAFKA_TOPIC='test_topic', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TEST", + "schema" : "`ROWKEY` STRING KEY, `ID` BIGINT, `EVENT_TS` STRING", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM TS WITH (TIMESTAMP='event_ts', TIMESTAMP_FORMAT='yyyy-MM-dd''T''HH:mm:ssX') AS SELECT\n TEST.ID ID,\n TEST.EVENT_TS EVENT_TS\nFROM TEST TEST\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "TS", + "schema" : "`ROWKEY` STRING KEY, `ID` BIGINT, `EVENT_TS` STRING", + "keyField" : null, + "timestampColumn" : { + "column" : "EVENT_TS", + "format" : "yyyy-MM-dd'T'HH:mm:ssX" + }, + "topicName" : "TS", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "TEST" ], + "sink" : "TS", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "TS" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : null, + "sourceSchema" : "`ROWKEY` STRING KEY, `ID` BIGINT, `EVENT_TS` STRING" + }, + "selectExpressions" : [ "ID AS ID", "EVENT_TS AS EVENT_TS" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "TS", + "timestampColumn" : { + "column" : "EVENT_TS", + "format" : "yyyy-MM-dd'T'HH:mm:ssX" + } + }, + "queryId" : "CSAS_TS_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/tmp/confluent8609378800848380873", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CTAS_using_a_string_TIMESTAMP_FORMAT/6.0.0_1582734342008/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CTAS_using_a_string_TIMESTAMP_FORMAT/6.0.0_1582734342008/spec.json new file mode 100644 index 000000000000..73945ca986b8 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CTAS_using_a_string_TIMESTAMP_FORMAT/6.0.0_1582734342008/spec.json @@ -0,0 +1,50 @@ +{ + "version" : "6.0.0", + "timestamp" : 1582734342008, + "schemas" : { + "CSAS_TS_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_TS_0.TS" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "", + "value" : { + "ID" : 1, + "EVENT_TS" : "2018-05-11T21:58:33Z" + }, + "timestamp" : 10 + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "ID" : 2, + "EVENT_TS" : "not a timestamp" + }, + "timestamp" : 10 + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "ID" : 3, + "EVENT_TS" : "2019-05-11T21:58:33Z" + }, + "timestamp" : 10 + } ], + "outputs" : [ { + "topic" : "TS", + "key" : "", + "value" : { + "ID" : 1, + "EVENT_TS" : "2018-05-11T21:58:33Z" + }, + "timestamp" : 1526075913000 + }, { + "topic" : "TS", + "key" : "", + "value" : { + "ID" : 3, + "EVENT_TS" : "2019-05-11T21:58:33Z" + }, + "timestamp" : 1557611913000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CTAS_using_a_string_TIMESTAMP_FORMAT/6.0.0_1582734342008/topology b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CTAS_using_a_string_TIMESTAMP_FORMAT/6.0.0_1582734342008/topology new file mode 100644 index 000000000000..3b938b733ae6 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_override_output_timestamp_for_CTAS_using_a_string_TIMESTAMP_FORMAT/6.0.0_1582734342008/topology @@ -0,0 +1,16 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> ApplyTimestampTransform-TS + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Processor: ApplyTimestampTransform-TS (stores: []) + --> KSTREAM-SINK-0000000003 + <-- Project + Sink: KSTREAM-SINK-0000000003 (topic: TS) + <-- ApplyTimestampTransform-TS + diff --git a/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_timestamp_column_of_source_should_not_influence_sink/6.0.0_1582734342024/plan.json b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_timestamp_column_of_source_should_not_influence_sink/6.0.0_1582734342024/plan.json new file mode 100644 index 000000000000..4e30a6c88c54 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_timestamp_column_of_source_should_not_influence_sink/6.0.0_1582734342024/plan.json @@ -0,0 +1,151 @@ +{ + "plan" : [ { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM INPUT (ID BIGINT, EVENT_TS BIGINT) WITH (KAFKA_TOPIC='test_topic', TIMESTAMP='EVENT_TS', VALUE_FORMAT='JSON');", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "INPUT", + "schema" : "`ROWKEY` STRING KEY, `ID` BIGINT, `EVENT_TS` BIGINT", + "keyField" : null, + "timestampColumn" : { + "column" : "EVENT_TS", + "format" : null + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : null + }, { + "@type" : "ksqlPlanV1", + "statementText" : "CREATE STREAM OUTPUT AS SELECT INPUT.ID EVENT_TS\nFROM INPUT INPUT\nEMIT CHANGES", + "ddlCommand" : { + "@type" : "createStreamV1", + "sourceName" : "OUTPUT", + "schema" : "`ROWKEY` STRING KEY, `EVENT_TS` BIGINT", + "keyField" : null, + "timestampColumn" : null, + "topicName" : "OUTPUT", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "windowInfo" : null + }, + "queryPlan" : { + "sources" : [ "INPUT" ], + "sink" : "OUTPUT", + "physicalPlan" : { + "@type" : "streamSinkV1", + "properties" : { + "queryContext" : "OUTPUT" + }, + "source" : { + "@type" : "streamSelectV1", + "properties" : { + "queryContext" : "Project" + }, + "source" : { + "@type" : "streamSourceV1", + "properties" : { + "queryContext" : "KsqlTopic/Source" + }, + "topicName" : "test_topic", + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "timestampColumn" : { + "column" : "EVENT_TS", + "format" : null + }, + "sourceSchema" : "`ROWKEY` STRING KEY, `ID` BIGINT, `EVENT_TS` BIGINT" + }, + "selectExpressions" : [ "ID AS EVENT_TS" ] + }, + "formats" : { + "keyFormat" : { + "format" : "KAFKA", + "properties" : { } + }, + "valueFormat" : { + "format" : "JSON", + "properties" : { } + }, + "options" : [ ] + }, + "topicName" : "OUTPUT", + "timestampColumn" : null + }, + "queryId" : "CSAS_OUTPUT_0" + } + } ], + "configs" : { + "ksql.extension.dir" : "ext", + "ksql.streams.cache.max.bytes.buffering" : "0", + "ksql.security.extension.class" : null, + "ksql.transient.prefix" : "transient_", + "ksql.persistence.wrap.single.values" : "true", + "ksql.authorization.cache.expiry.time.secs" : "30", + "ksql.schema.registry.url" : "", + "ksql.streams.default.deserialization.exception.handler" : "io.confluent.ksql.errors.LogMetricAndContinueExceptionHandler", + "ksql.output.topic.name.prefix" : "", + "ksql.streams.auto.offset.reset" : "earliest", + "ksql.query.pull.enable.standby.reads" : "false", + "ksql.connect.url" : "http://localhost:8083", + "ksql.service.id" : "some.ksql.service.id", + "ksql.internal.topic.min.insync.replicas" : "1", + "ksql.streams.shutdown.timeout.ms" : "300000", + "ksql.new.api.enabled" : "false", + "ksql.streams.state.dir" : "/tmp/confluent8609378800848380873", + "ksql.internal.topic.replicas" : "1", + "ksql.insert.into.values.enabled" : "true", + "ksql.query.pull.max.allowed.offset.lag" : "9223372036854775807", + "ksql.streams.default.production.exception.handler" : "io.confluent.ksql.errors.ProductionExceptionHandlerUtil$LogAndFailProductionExceptionHandler", + "ksql.access.validator.enable" : "auto", + "ksql.streams.bootstrap.servers" : "localhost:0", + "ksql.streams.commit.interval.ms" : "2000", + "ksql.metric.reporters" : "", + "ksql.streams.auto.commit.interval.ms" : "0", + "ksql.metrics.extension" : null, + "ksql.streams.topology.optimization" : "all", + "ksql.query.pull.streamsstore.rebalancing.timeout.ms" : "10000", + "ksql.hidden.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.streams.num.stream.threads" : "4", + "ksql.authorization.cache.max.entries" : "10000", + "ksql.metrics.tags.custom" : "", + "ksql.pull.queries.enable" : "true", + "ksql.udfs.enabled" : "true", + "ksql.udf.enable.security.manager" : "true", + "ksql.connect.worker.config" : "", + "ksql.sink.window.change.log.additional.retention" : "1000000", + "ksql.readonly.topics" : "_confluent.*,__confluent.*,_schemas,__consumer_offsets,__transaction_state,connect-configs,connect-offsets,connect-status,connect-statuses", + "ksql.udf.collect.metrics" : "false", + "ksql.persistent.prefix" : "query_", + "ksql.query.persistent.active.limit" : "2147483647" + } +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_timestamp_column_of_source_should_not_influence_sink/6.0.0_1582734342024/spec.json b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_timestamp_column_of_source_should_not_influence_sink/6.0.0_1582734342024/spec.json new file mode 100644 index 000000000000..20fef1766490 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_timestamp_column_of_source_should_not_influence_sink/6.0.0_1582734342024/spec.json @@ -0,0 +1,38 @@ +{ + "version" : "6.0.0", + "timestamp" : 1582734342024, + "schemas" : { + "CSAS_OUTPUT_0.KsqlTopic.Source" : "STRUCT NOT NULL", + "CSAS_OUTPUT_0.OUTPUT" : "STRUCT NOT NULL" + }, + "inputs" : [ { + "topic" : "test_topic", + "key" : "", + "value" : { + "ID" : 1, + "EVENT_TS" : 1526075913000 + } + }, { + "topic" : "test_topic", + "key" : "", + "value" : { + "ID" : 2, + "EVENT_TS" : 1589234313000 + } + } ], + "outputs" : [ { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "EVENT_TS" : 1 + }, + "timestamp" : 1526075913000 + }, { + "topic" : "OUTPUT", + "key" : "", + "value" : { + "EVENT_TS" : 2 + }, + "timestamp" : 1589234313000 + } ] +} \ No newline at end of file diff --git a/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_timestamp_column_of_source_should_not_influence_sink/6.0.0_1582734342024/topology b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_timestamp_column_of_source_should_not_influence_sink/6.0.0_1582734342024/topology new file mode 100644 index 000000000000..441a8f282644 --- /dev/null +++ b/ksql-functional-tests/src/test/resources/historical_plans/timestampformat_-_timestamp_column_of_source_should_not_influence_sink/6.0.0_1582734342024/topology @@ -0,0 +1,13 @@ +Topologies: + Sub-topology: 0 + Source: KSTREAM-SOURCE-0000000000 (topics: [test_topic]) + --> KSTREAM-TRANSFORMVALUES-0000000001 + Processor: KSTREAM-TRANSFORMVALUES-0000000001 (stores: []) + --> Project + <-- KSTREAM-SOURCE-0000000000 + Processor: Project (stores: []) + --> KSTREAM-SINK-0000000003 + <-- KSTREAM-TRANSFORMVALUES-0000000001 + Sink: KSTREAM-SINK-0000000003 (topic: OUTPUT) + <-- Project + diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/join-with-custom-timestamp.json b/ksql-functional-tests/src/test/resources/query-validation-tests/join-with-custom-timestamp.json index 5dac6aae0560..f2ad05714258 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/join-with-custom-timestamp.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/join-with-custom-timestamp.json @@ -24,9 +24,9 @@ {"topic": "s1", "key": 0, "value": {"ID": 0, "NAME": "jan", "TS": 8000}, "timestamp": 33000} ], "outputs": [ - {"topic": "S1_JOIN_S2", "key": 0, "value": {"ID": 0, "NAME": "zero", "TS": 0, "F1": "blah", "F2": "foo"}, "timestamp": 10000}, - {"topic": "S1_JOIN_S2", "key": 10, "value": {"ID": 10, "NAME": "100", "TS": 11000, "F1": "foo", "F2": "bar"}, "timestamp": 13000}, - {"topic": "S1_JOIN_S2", "key": 0, "value": {"ID": 0, "NAME": "jan", "TS": 8000, "F1": "blah", "F2": "foo"}, "timestamp": 10000} + {"topic": "S1_JOIN_S2", "key": 0, "value": {"ID": 0, "NAME": "zero", "TS": 0, "F1": "blah", "F2": "foo"}, "timestamp": 0}, + {"topic": "S1_JOIN_S2", "key": 10, "value": {"ID": 10, "NAME": "100", "TS": 11000, "F1": "foo", "F2": "bar"}, "timestamp": 11000}, + {"topic": "S1_JOIN_S2", "key": 0, "value": {"ID": 0, "NAME": "jan", "TS": 8000, "F1": "blah", "F2": "foo"}, "timestamp": 8000} ] }, { @@ -45,9 +45,9 @@ {"topic": "s1", "key": 0, "value": {"ID": 0, "NAME": "jan", "TS": 8000}, "timestamp": 0} ], "outputs": [ - {"topic": "S1_JOIN_S2", "key": 0, "value": {"ID": 0, "NAME": "zero", "TS": 0, "F1": "blah", "F2": "foo"}, "timestamp": 10000}, - {"topic": "S1_JOIN_S2", "key": 10, "value": {"ID": 10, "NAME": "100", "TS": 11000, "F1": "foo", "F2": "bar"}, "timestamp": 13000}, - {"topic": "S1_JOIN_S2", "key": 0, "value": {"ID": 0, "NAME": "jan", "TS": 8000, "F1": "blah", "F2": "foo"}, "timestamp": 10000} + {"topic": "S1_JOIN_S2", "key": 0, "value": {"ID": 0, "NAME": "zero", "TS": 0, "F1": "blah", "F2": "foo"}, "timestamp": 0}, + {"topic": "S1_JOIN_S2", "key": 10, "value": {"ID": 10, "NAME": "100", "TS": 11000, "F1": "foo", "F2": "bar"}, "timestamp": 11000}, + {"topic": "S1_JOIN_S2", "key": 0, "value": {"ID": 0, "NAME": "jan", "TS": 8000, "F1": "blah", "F2": "foo"}, "timestamp": 8000} ] }, { @@ -87,7 +87,7 @@ {"topic": "s1", "key": 0, "value": {"ID": 0, "NAME": "jan", "TS": 18000}, "timestamp": 33000} ], "outputs": [ - {"topic": "S1_JOIN_S2", "key": 0, "value": {"ID": 0, "NAME": "zero", "TS": 0, "F1": "blah", "F2": "foo"}, "timestamp": 10000}, + {"topic": "S1_JOIN_S2", "key": 0, "value": {"ID": 0, "NAME": "zero", "TS": 0, "F1": "blah", "F2": "foo"}, "timestamp": 0}, {"topic": "S1_JOIN_S2", "key": 10, "value": {"ID": 10, "NAME": "100", "TS": 19000, "F1": "foo", "F2": "bar"}, "timestamp": 19000}, {"topic": "S1_JOIN_S2", "key": 0, "value": {"ID": 0, "NAME": "jan", "TS": 18000, "F1": "blah", "F2": "foo"}, "timestamp": 18000} ] @@ -108,9 +108,9 @@ {"topic": "s1", "key": 0, "value": {"ID": 0, "NAME": "jan", "TS": 8000}, "timestamp": 0} ], "outputs": [ - {"topic": "S1_JOIN_S2", "key": 0, "value": {"ID": 0, "NAME": "zero", "TS": 0, "F1": "blah", "F2": "foo"}, "timestamp": 10000}, - {"topic": "S1_JOIN_S2", "key": 10, "value": {"ID": 10, "NAME": "100", "TS": 11000, "F1": "foo", "F2": "bar"}, "timestamp": 13000}, - {"topic": "S1_JOIN_S2", "key": 0, "value": {"ID": 0, "NAME": "jan", "TS": 8000, "F1": "blah", "F2": "foo"}, "timestamp": 10000} + {"topic": "S1_JOIN_S2", "key": 0, "value": {"ID": 0, "NAME": "zero", "TS": 0, "F1": "blah", "F2": "foo"}, "timestamp": 0}, + {"topic": "S1_JOIN_S2", "key": 10, "value": {"ID": 10, "NAME": "100", "TS": 11000, "F1": "foo", "F2": "bar"}, "timestamp": 11000}, + {"topic": "S1_JOIN_S2", "key": 0, "value": {"ID": 0, "NAME": "jan", "TS": 8000, "F1": "blah", "F2": "foo"}, "timestamp": 8000} ] } ] diff --git a/ksql-functional-tests/src/test/resources/query-validation-tests/timestampformat.json b/ksql-functional-tests/src/test/resources/query-validation-tests/timestampformat.json index cb9d91923cfe..2675d2e2443b 100644 --- a/ksql-functional-tests/src/test/resources/query-validation-tests/timestampformat.json +++ b/ksql-functional-tests/src/test/resources/query-validation-tests/timestampformat.json @@ -37,6 +37,117 @@ "outputs": [ {"topic": "TS", "value": {"ETS": 1566912669200}, "timestamp": 1566912669200} ] + }, + { + "name": "override output timestamp for CSAS", + "statements": [ + "CREATE STREAM TEST (ID bigint) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM TS WITH (timestamp='sink_ts') AS SELECT id as sink_ts, id FROM test;" + ], + "inputs": [ + {"topic": "test_topic", "value": {"ID": 1}, "timestamp": 1526075913000}, + {"topic": "test_topic", "value": {"ID": -2}, "timestamp": 1526075913000}, + {"topic": "test_topic", "value": {"ID": 3}, "timestamp": 1589234313000} + ], + "outputs": [ + {"topic": "TS", "value": {"SINK_TS":1, "ID": 1}, "timestamp": 1}, + {"topic": "TS", "value": {"SINK_TS":3, "ID": 3}, "timestamp": 3} + ] + }, + { + "name": "override output timestamp for CSAS using a string TIMESTAMP_FORMAT", + "statements": [ + "CREATE STREAM TEST (ID bigint, EVENT_TS varchar) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM TS WITH (timestamp='event_ts', timestamp_format='yyyy-MM-dd''T''HH:mm:ssX') AS SELECT id, event_ts FROM test;" + ], + "inputs": [ + {"topic": "test_topic", "value": {"ID": 1, "EVENT_TS": "2018-05-11T21:58:33Z"}, "timestamp": 10}, + {"topic": "test_topic", "value": {"ID": 2, "EVENT_TS": "not a timestamp"}, "timestamp": 10}, + {"topic": "test_topic", "value": {"ID": 3, "EVENT_TS": "2019-05-11T21:58:33Z"}, "timestamp": 10} + ], + "outputs": [ + {"topic": "TS", "value": {"ID": 1, "EVENT_TS": "2018-05-11T21:58:33Z"}, "timestamp": 1526075913000}, + {"topic": "TS", "value": {"ID": 3, "EVENT_TS": "2019-05-11T21:58:33Z"}, "timestamp": 1557611913000} + ] + }, + { + "name": "override output timestamp for CTAS", + "statements": [ + "CREATE TABLE TEST (ID bigint) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE TABLE TS WITH (timestamp='sink_ts') AS SELECT id as sink_ts, id FROM test;" + ], + "inputs": [ + {"topic": "test_topic", "value": {"ID": 1}, "timestamp": 1526075913000}, + {"topic": "test_topic", "value": {"ID": -2}, "timestamp": 1526075913000}, + {"topic": "test_topic", "value": {"ID": 3}, "timestamp": 1589234313000} + ], + "outputs": [ + {"topic": "TS", "value": {"SINK_TS":1, "ID": 1}, "timestamp": 1}, + {"topic": "TS", "value": {"SINK_TS":3, "ID": 3}, "timestamp": 3} + ] + }, + { + "name": "override output timestamp for CTAS using a string TIMESTAMP_FORMAT", + "statements": [ + "CREATE STREAM TEST (ID bigint, EVENT_TS varchar) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM TS WITH (timestamp='event_ts', timestamp_format='yyyy-MM-dd''T''HH:mm:ssX') AS SELECT id, event_ts FROM test;" + ], + "inputs": [ + {"topic": "test_topic", "value": {"ID": 1, "EVENT_TS": "2018-05-11T21:58:33Z"}, "timestamp": 10}, + {"topic": "test_topic", "value": {"ID": 2, "EVENT_TS": "not a timestamp"}, "timestamp": 10}, + {"topic": "test_topic", "value": {"ID": 3, "EVENT_TS": "2019-05-11T21:58:33Z"}, "timestamp": 10} + ], + "outputs": [ + {"topic": "TS", "value": {"ID": 1, "EVENT_TS": "2018-05-11T21:58:33Z"}, "timestamp": 1526075913000}, + {"topic": "TS", "value": {"ID": 3, "EVENT_TS": "2019-05-11T21:58:33Z"}, "timestamp": 1557611913000} + ] + }, + { + "name": "timestamp column of source should not influence sink", + "statements": [ + "CREATE STREAM INPUT (ID bigint, EVENT_TS bigint) WITH (kafka_topic='test_topic', value_format='JSON', timestamp='EVENT_TS');", + "CREATE STREAM OUTPUT AS SELECT id as EVENT_TS FROM INPUT;" + ], + "inputs": [ + {"topic": "test_topic", "value": {"ID": 1, "EVENT_TS": 1526075913000}}, + {"topic": "test_topic", "value": {"ID": 2, "EVENT_TS": 1589234313000}} + ], + "outputs": [ + {"topic": "OUTPUT", "value": {"EVENT_TS":1}, "timestamp": 1526075913000}, + {"topic": "OUTPUT", "value": {"EVENT_TS":2}, "timestamp": 1589234313000} + ] + }, + { + "name": "Invalid timestamp value should throw an exception", + "statements": [ + "CREATE STREAM TEST (ID bigint, EVENT_TS string) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM TS WITH (timestamp='event_ts', timestamp_format='yyyy-MM-dd''T''HH:mm:ssX') AS SELECT id, event_ts FROM test;" + ], + "inputs": [ + {"topic": "test_topic", "value": {"ID": 1, "EVENT_TS": "not a timestamp"}} + ] + }, + { + "name": "timestamp with column that does not exist should throw exception", + "statements": [ + "CREATE STREAM TEST (ID bigint, EVENT_TS bigint) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM TS WITH (timestamp='invalid_ts') AS SELECT id, event_ts FROM test;" + ], + "expectedException": { + "type": "io.confluent.ksql.util.KsqlException", + "message": "The TIMESTAMP column set in the WITH clause does not exist in the schema: 'INVALID_TS'" + } + }, + { + "name": "timestamp column with invalid data type should throw exception", + "statements": [ + "CREATE STREAM TEST (ID bigint, EVENT_TS int) WITH (kafka_topic='test_topic', value_format='JSON');", + "CREATE STREAM TS WITH (timestamp='event_ts') AS SELECT id, event_ts FROM test;" + ], + "expectedException": { + "type": "io.confluent.ksql.util.KsqlException", + "message": "Timestamp column, `EVENT_TS`, should be LONG(INT64) or a String with a timestamp_format specified" + } } ] } diff --git a/ksql-rest-app/src/test/resources/ksql-plan-schema/schema.json b/ksql-rest-app/src/test/resources/ksql-plan-schema/schema.json index 860062fc45f9..3fde5a933798 100644 --- a/ksql-rest-app/src/test/resources/ksql-plan-schema/schema.json +++ b/ksql-rest-app/src/test/resources/ksql-plan-schema/schema.json @@ -447,6 +447,9 @@ }, "topicName" : { "type" : "string" + }, + "timestampColumn" : { + "$ref" : "#/definitions/TimestampColumn" } }, "title" : "streamSinkV1", @@ -803,6 +806,9 @@ }, "topicName" : { "type" : "string" + }, + "timestampColumn" : { + "$ref" : "#/definitions/TimestampColumn" } }, "title" : "tableSinkV1", diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/ExecutionStepFactory.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/ExecutionStepFactory.java index 6086a6727ca9..1309e6284527 100644 --- a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/ExecutionStepFactory.java +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/ExecutionStepFactory.java @@ -142,13 +142,15 @@ public static StreamSink streamSink( final QueryContext.Stacker stacker, final Formats formats, final ExecutionStep> source, - final String topicName + final String topicName, + final Optional timestampColumn ) { final QueryContext queryContext = stacker.getQueryContext(); return new StreamSink<>(new ExecutionStepPropertiesV1(queryContext), source, formats, - topicName + topicName, + timestampColumn ); } @@ -244,14 +246,16 @@ public static TableSink tableSink( final QueryContext.Stacker stacker, final ExecutionStep> source, final Formats formats, - final String topicName + final String topicName, + final Optional timestampColumn ) { final QueryContext queryContext = stacker.getQueryContext(); return new TableSink<>( new ExecutionStepPropertiesV1(queryContext), source, formats, - topicName + topicName, + timestampColumn ); } diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/SinkBuilder.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/SinkBuilder.java new file mode 100644 index 000000000000..63f96dcb923a --- /dev/null +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/SinkBuilder.java @@ -0,0 +1,175 @@ +/* + * 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.execution.streams; + +import static java.util.Objects.requireNonNull; + +import io.confluent.ksql.GenericRow; +import io.confluent.ksql.execution.builder.KsqlQueryBuilder; +import io.confluent.ksql.execution.context.QueryContext; +import io.confluent.ksql.execution.context.QueryLoggerUtil; +import io.confluent.ksql.execution.plan.Formats; +import io.confluent.ksql.execution.plan.KeySerdeFactory; +import io.confluent.ksql.execution.streams.timestamp.KsqlTimestampExtractor; +import io.confluent.ksql.execution.streams.timestamp.TimestampExtractionPolicy; +import io.confluent.ksql.execution.streams.timestamp.TimestampExtractionPolicyFactory; +import io.confluent.ksql.execution.timestamp.TimestampColumn; +import io.confluent.ksql.execution.util.EngineProcessingLogMessageFactory; +import io.confluent.ksql.logging.processing.ProcessingLogger; +import io.confluent.ksql.schema.ksql.Column; +import io.confluent.ksql.schema.ksql.LogicalSchema; +import io.confluent.ksql.schema.ksql.PhysicalSchema; +import java.util.Optional; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.Named; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.kstream.Transformer; +import org.apache.kafka.streams.kstream.TransformerSupplier; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.To; + +public final class SinkBuilder { + private static final String TIMESTAMP_TRANSFORM_NAME = "ApplyTimestampTransform-"; + + private SinkBuilder() { + } + + public static void build( + final LogicalSchema schema, + final Formats formats, + final Optional timestampColumn, + final String topicName, + final KStream stream, + final KeySerdeFactory keySerdeFactory, + final QueryContext queryContext, + final KsqlQueryBuilder queryBuilder + ) { + final PhysicalSchema physicalSchema = PhysicalSchema.from(schema, formats.getOptions()); + + final Serde keySerde = keySerdeFactory.buildKeySerde( + formats.getKeyFormat(), + physicalSchema, + queryContext + ); + + final Serde valueSerde = queryBuilder.buildValueSerde( + formats.getValueFormat(), + physicalSchema, + queryContext + ); + + final Optional> tsTransformer = timestampTransformer( + queryBuilder, + queryContext, + schema, + timestampColumn + ); + + final KStream transformed = tsTransformer + .map(t -> stream.transform(t, Named.as(TIMESTAMP_TRANSFORM_NAME + + StreamsUtil.buildOpName(queryContext)))) + .orElse(stream); + + transformed.to(topicName, Produced.with(keySerde, valueSerde)); + } + + private static Optional> timestampTransformer( + final KsqlQueryBuilder queryBuilder, + final QueryContext queryContext, + final LogicalSchema sourceSchema, + final Optional timestampColumn + ) { + if (!timestampColumn.isPresent()) { + return Optional.empty(); + } + + final TimestampExtractionPolicy timestampPolicy = TimestampExtractionPolicyFactory.create( + queryBuilder.getKsqlConfig(), + sourceSchema, + timestampColumn + ); + + return timestampColumn + .map(TimestampColumn::getColumn) + .map(c -> sourceSchema.findValueColumn(c).orElseThrow(IllegalStateException::new)) + .map(Column::index) + .map(timestampPolicy::create) + .map(te -> new TransformTimestamp<>( + te, + queryBuilder + .getProcessingLogContext() + .getLoggerFactory() + .getLogger( + QueryLoggerUtil.queryLoggerName( + queryBuilder.getQueryId(), + queryContext + ) + ) + ) + ); + } + + static class TransformTimestamp + implements TransformerSupplier> { + private final KsqlTimestampExtractor timestampExtractor; + private final ProcessingLogger processingLogger; + + TransformTimestamp( + final KsqlTimestampExtractor timestampExtractor, + final ProcessingLogger processingLogger + ) { + this.timestampExtractor = requireNonNull(timestampExtractor, "timestampExtractor"); + this.processingLogger = requireNonNull(processingLogger, "processingLogger"); + } + + @Override + public Transformer> get() { + return new Transformer>() { + private ProcessorContext processorContext; + + @Override + public void init(final ProcessorContext processorContext) { + this.processorContext = requireNonNull(processorContext, "processorContext"); + } + + @Override + public KeyValue transform(final K key, final GenericRow row) { + try { + processorContext.forward( + key, + row, + To.all().withTimestamp(timestampExtractor.extract(row)) + ); + } catch (final Exception e) { + processingLogger.error( + EngineProcessingLogMessageFactory + .recordProcessingError("Error writing row with extracted timestamp: " + + e.getMessage(), e, row) + ); + } + + return null; + } + + + @Override + public void close() { + } + }; + } + } +} diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/StreamSinkBuilder.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/StreamSinkBuilder.java index cb3edf1f50b1..792c6bf6e9a4 100644 --- a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/StreamSinkBuilder.java +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/StreamSinkBuilder.java @@ -15,16 +15,9 @@ package io.confluent.ksql.execution.streams; -import io.confluent.ksql.GenericRow; import io.confluent.ksql.execution.builder.KsqlQueryBuilder; -import io.confluent.ksql.execution.context.QueryContext; -import io.confluent.ksql.execution.plan.Formats; import io.confluent.ksql.execution.plan.KStreamHolder; import io.confluent.ksql.execution.plan.StreamSink; -import io.confluent.ksql.schema.ksql.LogicalSchema; -import io.confluent.ksql.schema.ksql.PhysicalSchema; -import org.apache.kafka.common.serialization.Serde; -import org.apache.kafka.streams.kstream.Produced; public final class StreamSinkBuilder { private StreamSinkBuilder() { @@ -34,23 +27,15 @@ public static void build( final KStreamHolder stream, final StreamSink streamSink, final KsqlQueryBuilder queryBuilder) { - final QueryContext queryContext = streamSink.getProperties().getQueryContext(); - final LogicalSchema schema = stream.getSchema(); - final Formats formats = streamSink.getFormats(); - final PhysicalSchema physicalSchema = PhysicalSchema.from(schema, formats.getOptions()); - final Serde keySerde = stream.getKeySerdeFactory().buildKeySerde( - formats.getKeyFormat(), - physicalSchema, - queryContext + SinkBuilder.build( + stream.getSchema(), + streamSink.getFormats(), + streamSink.getTimestampColumn(), + streamSink.getTopicName(), + stream.getStream(), + stream.getKeySerdeFactory(), + streamSink.getProperties().getQueryContext(), + queryBuilder ); - - final Serde valueSerde = queryBuilder.buildValueSerde( - formats.getValueFormat(), - physicalSchema, - queryContext - ); - - stream.getStream() - .to(streamSink.getTopicName(), Produced.with(keySerde, valueSerde)); } } diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/TableSinkBuilder.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/TableSinkBuilder.java index b925c4a8ffde..3063bf0bf025 100644 --- a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/TableSinkBuilder.java +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/TableSinkBuilder.java @@ -15,16 +15,9 @@ package io.confluent.ksql.execution.streams; -import io.confluent.ksql.GenericRow; import io.confluent.ksql.execution.builder.KsqlQueryBuilder; -import io.confluent.ksql.execution.context.QueryContext; -import io.confluent.ksql.execution.plan.Formats; import io.confluent.ksql.execution.plan.KTableHolder; import io.confluent.ksql.execution.plan.TableSink; -import io.confluent.ksql.schema.ksql.LogicalSchema; -import io.confluent.ksql.schema.ksql.PhysicalSchema; -import org.apache.kafka.common.serialization.Serde; -import org.apache.kafka.streams.kstream.Produced; public final class TableSinkBuilder { private TableSinkBuilder() { @@ -34,24 +27,15 @@ public static void build( final KTableHolder table, final TableSink tableSink, final KsqlQueryBuilder queryBuilder) { - final QueryContext queryContext = tableSink.getProperties().getQueryContext(); - final LogicalSchema schema = table.getSchema(); - final Formats formats = tableSink.getFormats(); - final PhysicalSchema physicalSchema = PhysicalSchema.from(schema, formats.getOptions()); - final Serde keySerde = table.getKeySerdeFactory().buildKeySerde( - formats.getKeyFormat(), - physicalSchema, - queryContext + SinkBuilder.build( + table.getSchema(), + tableSink.getFormats(), + tableSink.getTimestampColumn(), + tableSink.getTopicName(), + table.getTable().toStream(), + table.getKeySerdeFactory(), + tableSink.getProperties().getQueryContext(), + queryBuilder ); - - final Serde valueSerde = queryBuilder.buildValueSerde( - formats.getValueFormat(), - physicalSchema, - queryContext - ); - - table.getTable() - .toStream() - .to(tableSink.getTopicName(), Produced.with(keySerde, valueSerde)); } } \ No newline at end of file diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/KsqlTimestampExtractor.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/KsqlTimestampExtractor.java new file mode 100644 index 000000000000..f2a649680703 --- /dev/null +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/KsqlTimestampExtractor.java @@ -0,0 +1,28 @@ +/* + * 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.execution.streams.timestamp; + +import io.confluent.ksql.GenericRow; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.streams.processor.TimestampExtractor; + +public interface KsqlTimestampExtractor extends TimestampExtractor { + default long extract(final ConsumerRecord record, final long previousTimestamp) { + return extract((GenericRow) record.value()); + } + + long extract(GenericRow row); +} diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/LongColumnTimestampExtractionPolicy.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/LongColumnTimestampExtractionPolicy.java index 9f66baa1274d..6101512d5853 100644 --- a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/LongColumnTimestampExtractionPolicy.java +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/LongColumnTimestampExtractionPolicy.java @@ -18,7 +18,6 @@ import com.google.errorprone.annotations.Immutable; import io.confluent.ksql.name.ColumnName; import java.util.Objects; -import org.apache.kafka.streams.processor.TimestampExtractor; @Immutable public class LongColumnTimestampExtractionPolicy implements TimestampExtractionPolicy { @@ -31,7 +30,7 @@ public LongColumnTimestampExtractionPolicy(final ColumnName timestampField) { } @Override - public TimestampExtractor create(final int columnIndex) { + public KsqlTimestampExtractor create(final int columnIndex) { return new LongTimestampExtractor(columnIndex); } diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/LongTimestampExtractor.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/LongTimestampExtractor.java index c9fb644eb7c6..92831643be90 100644 --- a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/LongTimestampExtractor.java +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/LongTimestampExtractor.java @@ -16,15 +16,8 @@ package io.confluent.ksql.execution.streams.timestamp; import io.confluent.ksql.GenericRow; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.streams.processor.TimestampExtractor; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class LongTimestampExtractor implements TimestampExtractor { - - private static final Logger log = LoggerFactory.getLogger(LongTimestampExtractor.class); +public class LongTimestampExtractor implements KsqlTimestampExtractor { private final int timestampColumnindex; LongTimestampExtractor(final int timestampColumnindex) { @@ -32,21 +25,7 @@ public class LongTimestampExtractor implements TimestampExtractor { } @Override - public long extract(final ConsumerRecord consumerRecord, final long l) { - if (timestampColumnindex < 0) { - return 0; - } else { - try { - if (consumerRecord.value() instanceof GenericRow) { - final GenericRow genericRow = (GenericRow) consumerRecord.value(); - if (genericRow.get(timestampColumnindex) instanceof Long) { - return (long) genericRow.get(timestampColumnindex); - } - } - } catch (final Exception e) { - log.error("Exception in extracting timestamp for row: " + consumerRecord.value(), e); - } - } - return 0; + public long extract(final GenericRow row) { + return (long)row.get(timestampColumnindex); } } diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/MetadataTimestampExtractionPolicy.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/MetadataTimestampExtractionPolicy.java index cef9a1d553ca..4dd97ea58eac 100644 --- a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/MetadataTimestampExtractionPolicy.java +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/MetadataTimestampExtractionPolicy.java @@ -33,8 +33,8 @@ public MetadataTimestampExtractionPolicy(final TimestampExtractor timestampExtra } @Override - public TimestampExtractor create(final int columnIndex) { - return timestampExtractor; + public KsqlTimestampExtractor create(final int columnIndex) { + return new MetadataTimestampExtractor(timestampExtractor); } @Override diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/MetadataTimestampExtractor.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/MetadataTimestampExtractor.java new file mode 100644 index 000000000000..bcc993f2bb6e --- /dev/null +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/MetadataTimestampExtractor.java @@ -0,0 +1,48 @@ +/* + * Copyright 2020 Confluent Inc. + * + * Licensed under the Confluent Community License (the "License"); you may not use + * this file except in compliance with the License. You may obtain a copy of the + * License at + * + * http://www.confluent.io/confluent-community-license + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ + +package io.confluent.ksql.execution.streams.timestamp; + +import static java.util.Objects.requireNonNull; + +import io.confluent.ksql.GenericRow; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.streams.processor.TimestampExtractor; + +public class MetadataTimestampExtractor implements KsqlTimestampExtractor { + private final TimestampExtractor timestampExtractor; + + public MetadataTimestampExtractor(final TimestampExtractor timestampExtractor) { + this.timestampExtractor = requireNonNull(timestampExtractor, "timestampExtractor"); + } + + public TimestampExtractor getTimestampExtractor() { + return timestampExtractor; + } + + @Override + public long extract( + final ConsumerRecord record, + final long previousTimestamp + ) { + return timestampExtractor.extract(record, previousTimestamp); + } + + @Override + public long extract(final GenericRow row) { + throw new UnsupportedOperationException("Operation not supported for class: " + + timestampExtractor.getClass()); + } +} diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/StringTimestampExtractionPolicy.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/StringTimestampExtractionPolicy.java index 947dba5d3570..e20681749ac8 100644 --- a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/StringTimestampExtractionPolicy.java +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/StringTimestampExtractionPolicy.java @@ -19,7 +19,6 @@ import com.google.errorprone.annotations.Immutable; import io.confluent.ksql.name.ColumnName; import java.util.Objects; -import org.apache.kafka.streams.processor.TimestampExtractor; @Immutable public class StringTimestampExtractionPolicy implements TimestampExtractionPolicy { @@ -38,7 +37,7 @@ public StringTimestampExtractionPolicy( } @Override - public TimestampExtractor create(final int timestampColumnIndex) { + public KsqlTimestampExtractor create(final int timestampColumnIndex) { return new StringTimestampExtractor(format, timestampColumnIndex); } diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/StringTimestampExtractor.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/StringTimestampExtractor.java index 3e7f66fa8481..1eff1f3667d8 100644 --- a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/StringTimestampExtractor.java +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/StringTimestampExtractor.java @@ -20,10 +20,8 @@ import io.confluent.ksql.util.KsqlException; import io.confluent.ksql.util.timestamp.StringToTimestampParser; import java.util.Objects; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.streams.processor.TimestampExtractor; -public class StringTimestampExtractor implements TimestampExtractor { +public class StringTimestampExtractor implements KsqlTimestampExtractor { private final StringToTimestampParser timestampParser; private final int timestampColumn; private final String format; @@ -36,15 +34,13 @@ public class StringTimestampExtractor implements TimestampExtractor { } @Override - public long extract(final ConsumerRecord consumerRecord, - final long previousTimestamp) { - final GenericRow row = (GenericRow) consumerRecord.value(); + public long extract(final GenericRow row) { final String value = (String)row.get(timestampColumn); + try { return timestampParser.parse(value); } catch (final KsqlException e) { - throw new KsqlException("Unable to parse string timestamp from record." - + " record=" + consumerRecord + throw new KsqlException("Unable to parse string timestamp." + " timestamp=" + value + " timestamp_format=" + format, e); diff --git a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/TimestampExtractionPolicy.java b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/TimestampExtractionPolicy.java index 3c9e4527751e..4bafb69a6922 100644 --- a/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/TimestampExtractionPolicy.java +++ b/ksql-streams/src/main/java/io/confluent/ksql/execution/streams/timestamp/TimestampExtractionPolicy.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo.As; import com.google.errorprone.annotations.Immutable; import io.confluent.ksql.name.ColumnName; -import org.apache.kafka.streams.processor.TimestampExtractor; @Immutable @JsonTypeInfo( @@ -35,7 +34,7 @@ }) public interface TimestampExtractionPolicy { - TimestampExtractor create(int columnIndex); + KsqlTimestampExtractor create(int columnIndex); default ColumnName getTimestampField() { return null; diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/SinkBuilderTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/SinkBuilderTest.java new file mode 100644 index 000000000000..b0d59c59d3ca --- /dev/null +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/SinkBuilderTest.java @@ -0,0 +1,306 @@ +/* + * 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.execution.streams; + +import io.confluent.ksql.GenericRow; +import io.confluent.ksql.execution.builder.KsqlQueryBuilder; +import io.confluent.ksql.execution.context.QueryContext; +import io.confluent.ksql.execution.plan.Formats; +import io.confluent.ksql.execution.plan.KeySerdeFactory; +import io.confluent.ksql.execution.streams.timestamp.KsqlTimestampExtractor; +import io.confluent.ksql.execution.timestamp.TimestampColumn; +import io.confluent.ksql.logging.processing.ProcessingLogContext; +import io.confluent.ksql.logging.processing.ProcessingLogger; +import io.confluent.ksql.logging.processing.ProcessingLoggerFactory; +import io.confluent.ksql.name.ColumnName; +import io.confluent.ksql.query.QueryId; +import io.confluent.ksql.schema.ksql.LogicalSchema; +import io.confluent.ksql.schema.ksql.PhysicalSchema; +import io.confluent.ksql.schema.ksql.types.SqlTypes; +import io.confluent.ksql.serde.FormatFactory; +import io.confluent.ksql.serde.FormatInfo; +import io.confluent.ksql.serde.SerdeOption; +import io.confluent.ksql.util.KsqlException; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.Named; +import org.apache.kafka.streams.kstream.Produced; +import org.apache.kafka.streams.kstream.Transformer; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.To; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.InOrder; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnitRunner; + +import java.util.Arrays; +import java.util.Optional; + +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyZeroInteractions; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class SinkBuilderTest { + private static final String TOPIC = "TOPIC"; + private static final QueryId QUERY_ID = new QueryId("QUERY_ID"); + private static final String QUERY_CONTEXT_NAME = "TIMESTAMP-TRANSFORM"; + + private static final LogicalSchema SCHEMA = LogicalSchema.builder() + .valueColumn(ColumnName.of("BLUE"), SqlTypes.BIGINT) + .valueColumn(ColumnName.of("GREEN"), SqlTypes.STRING) + .build(); + + private static final FormatInfo KEY_FORMAT = FormatInfo.of(FormatFactory.KAFKA.name()); + private static final FormatInfo VALUE_FORMAT = FormatInfo.of(FormatFactory.JSON.name()); + private static final PhysicalSchema PHYSICAL_SCHEMA = + PhysicalSchema.from(SCHEMA.withoutMetaAndKeyColsInValue(), SerdeOption.none()); + + @Mock + private KsqlQueryBuilder queryBuilder; + @Mock + private KeySerdeFactory keySerdeFactory; + @Mock + private KStream kStream; + @Mock + private Serde keySerde; + @Mock + private Serde valSerde; + @Mock + private QueryContext queryContext; + @Mock + private GenericRow row; + @Mock + private ProcessorContext processorContext; + @Mock + private ProcessingLogger processingLogger; + @Captor + private ArgumentCaptor toCaptor; + + @Before + public void setup() { + when(queryBuilder.getQueryId()).thenReturn(QUERY_ID); + when(keySerdeFactory.buildKeySerde(any(), any(), any())).thenReturn(keySerde); + when(queryBuilder.buildValueSerde(any(), any(), any())).thenReturn(valSerde); + + final ProcessingLogContext processingLogContext = mock(ProcessingLogContext.class); + when(queryBuilder.getProcessingLogContext()).thenReturn(processingLogContext); + final ProcessingLoggerFactory processingLoggerFactory = mock(ProcessingLoggerFactory.class); + when(processingLogContext.getLoggerFactory()).thenReturn(processingLoggerFactory); + when(processingLoggerFactory.getLogger(anyString())).thenReturn(processingLogger); + + when(queryContext.getContext()).thenReturn(Arrays.asList(QUERY_CONTEXT_NAME)); + } + + @Test + public void shouldBuildKeySerdeCorrectly() { + // Given/When + buildDefaultSinkBuilder(); + + // Then: + verify(keySerdeFactory).buildKeySerde(KEY_FORMAT, PHYSICAL_SCHEMA, queryContext); + } + + @Test + public void shouldBuildValueSerdeCorrectly() { + // Given/When + buildDefaultSinkBuilder(); + + // Then: + verify(queryBuilder).buildValueSerde( + VALUE_FORMAT, + PHYSICAL_SCHEMA, + queryContext + ); + } + + @Test + public void shouldWriteOutStreamWithCorrectSerdes() { + // Given/When + buildDefaultSinkBuilder(); + + // Then + verify(kStream).to(anyString(), eq(Produced.with(keySerde, valSerde))); + } + + @Test + public void shouldWriteOutStreamToCorrectTopic() { + // Given/When + buildDefaultSinkBuilder(); + + // Then + verify(kStream).to(eq(TOPIC), any()); + } + + @Test + public void shouldBuildStreamUsingTransformTimestampWhenTimestampIsSpecified() { + // Given/When + SinkBuilder.build( + SCHEMA, + Formats.of(KEY_FORMAT, VALUE_FORMAT, SerdeOption.none()), + Optional.of(new TimestampColumn(ColumnName.of("BLUE"), Optional.empty())), + TOPIC, + kStream, + keySerdeFactory, + queryContext, + queryBuilder + ); + + // Then + final InOrder inOrder = Mockito.inOrder(kStream); + inOrder.verify(kStream).transform(any(), any(Named.class)); + inOrder.verify(kStream).to(eq(TOPIC), any()); + inOrder.verifyNoMoreInteractions(); + } + + @Test + public void shouldBuildStreamWithoutTransformTimestampWhenNoTimestampIsSpecified() { + // Given/When + buildDefaultSinkBuilder(); + + // Then + final InOrder inOrder = Mockito.inOrder(kStream); + inOrder.verify(kStream).to(anyString(), any()); + inOrder.verifyNoMoreInteractions(); + } + + @Test + public void shouldTransformTimestampRow() { + // Given + final long timestampColumnValue = 10001; + final KsqlTimestampExtractor timestampExtractor + = mock(KsqlTimestampExtractor.class); + when(timestampExtractor.extract(any())).thenReturn(timestampColumnValue); + + // When + final Transformer> transformer = + getTransformer(timestampExtractor, processingLogger); + transformer.init(processorContext); + final KeyValue kv = transformer.transform("key", row); + + // Then + assertNull(kv); + verify(timestampExtractor).extract(row); + verify(processorContext, Mockito.times(1)) + .forward(eq("key"), eq(row), toCaptor.capture()); + assertTrue(toCaptor.getValue().equals(To.all().withTimestamp(timestampColumnValue))); + verifyZeroInteractions(processingLogger); + } + + @Test + public void shouldCallProcessingLoggerOnForwardError() { + // Given + final long timestampColumnValue = 10001; + final KsqlTimestampExtractor timestampExtractor + = mock(KsqlTimestampExtractor.class); + when(timestampExtractor.extract(any())).thenReturn(timestampColumnValue); + doThrow(KsqlException.class) + .when(processorContext) + .forward(any(), any(), any(To.class)); + + // When + final Transformer> transformer = + getTransformer(timestampExtractor, processingLogger); + transformer.init(processorContext); + final KeyValue kv = transformer.transform("key", row); + + // Then + assertNull(kv); + verify(timestampExtractor).extract(row); + verify(processorContext, Mockito.times(1)) + .forward(eq("key"), eq(row), toCaptor.capture()); + verify(processingLogger, Mockito.times(1)).error(any()); + } + + @Test + public void shouldCallProcessingLoggerOnTimestampExtractorError() { + // Given + final KsqlTimestampExtractor timestampExtractor + = mock(KsqlTimestampExtractor.class); + doThrow(KsqlException.class).when(timestampExtractor).extract(row); + + // When + final Transformer> transformer = + getTransformer(timestampExtractor, processingLogger); + transformer.init(processorContext); + final KeyValue kv = transformer.transform("key", row); + + // Then + assertNull(kv); + verify(timestampExtractor).extract(row); + verify(processingLogger, Mockito.times(1)).error(any()); + verifyZeroInteractions(processorContext); + } + + @Test(expected = NullPointerException.class) + public void shouldThrowOnNullProcessorContext() { + // Given + final KsqlTimestampExtractor timestampExtractor + = mock(KsqlTimestampExtractor.class); + + // When/Then + getTransformer(timestampExtractor, processingLogger).init(null); + } + + @Test(expected = NullPointerException.class) + public void shouldThrowOnNullTimestampExtractor() { + // When/Then + getTransformer(null, null); + } + + @Test(expected = NullPointerException.class) + public void shouldThrowOnNullProcessingLogger() { + // Given + final KsqlTimestampExtractor timestampExtractor + = mock(KsqlTimestampExtractor.class); + + // When/Then + getTransformer(timestampExtractor, null); + } + + private void buildDefaultSinkBuilder() { + SinkBuilder.build( + SCHEMA, + Formats.of(KEY_FORMAT, VALUE_FORMAT, SerdeOption.none()), + Optional.empty(), + TOPIC, + kStream, + keySerdeFactory, + queryContext, + queryBuilder + ); + } + + private Transformer> getTransformer( + final KsqlTimestampExtractor timestampExtractor, + final ProcessingLogger processingLogger + ) { + return new SinkBuilder.TransformTimestamp(timestampExtractor, processingLogger).get(); + } +} diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StreamSinkBuilderTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StreamSinkBuilderTest.java deleted file mode 100644 index 5a62ecf8abdd..000000000000 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/StreamSinkBuilderTest.java +++ /dev/null @@ -1,147 +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.execution.streams; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import io.confluent.ksql.GenericRow; -import io.confluent.ksql.execution.builder.KsqlQueryBuilder; -import io.confluent.ksql.execution.context.QueryContext; -import io.confluent.ksql.execution.plan.ExecutionStep; -import io.confluent.ksql.execution.plan.ExecutionStepPropertiesV1; -import io.confluent.ksql.execution.plan.KStreamHolder; -import io.confluent.ksql.execution.plan.KeySerdeFactory; -import io.confluent.ksql.execution.plan.PlanBuilder; -import io.confluent.ksql.execution.plan.StreamSink; -import io.confluent.ksql.name.ColumnName; -import io.confluent.ksql.schema.ksql.LogicalSchema; -import io.confluent.ksql.schema.ksql.PhysicalSchema; -import io.confluent.ksql.schema.ksql.types.SqlTypes; -import io.confluent.ksql.serde.FormatFactory; -import io.confluent.ksql.serde.FormatInfo; -import io.confluent.ksql.serde.SerdeOption; -import org.apache.kafka.common.serialization.Serde; -import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.streams.kstream.KStream; -import org.apache.kafka.streams.kstream.Produced; -import org.apache.kafka.streams.kstream.ValueMapper; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.ArgumentCaptor; -import org.mockito.Captor; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; - -@RunWith(MockitoJUnitRunner.class) -public class StreamSinkBuilderTest { - - private static final String TOPIC = "TOPIC"; - private static final LogicalSchema SCHEMA = LogicalSchema.builder() - .valueColumn(ColumnName.of("BLUE"), SqlTypes.BIGINT) - .valueColumn(ColumnName.of("GREEN"), SqlTypes.STRING) - .build(); - - private static final PhysicalSchema PHYSICAL_SCHEMA = - PhysicalSchema.from(SCHEMA.withoutMetaAndKeyColsInValue(), SerdeOption.none()); - private static final FormatInfo KEY_FORMAT = FormatInfo.of(FormatFactory.KAFKA.name()); - private static final FormatInfo VALUE_FORMAT = FormatInfo.of(FormatFactory.JSON.name()); - - @Mock - private KsqlQueryBuilder queryBuilder; - @Mock - private KeySerdeFactory keySerdeFactory; - @Mock - private KStream kStream; - @Mock - private ExecutionStep> source; - @Mock - private Serde keySerde; - @Mock - private Serde valSerde; - @Captor - private ArgumentCaptor> mapperCaptor; - @Mock - private QueryContext queryContext; - - private PlanBuilder planBuilder; - private StreamSink sink; - - @Before - public void setup() { - when(keySerdeFactory.buildKeySerde(any(), any(), any())).thenReturn(keySerde); - when(queryBuilder.buildValueSerde(any(), any(), any())).thenReturn(valSerde); - when(source.build(any())).thenReturn(new KStreamHolder<>(kStream, SCHEMA, keySerdeFactory)); - - sink = new StreamSink<>( - new ExecutionStepPropertiesV1(queryContext), - source, - io.confluent.ksql.execution.plan.Formats.of(KEY_FORMAT, VALUE_FORMAT, SerdeOption.none()), - TOPIC - ); - planBuilder = new KSPlanBuilder( - queryBuilder, - mock(SqlPredicateFactory.class), - mock(AggregateParamsFactory.class), - mock(StreamsFactories.class) - ); - } - - @Test - public void shouldWriteOutStreamToCorrectTopic() { - // When: - sink.build(planBuilder); - - // Then: - verify(kStream).to(eq(TOPIC), any()); - } - - @Test - public void shouldBuildKeySerdeCorrectly() { - // When: - sink.build(planBuilder); - - // Then: - verify(keySerdeFactory).buildKeySerde(KEY_FORMAT, PHYSICAL_SCHEMA, queryContext); - } - - @Test - public void shouldBuildValueSerdeCorrectly() { - // When: - sink.build(planBuilder); - - // Then: - verify(queryBuilder).buildValueSerde( - VALUE_FORMAT, - PHYSICAL_SCHEMA, - queryContext - ); - } - - @Test - public void shouldWriteOutStreamWithCorrectSerdes() { - // When: - sink.build(planBuilder); - - // Then: - verify(kStream).to(anyString(), eq(Produced.with(keySerde, valSerde))); - } -} diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/TableSinkBuilderTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/TableSinkBuilderTest.java deleted file mode 100644 index 3fcda4a47311..000000000000 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/TableSinkBuilderTest.java +++ /dev/null @@ -1,168 +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.execution.streams; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoMoreInteractions; -import static org.mockito.Mockito.when; - -import io.confluent.ksql.GenericRow; -import io.confluent.ksql.execution.builder.KsqlQueryBuilder; -import io.confluent.ksql.execution.context.QueryContext; -import io.confluent.ksql.execution.plan.ExecutionStep; -import io.confluent.ksql.execution.plan.ExecutionStepPropertiesV1; -import io.confluent.ksql.execution.plan.KTableHolder; -import io.confluent.ksql.execution.plan.KeySerdeFactory; -import io.confluent.ksql.execution.plan.PlanBuilder; -import io.confluent.ksql.execution.plan.TableSink; -import io.confluent.ksql.name.ColumnName; -import io.confluent.ksql.schema.ksql.LogicalSchema; -import io.confluent.ksql.schema.ksql.PhysicalSchema; -import io.confluent.ksql.schema.ksql.types.SqlTypes; -import io.confluent.ksql.serde.FormatFactory; -import io.confluent.ksql.serde.FormatInfo; -import io.confluent.ksql.serde.SerdeOption; -import org.apache.kafka.common.serialization.Serde; -import org.apache.kafka.connect.data.Struct; -import org.apache.kafka.streams.kstream.KStream; -import org.apache.kafka.streams.kstream.KTable; -import org.apache.kafka.streams.kstream.Produced; -import org.apache.kafka.streams.kstream.ValueMapper; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.ArgumentCaptor; -import org.mockito.Captor; -import org.mockito.InOrder; -import org.mockito.Mock; -import org.mockito.Mockito; -import org.mockito.junit.MockitoJUnitRunner; - -@RunWith(MockitoJUnitRunner.class) -public class TableSinkBuilderTest { - - private static final String TOPIC = "TOPIC"; - private static final LogicalSchema SCHEMA = LogicalSchema.builder() - .valueColumn(ColumnName.of("BLUE"), SqlTypes.BIGINT) - .valueColumn(ColumnName.of("GREEN"), SqlTypes.STRING) - .build(); - - private static final PhysicalSchema PHYSICAL_SCHEMA = - PhysicalSchema.from(SCHEMA.withoutMetaAndKeyColsInValue(), SerdeOption.none()); - private static final FormatInfo KEY_FORMAT = FormatInfo.of(FormatFactory.KAFKA.name()); - private static final FormatInfo VALUE_FORMAT = FormatInfo.of(FormatFactory.JSON.name()); - - @Mock - private KsqlQueryBuilder queryBuilder; - @Mock - private KeySerdeFactory keySerdeFactory; - @Mock - private KTable kTable; - @Mock - private KStream kStream; - @Mock - private ExecutionStep> source; - @Mock - private Serde keySerde; - @Mock - private Serde valSerde; - @Captor - private ArgumentCaptor> mapperCaptor; - - private final QueryContext queryContext = - new QueryContext.Stacker().push("sink").getQueryContext(); - - private PlanBuilder planBuilder; - private TableSink sink; - - @Before - public void setup() { - when(keySerdeFactory.buildKeySerde(any(), any(), any())).thenReturn(keySerde); - when(queryBuilder.buildValueSerde(any(), any(), any())).thenReturn(valSerde); - when(kTable.toStream()).thenReturn(kStream); - when(source.build(any())).thenReturn( - KTableHolder.unmaterialized(kTable, SCHEMA, keySerdeFactory)); - - sink = new TableSink<>( - new ExecutionStepPropertiesV1(queryContext), - source, - io.confluent.ksql.execution.plan.Formats.of(KEY_FORMAT, VALUE_FORMAT, SerdeOption.none()), - TOPIC - ); - planBuilder = new KSPlanBuilder( - queryBuilder, - mock(SqlPredicateFactory.class), - mock(AggregateParamsFactory.class), - mock(StreamsFactories.class) - ); - } - - @Test - public void shouldWriteOutTable() { - // When: - sink.build(planBuilder); - - // Then: - final InOrder inOrder = Mockito.inOrder(kTable, kStream); - inOrder.verify(kTable).toStream(); - inOrder.verify(kStream).to(anyString(), any()); - verifyNoMoreInteractions(kStream); - } - - @Test - public void shouldWriteOutTableToCorrectTopic() { - // When: - sink.build(planBuilder); - - // Then: - verify(kStream).to(eq(TOPIC), any()); - } - - @Test - public void shouldBuildKeySerdeCorrectly() { - // When: - sink.build(planBuilder); - - // Then: - verify(keySerdeFactory).buildKeySerde(KEY_FORMAT, PHYSICAL_SCHEMA, queryContext); - } - - @Test - public void shouldBuildValueSerdeCorrectly() { - // When: - sink.build(planBuilder); - - // Then: - verify(queryBuilder).buildValueSerde( - VALUE_FORMAT, - PHYSICAL_SCHEMA, - queryContext - ); - } - - @Test - public void shouldWriteOutTableWithCorrectSerdes() { - // When: - sink.build(planBuilder); - - // Then: - verify(kStream).to(anyString(), eq(Produced.with(keySerde, valSerde))); - } -} diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/timestamp/KsqlTimestampExtractorTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/timestamp/KsqlTimestampExtractorTest.java new file mode 100644 index 000000000000..4146e5a28095 --- /dev/null +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/timestamp/KsqlTimestampExtractorTest.java @@ -0,0 +1,29 @@ +package io.confluent.ksql.execution.streams.timestamp; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.junit.Test; + +import static io.confluent.ksql.GenericRow.genericRow; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; + +public class KsqlTimestampExtractorTest { + @Test + public void shouldCallExtractTimestampFromGenericRow() { + // Given + final long timestamp = 1526075913000L; + final KsqlTimestampExtractor timestampExtractor = (row -> timestamp); + + // When + final long actualTime = timestampExtractor.extract(new ConsumerRecord<>("topic", + 1, + 1, + null, + genericRow(0)), + 1 + ); + + // Then + assertThat(actualTime, equalTo(timestamp)); + } +} diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/timestamp/MetadataTimestampExtractorTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/timestamp/MetadataTimestampExtractorTest.java new file mode 100644 index 000000000000..171a2d092e34 --- /dev/null +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/timestamp/MetadataTimestampExtractorTest.java @@ -0,0 +1,61 @@ +/* + * 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.execution.streams.timestamp; + +import io.confluent.ksql.GenericRow; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.streams.processor.TimestampExtractor; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnitRunner; + +import static org.mockito.Mockito.mock; + +@RunWith(MockitoJUnitRunner.class) +public class MetadataTimestampExtractorTest { + @Mock + private TimestampExtractor timestampExtractor; + @Mock + private ConsumerRecord record; + + @Test + public void shouldCallInternalTimestampExtractorOnExtract() { + // Given + final MetadataTimestampExtractor metadataTimestampExtractor = + new MetadataTimestampExtractor(timestampExtractor); + + // When + metadataTimestampExtractor.extract(record, 1); + + // Then + Mockito.verify(timestampExtractor, Mockito.times(1)) + .extract(record, 1); + } + + @Test(expected = NullPointerException.class) + public void shouldThrowOnNullTimestampExtractor() { + // When/Then + new MetadataTimestampExtractor(null); + } + + @Test(expected = UnsupportedOperationException.class) + public void shouldThrowUnsupportedExceptionOnExtractGenericRow() { + // when/Then + new MetadataTimestampExtractor(timestampExtractor).extract(mock(GenericRow.class)); + } +} diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/timestamp/StringTimestampExtractorTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/timestamp/StringTimestampExtractorTest.java index 168e57a6a5a6..40ce12554b97 100644 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/timestamp/StringTimestampExtractorTest.java +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/timestamp/StringTimestampExtractorTest.java @@ -34,13 +34,7 @@ public void shouldExtractTimestampFromStringWithFormat() throws ParseException { final String stringTime = "2010-Jan-11"; final long expectedTime = new SimpleDateFormat(format).parse(stringTime).getTime(); - final long actualTime = timestampExtractor.extract(new ConsumerRecord<>("topic", - 1, - 1, - null, - genericRow(stringTime)), - 1 - ); + final long actualTime = timestampExtractor.extract(genericRow(stringTime)); assertThat(actualTime, equalTo(expectedTime)); } @@ -52,6 +46,6 @@ public void shouldThrowIfColumnIndexIsNegative() { @Test(expected = NullPointerException.class) public void shouldThrowOnNullFormat() { - new StringTimestampExtractor(null, -1); + new StringTimestampExtractor(null, 0); } } \ No newline at end of file diff --git a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/timestamp/TimestampExtractionPolicyFactoryTest.java b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/timestamp/TimestampExtractionPolicyFactoryTest.java index 1a04cea6ec60..656e7c090f01 100644 --- a/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/timestamp/TimestampExtractionPolicyFactoryTest.java +++ b/ksql-streams/src/test/java/io/confluent/ksql/execution/streams/timestamp/TimestampExtractionPolicyFactoryTest.java @@ -63,7 +63,8 @@ public void shouldCreateMetadataPolicyWhenTimestampFieldNotProvided() { // Then: assertThat(result, instanceOf(MetadataTimestampExtractionPolicy.class)); - assertThat(result.create(0), instanceOf(FailOnInvalidTimestamp.class)); + assertThat(((MetadataTimestampExtractor)result.create(0)).getTimestampExtractor(), + instanceOf(FailOnInvalidTimestamp.class)); } @Test @@ -100,7 +101,8 @@ public void shouldCreateMetadataPolicyWithDefaultFailedOnInvalidTimestamp() { // Then: assertThat(result, instanceOf(MetadataTimestampExtractionPolicy.class)); - assertThat(result.create(0), instanceOf(FailOnInvalidTimestamp.class)); + assertThat(((MetadataTimestampExtractor)result.create(0)).getTimestampExtractor(), + instanceOf(FailOnInvalidTimestamp.class)); } @Test @@ -121,7 +123,8 @@ public void shouldCreateMetadataPolicyWithConfiguredUsePartitionTimeOnInvalidTim // Then: assertThat(result, instanceOf(MetadataTimestampExtractionPolicy.class)); - assertThat(result.create(0), instanceOf(UsePartitionTimeOnInvalidTimestamp.class)); + assertThat(((MetadataTimestampExtractor)result.create(0)).getTimestampExtractor(), + instanceOf(UsePartitionTimeOnInvalidTimestamp.class)); } @Test