Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

fix: changes required for compatibility with KIP-479 #3466

Merged
merged 5 commits into from
Oct 9, 2019
Merged
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,12 @@
import io.confluent.ksql.execution.expression.tree.Expression;
import io.confluent.ksql.execution.expression.tree.FunctionCall;
import io.confluent.ksql.execution.expression.tree.LongLiteral;
import io.confluent.ksql.execution.streams.StreamJoinedFactory;
import io.confluent.ksql.name.ColumnName;
import io.confluent.ksql.name.FunctionName;
import io.confluent.ksql.name.SourceName;
import io.confluent.ksql.schema.ksql.ColumnRef;
import io.confluent.ksql.execution.expression.tree.ColumnReferenceExp;
import io.confluent.ksql.execution.plan.DefaultExecutionStepProperties;
import io.confluent.ksql.execution.plan.PlanBuilder;
import io.confluent.ksql.execution.plan.ExecutionStep;
Expand Down Expand Up @@ -119,6 +125,8 @@
import org.apache.kafka.streams.kstream.KTable;
import org.apache.kafka.streams.kstream.KeyValueMapper;
import org.apache.kafka.streams.kstream.Predicate;
import org.apache.kafka.streams.kstream.StreamJoined;
import org.apache.kafka.streams.kstream.ValueMapper;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
Expand All @@ -143,6 +151,8 @@ public class SchemaKStreamTest {
"group", Serdes.String(), Serdes.String());
private final Joined joined = Joined.with(
Serdes.String(), Serdes.String(), Serdes.String(), "join");
private final StreamJoined streamJoined = StreamJoined.with(
Serdes.String(), Serdes.String(), Serdes.String()).withName("join");
private final KeyField validJoinKeyField = KeyField.of(
Optional.of(ColumnRef.of(SourceName.of("left"), ColumnName.of("COL1"))),
metaStore.getSource(SourceName.of("TEST1"))
Expand Down Expand Up @@ -179,6 +189,8 @@ public class SchemaKStreamTest {
@Mock
private MaterializedFactory mockMaterializedFactory;
@Mock
private StreamJoinedFactory mockStreamJoinedFactory;
@Mock
private KStream mockKStream;
@Mock
private KeySerde keySerde;
Expand Down Expand Up @@ -206,6 +218,7 @@ public void init() {
when(mockGroupedFactory.create(anyString(), any(Serde.class), any(Serde.class)))
.thenReturn(grouped);
when(mockJoinedFactory.create(any(), any(), any(), anyString())).thenReturn(joined);
when(mockStreamJoinedFactory.create(any(), any(), any(), anyString(), anyString())).thenReturn(streamJoined);

final KsqlStream secondKsqlStream = (KsqlStream) metaStore.getSource(SourceName.of("ORDERS"));
secondKStream = builder
Expand Down Expand Up @@ -264,7 +277,7 @@ public void init() {
queryBuilder,
mock(SqlPredicateFactory.class),
mock(AggregateParams.Factory.class),
new StreamsFactories(mockGroupedFactory, mockJoinedFactory, mockMaterializedFactory)
new StreamsFactories(mockGroupedFactory, mockJoinedFactory, mockMaterializedFactory, mockStreamJoinedFactory)
);
}

Expand Down Expand Up @@ -879,7 +892,7 @@ public void shouldPerformStreamToStreamLeftJoin() {
any(KStream.class),
any(KsqlValueJoiner.class),
any(JoinWindows.class),
any(Joined.class))
any(StreamJoined.class))
).thenReturn(mockKStream);
when(queryBuilder.buildValueSerde(any(), any(), any()))
.thenReturn(leftSerde)
Expand All @@ -900,12 +913,12 @@ public void shouldPerformStreamToStreamLeftJoin() {

// Then:
joinedKStream.getSourceStep().build(planBuilder);
verifyCreateJoined(rightSerde);
verifyCreateStreamJoined(rightSerde);
verify(mockKStream).leftJoin(
eq(secondKStream),
any(KsqlValueJoiner.class),
eq(joinWindow),
same(joined)
same(streamJoined)
);
assertThat(joinedKStream, instanceOf(SchemaKStream.class));
assertEquals(SchemaKStream.Type.JOIN, joinedKStream.type);
Expand Down Expand Up @@ -983,7 +996,7 @@ public void shouldPerformStreamToStreamInnerJoin() {
any(KStream.class),
any(KsqlValueJoiner.class),
any(JoinWindows.class),
any(Joined.class))
any(StreamJoined.class))
).thenReturn(mockKStream);
when(queryBuilder.buildValueSerde(any(), any(), any()))
.thenReturn(leftSerde)
Expand All @@ -1004,12 +1017,12 @@ public void shouldPerformStreamToStreamInnerJoin() {

// Then:
joinedKStream.getSourceStep().build(planBuilder);
verifyCreateJoined(rightSerde);
verifyCreateStreamJoined(rightSerde);
verify(mockKStream).join(
eq(secondKStream),
any(KsqlValueJoiner.class),
eq(joinWindow),
same(joined)
same(streamJoined)
);

assertThat(joinedKStream, instanceOf(SchemaKStream.class));
Expand All @@ -1031,7 +1044,7 @@ public void shouldPerformStreamToStreamOuterJoin() {
any(KStream.class),
any(KsqlValueJoiner.class),
any(JoinWindows.class),
any(Joined.class))
any(StreamJoined.class))
).thenReturn(mockKStream);
when(queryBuilder.buildValueSerde(any(), any(), any()))
.thenReturn(leftSerde)
Expand All @@ -1052,12 +1065,12 @@ public void shouldPerformStreamToStreamOuterJoin() {

// Then:
joinedKStream.getSourceStep().build(planBuilder);
verifyCreateJoined(rightSerde);
verifyCreateStreamJoined(rightSerde);
verify(mockKStream).outerJoin(
eq(secondKStream),
any(KsqlValueJoiner.class),
eq(joinWindow),
same(joined)
same(streamJoined)
);
assertThat(joinedKStream, instanceOf(SchemaKStream.class));
assertEquals(SchemaKStream.Type.JOIN, joinedKStream.type);
Expand Down Expand Up @@ -1311,6 +1324,16 @@ private void givenSourcePropertiesWithSchema(
);
}

private void verifyCreateStreamJoined(final Serde<GenericRow> rightSerde) {
verify(mockStreamJoinedFactory).create(
same(keySerde),
same(leftSerde),
same(rightSerde),
eq(StreamsUtil.buildOpName(childContextStacker.getQueryContext())),
eq(StreamsUtil.buildOpName(childContextStacker.getQueryContext()))
);
}

private SchemaKStream buildSchemaKStream(
final LogicalSchema schema,
final KeyField keyField,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@
import io.confluent.ksql.execution.expression.tree.ComparisonExpression;
import io.confluent.ksql.execution.expression.tree.Expression;
import io.confluent.ksql.execution.expression.tree.LongLiteral;
import io.confluent.ksql.execution.streams.StreamJoinedFactory;
import io.confluent.ksql.name.ColumnName;
import io.confluent.ksql.name.SourceName;
import io.confluent.ksql.schema.ksql.ColumnRef;
Expand Down Expand Up @@ -206,7 +207,8 @@ public void init() {
new StreamsFactories(
groupedFactory,
mock(JoinedFactory.class),
mock(MaterializedFactory.class)
mock(MaterializedFactory.class),
mock(StreamJoinedFactory.class)
)
);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -160,7 +160,7 @@ public <K> KStreamHolder<K> visitStreamStreamJoin(final StreamStreamJoin<K> join
right,
join,
queryBuilder,
streamsFactories.getJoinedFactory()
streamsFactories.getStreamJoinedFactory()
);
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
/*
* Copyright 2018 Confluent Inc.
*
* Licensed under the Confluent Community License (the "License"); you may not use
* this file except in compliance with the License. You may obtain a copy of the
* License at
*
* http://www.confluent.io/confluent-community-license
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/

package io.confluent.ksql.execution.streams;

import io.confluent.ksql.util.KsqlConfig;
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.streams.kstream.StreamJoined;

public interface StreamJoinedFactory {
<K, V, V0> StreamJoined<K, V, V0> create(
Serde<K> keySerde,
Serde<V> leftSerde,
Serde<V0> rightSerde,
String name,
String storeName);


static StreamJoinedFactory create(final KsqlConfig ksqlConfig) {
if (StreamsUtil.useProvidedName(ksqlConfig)) {
return new StreamJoinedFactory() {
@Override
public <K, V, V0> StreamJoined<K, V, V0> create(
final Serde<K> keySerde,
final Serde<V> leftSerde,
final Serde<V0> rightSerde,
final String name,
final String storeName) {
return StreamJoined.with(keySerde, leftSerde, rightSerde)
.withName(name).withStoreName(storeName);
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The withStoreName is the key change/addition. If we want to stick with generated store names, we can omit the withStoreName call and update the expected topologies for the QueryTranslationTest.

}
};
}
return new StreamJoinedFactory() {
@Override
public <K, V, V0> StreamJoined<K, V, V0> create(
final Serde<K> keySerde,
final Serde<V> leftSerde,
final Serde<V0> rightSerde,
final String name,
final String storeName) {
return StreamJoined.with(keySerde, leftSerde, rightSerde);
}
};
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,8 @@
import io.confluent.ksql.serde.KeySerde;
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.streams.kstream.JoinWindows;
import org.apache.kafka.streams.kstream.Joined;
import org.apache.kafka.streams.kstream.KStream;
import org.apache.kafka.streams.kstream.StreamJoined;

public final class StreamStreamJoinBuilder {
private static final String LEFT_SERDE_CTX = "left";
Expand All @@ -41,7 +41,7 @@ public static <K> KStreamHolder<K> build(
final KStreamHolder<K> right,
final StreamStreamJoin<K> join,
final KsqlQueryBuilder queryBuilder,
final JoinedFactory joinedFactory) {
final StreamJoinedFactory streamJoinedFactory) {
final Formats leftFormats = join.getLeftFormats();
final QueryContext queryContext = join.getProperties().getQueryContext();
final QueryContext.Stacker stacker = QueryContext.Stacker.of(queryContext);
Expand Down Expand Up @@ -71,10 +71,11 @@ public static <K> KStreamHolder<K> build(
leftPhysicalSchema,
queryContext
);
final Joined<K, GenericRow, GenericRow> joined = joinedFactory.create(
final StreamJoined<K, GenericRow, GenericRow> joined = streamJoinedFactory.create(
keySerde,
leftSerde,
rightSerde,
StreamsUtil.buildOpName(queryContext),
StreamsUtil.buildOpName(queryContext)
);
final KsqlValueJoiner joiner = new KsqlValueJoiner(leftSchema, rightSchema);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,23 +22,27 @@ public class StreamsFactories {
private final GroupedFactory groupedFactory;
private final JoinedFactory joinedFactory;
private final MaterializedFactory materializedFactory;
private final StreamJoinedFactory streamJoinedFactory;

public static StreamsFactories create(final KsqlConfig ksqlConfig) {
Objects.requireNonNull(ksqlConfig);
return new StreamsFactories(
GroupedFactory.create(ksqlConfig),
JoinedFactory.create(ksqlConfig),
MaterializedFactory.create(ksqlConfig)
MaterializedFactory.create(ksqlConfig),
StreamJoinedFactory.create(ksqlConfig)
);
}

public StreamsFactories(
final GroupedFactory groupedFactory,
final JoinedFactory joinedFactory,
final MaterializedFactory materializedFactory) {
final MaterializedFactory materializedFactory,
final StreamJoinedFactory streamJoinedFactory) {
this.groupedFactory = Objects.requireNonNull(groupedFactory);
this.joinedFactory = Objects.requireNonNull(joinedFactory);
this.materializedFactory = Objects.requireNonNull(materializedFactory);
this.streamJoinedFactory = Objects.requireNonNull(streamJoinedFactory);
}

public GroupedFactory getGroupedFactory() {
Expand All @@ -52,4 +56,8 @@ public JoinedFactory getJoinedFactory() {
public MaterializedFactory getMaterializedFactory() {
return materializedFactory;
}

public StreamJoinedFactory getStreamJoinedFactory() {
return streamJoinedFactory;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -197,7 +197,8 @@ public void init() {
new StreamsFactories(
mock(GroupedFactory.class),
mock(JoinedFactory.class),
materializedFactory
materializedFactory,
mock(StreamJoinedFactory.class)
)
);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -152,7 +152,8 @@ public void init() {
new StreamsFactories(
groupedFactory,
mock(JoinedFactory.class),
mock(MaterializedFactory.class)
mock(MaterializedFactory.class),
mock(StreamJoinedFactory.class)
)
);
streamGroupByKey = new StreamGroupByKey(PROPERTIES, sourceStep, FORMATS);
Expand Down
Loading