[GitHub] [kafka] bellemare commented on a change in pull request #8764: KAFKA-10049: Fixed FKJ bug where wrapped serdes are set incorrectly when using default StreamsConfig serdes

2020-06-09 Thread GitBox


bellemare commented on a change in pull request #8764:
URL: https://github.com/apache/kafka/pull/8764#discussion_r437560085



##
File path: 
streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableForeignKeyJoinScenarioTest.java
##
@@ -243,17 +244,17 @@ private void validateTopologyCanProcessData(final 
StreamsBuilder builder) {
 final String safeTestName = safeUniqueTestName(getClass(), testName);
 config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "dummy-" + 
safeTestName);
 config.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "dummy");
-config.setProperty(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, 
Serdes.StringSerde.class.getName());
+config.setProperty(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, 
Serdes.IntegerSerde.class.getName());
 config.setProperty(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, 
Serdes.StringSerde.class.getName());
 config.setProperty(StreamsConfig.STATE_DIR_CONFIG, 
TestUtils.tempDirectory().getAbsolutePath());
 try (final TopologyTestDriver topologyTestDriver = new 
TopologyTestDriver(builder.build(), config)) {
-final TestInputTopic aTopic = 
topologyTestDriver.createInputTopic("A", new StringSerializer(), new 
StringSerializer());
-final TestInputTopic bTopic = 
topologyTestDriver.createInputTopic("B", new StringSerializer(), new 
StringSerializer());
-final TestOutputTopic output = 
topologyTestDriver.createOutputTopic("output", new StringDeserializer(), new 
StringDeserializer());
-aTopic.pipeInput("a1", "b1-alpha");
-bTopic.pipeInput("b1", "beta");
-final Map x = output.readKeyValuesToMap();
-assertThat(x, is(Collections.singletonMap("a1", 
"(b1-alpha,(b1-alpha,beta))")));
+final TestInputTopic aTopic = 
topologyTestDriver.createInputTopic("A", new IntegerSerializer(), new 
StringSerializer());
+final TestInputTopic bTopic = 
topologyTestDriver.createInputTopic("B", new IntegerSerializer(), new 
StringSerializer());
+final TestOutputTopic output = 
topologyTestDriver.createOutputTopic("output", new IntegerDeserializer(), new 
StringDeserializer());
+aTopic.pipeInput(1, "1-alpha");
+bTopic.pipeInput(1, "beta");

Review comment:
   I swear I know how to read... T_T





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [kafka] bellemare commented on a change in pull request #8764: KAFKA-10049: Fixed FKJ bug where wrapped serdes are set incorrectly when using default StreamsConfig serdes

2020-06-09 Thread GitBox


bellemare commented on a change in pull request #8764:
URL: https://github.com/apache/kafka/pull/8764#discussion_r436871862



##
File path: 
streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableForeignKeyJoinScenarioTest.java
##
@@ -243,17 +244,17 @@ private void validateTopologyCanProcessData(final 
StreamsBuilder builder) {
 final String safeTestName = safeUniqueTestName(getClass(), testName);
 config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "dummy-" + 
safeTestName);
 config.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "dummy");
-config.setProperty(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, 
Serdes.StringSerde.class.getName());
+config.setProperty(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, 
Serdes.IntegerSerde.class.getName());
 config.setProperty(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, 
Serdes.StringSerde.class.getName());
 config.setProperty(StreamsConfig.STATE_DIR_CONFIG, 
TestUtils.tempDirectory().getAbsolutePath());
 try (final TopologyTestDriver topologyTestDriver = new 
TopologyTestDriver(builder.build(), config)) {
-final TestInputTopic aTopic = 
topologyTestDriver.createInputTopic("A", new StringSerializer(), new 
StringSerializer());
-final TestInputTopic bTopic = 
topologyTestDriver.createInputTopic("B", new StringSerializer(), new 
StringSerializer());
-final TestOutputTopic output = 
topologyTestDriver.createOutputTopic("output", new StringDeserializer(), new 
StringDeserializer());
-aTopic.pipeInput("a1", "b1-alpha");
-bTopic.pipeInput("b1", "beta");
-final Map x = output.readKeyValuesToMap();
-assertThat(x, is(Collections.singletonMap("a1", 
"(b1-alpha,(b1-alpha,beta))")));
+final TestInputTopic aTopic = 
topologyTestDriver.createInputTopic("A", new IntegerSerializer(), new 
StringSerializer());
+final TestInputTopic bTopic = 
topologyTestDriver.createInputTopic("B", new IntegerSerializer(), new 
StringSerializer());
+final TestOutputTopic output = 
topologyTestDriver.createOutputTopic("output", new IntegerDeserializer(), new 
StringDeserializer());
+aTopic.pipeInput(1, "1-alpha");
+bTopic.pipeInput(1, "beta");

Review comment:
   Agreed. Good catch.

##
File path: 
streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableForeignKeyJoinScenarioTest.java
##
@@ -181,60 +182,60 @@ public void shouldWorkWithDefaultAndProducedSerdes() {
 public void shouldUseExpectedTopicsWithSerde() {
 final String applicationId = "ktable-ktable-joinOnForeignKey";
 final Properties streamsConfig = mkProperties(mkMap(
-mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, applicationId),
-mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "asdf:"),
-mkEntry(StreamsConfig.STATE_DIR_CONFIG, 
TestUtils.tempDirectory().getPath())
+mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, applicationId),
+mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "asdf:"),
+mkEntry(StreamsConfig.STATE_DIR_CONFIG, 
TestUtils.tempDirectory().getPath())
 ));
 
 final UniqueTopicSerdeScope serdeScope = new UniqueTopicSerdeScope();
 final StreamsBuilder builder = new StreamsBuilder();
 
-final KTable left = builder.table(
-LEFT_TABLE,
-Consumed.with(serdeScope.decorateSerde(Serdes.String(), 
streamsConfig, true),
-  serdeScope.decorateSerde(Serdes.String(), 
streamsConfig, false))
+final KTable left = builder.table(
+LEFT_TABLE,
+Consumed.with(serdeScope.decorateSerde(Serdes.Integer(), 
streamsConfig, true),
+serdeScope.decorateSerde(Serdes.String(), 
streamsConfig, false))
 );
-final KTable right = builder.table(
-RIGHT_TABLE,
-Consumed.with(serdeScope.decorateSerde(Serdes.String(), 
streamsConfig, true),
-  serdeScope.decorateSerde(Serdes.String(), 
streamsConfig, false))
+final KTable right = builder.table(
+RIGHT_TABLE,
+Consumed.with(serdeScope.decorateSerde(Serdes.Integer(), 
streamsConfig, true),
+serdeScope.decorateSerde(Serdes.String(), 
streamsConfig, false))
 );
 
 left.join(
-right,
-value -> value.split("\\|")[1],
-(value1, value2) -> "(" + value1 + "," + value2 + ")",
-Materialized.with(null, serdeScope.decorateSerde(Serdes.String(), 
streamsConfig, false)
-))
-.toStream()
-.to(OUTPUT);
+right,
+value -> Integer.parseInt(value.split("\\|")[1]),
+(value1, value2) -> "(" + value1 + "," + value2 + ")",
+ 

[GitHub] [kafka] bellemare commented on a change in pull request #8764: KAFKA-10049: Fixed FKJ bug where wrapped serdes are set incorrectly when using default StreamsConfig serdes

2020-06-06 Thread GitBox


bellemare commented on a change in pull request #8764:
URL: https://github.com/apache/kafka/pull/8764#discussion_r436278818



##
File path: 
streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyJoinMaterializationIntegrationTest.java
##
@@ -94,15 +101,49 @@ public void before() {
 );
 }
 
+@Test
+public void shouldEmitRecordWhenJoiningWithDefaultSerdes() {

Review comment:
   Went with "keep it simple" basic Integer and String. 





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [kafka] bellemare commented on a change in pull request #8764: KAFKA-10049: Fixed FKJ bug where wrapped serdes are set incorrectly when using default StreamsConfig serdes

2020-06-05 Thread GitBox


bellemare commented on a change in pull request #8764:
URL: https://github.com/apache/kafka/pull/8764#discussion_r435886441



##
File path: 
streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyJoinMaterializationIntegrationTest.java
##
@@ -94,15 +101,49 @@ public void before() {
 );
 }
 
+@Test
+public void shouldEmitRecordWhenJoiningWithDefaultSerdes() {

Review comment:
   @vvcephei Just getting back to this now. Before I rework 
KTableKTableForeignKeyJoinScenarioTest:
   
   I can keep it simple and try something like:
   KTable aTable
   KTable bTable
   
   Or I can go ahead and use the JSONSerde, User, and Foo classes. I think the 
former will suffice for coverage purposes, but I like the latter because it may 
better reflect real-life use-cases (and I usually look to the tests myself to 
show me how to use a function). Thoughts on this before I refactor?





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [kafka] bellemare commented on a change in pull request #8764: KAFKA-10049: Fixed FKJ bug where wrapped serdes are set incorrectly when using default StreamsConfig serdes

2020-06-01 Thread GitBox


bellemare commented on a change in pull request #8764:
URL: https://github.com/apache/kafka/pull/8764#discussion_r433486855



##
File path: 
streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyJoinMaterializationIntegrationTest.java
##
@@ -94,15 +101,49 @@ public void before() {
 );
 }
 
+@Test
+public void shouldEmitRecordWhenJoiningWithDefaultSerdes() {

Review comment:
   Okay, will check.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [kafka] bellemare commented on a change in pull request #8764: KAFKA-10049: Fixed FKJ bug where wrapped serdes are set incorrectly when using default StreamsConfig serdes

2020-06-01 Thread GitBox


bellemare commented on a change in pull request #8764:
URL: https://github.com/apache/kafka/pull/8764#discussion_r433486418



##
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java
##
@@ -66,10 +66,13 @@ public void init(final InternalProcessorContext context) {
 valSerializer = (Serializer) context.valueSerde().serializer();
 }
 
-// if value serializers are internal wrapping serializers that may 
need to be given the default serializer
+// if serializers are internal wrapping serializers that may need to 
be given the default serializer
 // then pass it the default one from the context
 if (valSerializer instanceof WrappingNullableSerializer) {
-((WrappingNullableSerializer) 
valSerializer).setIfUnset(context.valueSerde().serializer());
+((WrappingNullableSerializer) valSerializer).setIfUnset(

Review comment:
   @guozhangwang This is 1/2 of the areas that needed the fix. The 
valueSerde was being passed into the underlying Serde, despite it really 
needing the keySerde. 

##
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java
##
@@ -87,10 +87,13 @@ public void init(final InternalProcessorContext context) {
 this.valDeserializer = (Deserializer) 
context.valueSerde().deserializer();
 }
 
-// if value deserializers are internal wrapping deserializers that may 
need to be given the default
+// if deserializers are internal wrapping deserializers that may need 
to be given the default
 // then pass it the default one from the context
 if (valDeserializer instanceof WrappingNullableDeserializer) {
-((WrappingNullableDeserializer) 
valDeserializer).setIfUnset(context.valueSerde().deserializer());
+((WrappingNullableDeserializer) valDeserializer).setIfUnset(

Review comment:
   @guozhangwang This is 2/2 of the areas that needed the fix. Same thing, 
the valueSerde was being passed into the underlying Serde, despite it really 
needing the keySerde. 





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org