vvcephei commented on a change in pull request #9338:
URL: https://github.com/apache/kafka/pull/9338#discussion_r504228659



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java
##########
@@ -103,6 +104,13 @@ public void init(final StateStoreContext context,
         // register and possibly restore the state from the logs
         maybeMeasureLatency(() -> super.init(context, root), time, 
restoreSensor);
     }
+    protected Serde<K> prepareKeySerde(final Serde<K> keySerde, final Serde<?> 
contextKeySerde, final Serde<?> contextValueSerde) {

Review comment:
       This can be `private`

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java
##########
@@ -112,28 +120,26 @@ private void registerMetrics() {
     }
 
     @Deprecated
-    @SuppressWarnings("unchecked")
     void initStoreSerde(final ProcessorContext context) {

Review comment:
       This (and the StateStoreContext one) can be `private` now. Thanks for 
factoring out the smallest chunk that needs to be overridden in the subclass. I 
wish I'd thought of it.

##########
File path: 
streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyJoinDistributedTest.java
##########
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (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.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.streams.integration;
+
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.Topology;
+import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
+import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
+import org.apache.kafka.streams.kstream.KTable;
+import org.apache.kafka.streams.kstream.ValueJoiner;
+import org.apache.kafka.streams.processor.ThreadMetadata;
+import org.apache.kafka.test.IntegrationTest;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
+import static org.junit.Assert.assertEquals;
+
+@Category({IntegrationTest.class})
+public class KTableKTableForeignKeyJoinDistributedTest {
+    private static final int NUM_BROKERS = 1;
+    private static final String LEFT_TABLE = "left_table";
+    private static final String RIGHT_TABLE = "right_table";
+    private static final String OUTPUT = "output-topic";
+    @ClassRule
+    public static final EmbeddedKafkaCluster CLUSTER = new 
EmbeddedKafkaCluster(NUM_BROKERS);
+    private static final Properties CONSUMER_CONFIG = new Properties();
+
+    @Rule
+    public TestName testName = new TestName();
+
+
+    private static final String INPUT_TOPIC = "input-topic";
+
+    private KafkaStreams client1;
+    private KafkaStreams client2;
+
+    private volatile boolean client1IsOk = false;
+    private volatile boolean client2IsOk = false;
+
+    @BeforeClass
+    public static void createTopics() throws InterruptedException {
+        CLUSTER.createTopic(INPUT_TOPIC, 2, 1);
+    }
+
+    @Before
+    public void setupTopics() throws InterruptedException {
+        CLUSTER.createTopic(LEFT_TABLE, 1, 1);
+        CLUSTER.createTopic(RIGHT_TABLE, 1, 1);
+        CLUSTER.createTopic(OUTPUT, 11, 1);
+
+        //Fill test tables
+        final Properties producerConfig = new Properties();
+        producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, 
CLUSTER.bootstrapServers());
+        producerConfig.put(ProducerConfig.ACKS_CONFIG, "all");
+        producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, 
StringSerializer.class);
+        producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, 
StringSerializer.class);
+        final List<KeyValue<String, String>> leftTable = Arrays.asList(
+                new KeyValue<>("lhsValue1", "lhsValue1|rhs1"),
+                new KeyValue<>("lhsValue2", "lhsValue2|rhs2"),
+                new KeyValue<>("lhsValue3", "lhsValue3|rhs3"),
+                new KeyValue<>("lhsValue4", "lhsValue4|rhs4")
+        );
+        final List<KeyValue<String, String>> rightTable = Arrays.asList(
+                new KeyValue<>("rhs1", "rhsValue1"),
+                new KeyValue<>("rhs2", "rhsValue2"),
+                new KeyValue<>("rhs3", "rhsValue3")
+        );
+
+        IntegrationTestUtils.produceKeyValuesSynchronously(LEFT_TABLE, 
leftTable, producerConfig, CLUSTER.time);
+        IntegrationTestUtils.produceKeyValuesSynchronously(RIGHT_TABLE, 
rightTable, producerConfig, CLUSTER.time);
+
+        CONSUMER_CONFIG.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, 
CLUSTER.bootstrapServers());
+        CONSUMER_CONFIG.put(ConsumerConfig.GROUP_ID_CONFIG, 
"ktable-ktable-distributed-consumer");
+        CONSUMER_CONFIG.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, 
StringDeserializer.class);
+        CONSUMER_CONFIG.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, 
StringDeserializer.class);
+    }
+
+    @After
+    public void after() {
+        client1.close();
+        client2.close();

Review comment:
       It would be good to also clean up the local disk and the topics after 
the test: 
`org.apache.kafka.streams.integration.utils.IntegrationTestUtils#quietlyCleanStateAfterTest`
   
   Not always necessary, but it keeps tests from becoming flaky after seemingly 
unrelated changes.

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/kstream/internals/WrappingNullableSerde.java
##########
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (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.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serializer;
+
+public interface WrappingNullableSerde<T, InnerK, InnerV> extends Serde<T> {
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    default void setIfUnset(final Serde<InnerK> defaultKeySerde, final 
Serde<InnerV> defaultValueSerde) {
+        final Serializer<T> serializer = this.serializer();

Review comment:
       This is probably hypocritical, since I've surely written similar code, 
but this makes me a little uncomfortable, since there's no guarantee that 
`serializer()` would return the same reference every time it's called. I'd 
propose to make this an abstract class instead so we can make this guarantee 
ourselves.




----------------------------------------------------------------
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


Reply via email to