You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/09/10 04:53:28 UTC

[GitHub] [kafka] mjsax commented on a change in pull request #11252: KAFKA-13216: Use a KV with list serde for the shared store

mjsax commented on a change in pull request #11252:
URL: https://github.com/apache/kafka/pull/11252#discussion_r705883780



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImplJoin.java
##########
@@ -280,85 +279,51 @@ private void assertUniqueStoreNames(final WindowBytesStoreSupplier supplier,
         }
     }
 
-    @SuppressWarnings("unchecked")
-    private <K, V1, V2> StoreBuilder<WindowStore<KeyAndJoinSide<K>, LeftOrRightValue<V1, V2>>> sharedOuterJoinWindowStoreBuilder(final JoinWindows windows,
-                                                                                                                                 final StreamJoinedInternal<K, V1, V2> streamJoinedInternal,
-                                                                                                                                 final String joinThisGeneratedName) {
+    private <K, V1, V2> StoreBuilder<KeyValueStore<TimestampedKeyAndJoinSide<K>, LeftOrRightValue<V1, V2>>> sharedOuterJoinWindowStoreBuilder(final JoinWindows windows,
+                                                                                                                                              final StreamJoinedInternal<K, V1, V2> streamJoinedInternal,
+                                                                                                                                              final String joinThisGeneratedName) {
         final boolean persistent = streamJoinedInternal.thisStoreSupplier() == null || streamJoinedInternal.thisStoreSupplier().get().persistent();
-        final String storeName = buildOuterJoinWindowStoreName(streamJoinedInternal, joinThisGeneratedName);
-
-        final KeyAndJoinSideSerde keyAndJoinSideSerde = new KeyAndJoinSideSerde<>(streamJoinedInternal.keySerde());
-        final LeftOrRightValueSerde leftOrRightValueSerde = new LeftOrRightValueSerde(streamJoinedInternal.valueSerde(), streamJoinedInternal.otherValueSerde());
-
-        final StoreBuilder<WindowStore<KeyAndJoinSide<K>, LeftOrRightValue<V1, V2>>> builder;
-        if (persistent) {
-            builder = new TimeOrderedWindowStoreBuilder<KeyAndJoinSide<K>, LeftOrRightValue<V1, V2>>(
-                persistentTimeOrderedWindowStore(
-                    storeName + "-store",
-                    Duration.ofMillis(windows.size() + windows.gracePeriodMs()),
-                    Duration.ofMillis(windows.size())
-                ),
-                keyAndJoinSideSerde,
-                leftOrRightValueSerde,
-                Time.SYSTEM
-            );
-        } else {
-            builder = Stores.windowStoreBuilder(
-                Stores.inMemoryWindowStore(
-                    storeName + "-store",
-                    Duration.ofMillis(windows.size() + windows.gracePeriodMs()),
-                    Duration.ofMillis(windows.size()),
-                    false
-                ),
-                keyAndJoinSideSerde,
-                leftOrRightValueSerde
-            );
-        }
-
-        if (streamJoinedInternal.loggingEnabled()) {
-            builder.withLoggingEnabled(streamJoinedInternal.logConfig());
-        } else {
-            builder.withLoggingDisabled();
-        }
-
-        return builder;
-    }
+        final String storeName = buildOuterJoinWindowStoreName(streamJoinedInternal, joinThisGeneratedName) + "-store";

Review comment:
       Did we miss to add the `"-store"` suffix originally?

##########
File path: streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimeOrderedKeyValueBytesStore.java
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.state.internals;
+
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.state.KeyValueStore;
+
+public class ChangeLoggingTimeOrderedKeyValueBytesStore extends ChangeLoggingKeyValueBytesStore {

Review comment:
       Should this class be called `ChangeLoggingListValueByteStore` ?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java
##########
@@ -209,37 +208,37 @@ private void emitNonJoinedOuterRecords(final WindowStore<KeyAndJoinSide<K>, Left
             // reset to MAX_VALUE in case the store is empty
             sharedTimeTracker.minTime = Long.MAX_VALUE;
 
-            try (final KeyValueIterator<Windowed<KeyAndJoinSide<K>>, LeftOrRightValue> it = store.all()) {
+            try (final KeyValueIterator<TimestampedKeyAndJoinSide<K>, LeftOrRightValue<V1, V2>> it = store.all()) {
                 while (it.hasNext()) {
-                    final KeyValue<Windowed<KeyAndJoinSide<K>>, LeftOrRightValue> record = it.next();
+                    final KeyValue<TimestampedKeyAndJoinSide<K>, LeftOrRightValue<V1, V2>> record = it.next();
 
-                    final Windowed<KeyAndJoinSide<K>> windowedKey = record.key;
-                    final LeftOrRightValue value = record.value;
-                    sharedTimeTracker.minTime = windowedKey.window().start();
+                    final TimestampedKeyAndJoinSide<K> timestampedKeyAndJoinSide = record.key;
+                    final LeftOrRightValue<V1, V2> value = record.value;
+                    final K key = timestampedKeyAndJoinSide.getKey();
+                    final long timestamp = timestampedKeyAndJoinSide.getTimestamp();
+                    sharedTimeTracker.minTime = timestamp;
 
                     // Skip next records if window has not closed
-                    if (windowedKey.window().start() + joinAfterMs + joinGraceMs >= sharedTimeTracker.streamTime) {
+                    if (timestamp + joinAfterMs + joinGraceMs >= sharedTimeTracker.streamTime) {
                         break;
                     }
 
-                    final K key = windowedKey.key().getKey();
-                    final long time = windowedKey.window().start();
-
                     final R nullJoinedValue;
                     if (isLeftSide) {
                         nullJoinedValue = joiner.apply(key,
-                            (V1) value.getLeftValue(),
-                            (V2) value.getRightValue());
+                                value.getLeftValue(),
+                                value.getRightValue());

Review comment:
       Why do we not need to cast here any longer, but move the cast to the `else` branch ?

##########
File path: streams/src/test/java/org/apache/kafka/streams/state/internals/ListValueStoreTest.java
##########
@@ -0,0 +1,174 @@
+/*
+ * 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.state.internals;
+
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.serialization.Serde;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.internals.MockStreamsMetrics;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.test.InternalMockProcessorContext;
+import org.apache.kafka.test.MockRecordCollector;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+
+import static java.util.Arrays.asList;
+import static org.apache.kafka.test.StreamsTestUtils.toList;
+import static org.junit.Assert.assertEquals;
+
+public class ListValueStoreTest {
+    private static final String STORE_NAME = "rocksDB list value store";
+
+    MockRecordCollector recordCollector;
+    KeyValueStore<Integer, String> listStore;
+    InternalMockProcessorContext<Integer, String> context;
+
+    final File baseDir = TestUtils.tempDirectory("test");
+
+    @Before
+    public void setup() {
+        listStore = buildStore(Serdes.Integer(), Serdes.String());
+
+        recordCollector = new MockRecordCollector();
+        context = new InternalMockProcessorContext<>(
+            baseDir,
+            Serdes.String(),
+            Serdes.Integer(),
+            recordCollector,
+            new ThreadCache(
+                new LogContext("testCache"),
+                0,
+                new MockStreamsMetrics(new Metrics())));
+        context.setTime(1L);
+
+        listStore.init((StateStoreContext) context, listStore);
+    }
+
+    @After
+    public void after() {
+        listStore.close();
+    }
+
+    <K, V> KeyValueStore<K, V> buildStore(final Serde<K> keySerde,
+                                          final Serde<V> valueSerde) {
+        return new ListValueStoreBuilder<>(
+            new RocksDbKeyValueBytesStoreSupplier(STORE_NAME, false),
+            keySerde,
+            valueSerde,
+            Time.SYSTEM)
+            .build();
+    }
+
+    @Test
+    public void shouldGetAll() {
+        listStore.put(0, "zero");
+        // should retain duplicates
+        listStore.put(0, "zero again");
+        listStore.put(1, "one");
+        listStore.put(2, "two");
+
+        final KeyValue<Integer, String> zero = KeyValue.pair(0, "zero");
+        final KeyValue<Integer, String> zeroAgain = KeyValue.pair(0, "zero again");
+        final KeyValue<Integer, String> one = KeyValue.pair(1, "one");
+        final KeyValue<Integer, String> two = KeyValue.pair(2, "two");
+
+        assertEquals(
+            asList(zero, zeroAgain, one, two),
+            toList(listStore.all())
+        );
+    }
+
+    @Test
+    public void shouldGetAllNonDeletedRecords() {
+        // Add some records
+        listStore.put(0, "zero");
+        listStore.put(1, "one");
+        listStore.put(1, "one again");
+        listStore.put(2, "two");
+        listStore.put(3, "three");
+        listStore.put(4, "four");
+
+        // Delete some records
+        listStore.put(1, null);
+        listStore.put(3, null);
+
+        // Only non-deleted records should appear in the all() iterator
+        final KeyValue<Integer, String> zero = KeyValue.pair(0, "zero");
+        final KeyValue<Integer, String> two = KeyValue.pair(2, "two");
+        final KeyValue<Integer, String> four = KeyValue.pair(4, "four");
+
+        assertEquals(
+            asList(zero, two, four),
+            toList(listStore.all())
+        );
+    }
+
+    @Test
+    public void shouldGetAllReturnTimestampOrderedRecords() {
+        // Add some records in different order
+        listStore.put(4, "four");
+        listStore.put(0, "zero");
+        listStore.put(2, "two1");
+        listStore.put(3, "three");
+        listStore.put(1, "one");
+
+        // Add duplicates
+        listStore.put(2, "two2");
+
+        // Only non-deleted records should appear in the all() iterator
+        final KeyValue<Integer, String> zero = KeyValue.pair(0, "zero");
+        final KeyValue<Integer, String> one = KeyValue.pair(1, "one");
+        final KeyValue<Integer, String> two1 = KeyValue.pair(2, "two1");
+        final KeyValue<Integer, String> two2 = KeyValue.pair(2, "two2");
+        final KeyValue<Integer, String> three = KeyValue.pair(3, "three");
+        final KeyValue<Integer, String> four = KeyValue.pair(4, "four");
+
+        assertEquals(
+            asList(zero, one, two1, two2, three, four),
+            toList(listStore.all())
+        );
+    }
+
+    @Test
+    public void shouldEarlyClosedIteratorStillGetAllRecords() {
+        listStore.put(0, "zero");
+        listStore.put(1, "one");
+
+        final KeyValue<Integer, String> zero = KeyValue.pair(0, "zero");
+        final KeyValue<Integer, String> one = KeyValue.pair(1, "one");
+
+        final KeyValueIterator<Integer, String> it = listStore.all();
+        assertEquals(zero, it.next());
+        it.close();
+
+        // A new all() iterator after a previous all() iterator was closed should return all elements.

Review comment:
       Why is this a requirement?
   
   Should we instead on call `valueList.removeAll()` on `iter.close()`? -- In this test, if there would be 3 keys, the third key won't be returned either after `close()` to why allow to keep iteration on the current key ?

##########
File path: streams/src/main/java/org/apache/kafka/streams/state/internals/ListValueStore.java
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.state.internals;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import org.apache.kafka.common.utils.AbstractIterator;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serde;
+
+/**
+ * A wrapper key-value store that serializes the record values bytes as a list.
+ * As a result put calls would be interpreted as a get-append-put to the underlying RocksDB store.

Review comment:
       Can we add: `A put(k,null) will still delete the key, ie, the full list of all values of this key.`

##########
File path: streams/src/main/java/org/apache/kafka/streams/state/internals/ListValueStore.java
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.state.internals;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import org.apache.kafka.common.utils.AbstractIterator;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serde;
+
+/**
+ * A wrapper key-value store that serializes the record values bytes as a list.
+ * As a result put calls would be interpreted as a get-append-put to the underlying RocksDB store.
+ * Range iterators would also flatten the value lists and return the values one-by-one.
+ *
+ * This store is used for cases where we do not want to de-duplicate values of the same keys but want to retain all such values.
+ */
+@SuppressWarnings("unchecked")
+public class ListValueStore
+    extends WrappedStateStore<KeyValueStore<Bytes, byte[]>, Bytes, byte[]>
+    implements KeyValueStore<Bytes, byte[]> {
+
+    static private final Serde<List<byte[]>> LIST_SERDE = Serdes.ListSerde(ArrayList.class, Serdes.ByteArray());
+
+    ListValueStore(final KeyValueStore<Bytes, byte[]> bytesStore) {
+        super(bytesStore);
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value) {
+        // if the value is null we can skip the get and blind delete
+        if (value == null) {
+            wrapped().put(key, null);
+        } else {
+            final byte[] oldValue = wrapped().get(key);
+
+            if (oldValue == null) {
+                wrapped().put(key, LIST_SERDE.serializer().serialize(null, Collections.singletonList(value)));
+            } else {
+                final List<byte[]> list = LIST_SERDE.deserializer().deserialize(null, oldValue);
+                list.add(value);
+
+                wrapped().put(key, LIST_SERDE.serializer().serialize(null, list));
+            }
+        }
+    }
+
+    @Override
+    public byte[] putIfAbsent(final Bytes key, final byte[] value) {
+        throw new UnsupportedOperationException("putIfAbsent not supported");
+    }
+
+    @Override
+    public void putAll(final List<KeyValue<Bytes, byte[]>> entries) {
+        throw new UnsupportedOperationException("putAll not supported");
+    }
+
+    @Override
+    public byte[] delete(final Bytes key) {
+        // we intentionally disable delete calls since the returned bytes would
+        // represent a list, not a single value; we need to have a new API for delete if we do need it
+        throw new UnsupportedOperationException("delete not supported");

Review comment:
       I am fine with this, but I don't think I agree to the reasoning. `delete(k)` is just syntactic sugar for `put(k, null)`, and `get()` also returns the full list of all values... Thus, this store has some inconsistencies to a regular kv-store all over the place anyway... (what is still ok, as it's an internal store).

##########
File path: streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimeOrderedKeyValueBytesStore.java
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.state.internals;
+
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.state.KeyValueStore;
+
+public class ChangeLoggingTimeOrderedKeyValueBytesStore extends ChangeLoggingKeyValueBytesStore {
+
+    ChangeLoggingTimeOrderedKeyValueBytesStore(final KeyValueStore<Bytes, byte[]> inner) {
+        super(inner);
+    }
+
+    @Override
+    public void put(final Bytes key,
+                    final byte[] value) {
+        wrapped().put(key, value);
+        // we need to log the new value, which is different from the put value;

Review comment:
       Can we update this comment to be clearer:
   ```
   // the provided new value will be added to the list in the inner put()
   // we need to log the full new list and thus call get() on the inner store below
   // if the value is a tombstone, we delete the whole list and thus can save the get call
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/state/internals/ListValueStore.java
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.state.internals;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import org.apache.kafka.common.utils.AbstractIterator;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.state.KeyValueIterator;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.Serde;
+
+/**
+ * A wrapper key-value store that serializes the record values bytes as a list.
+ * As a result put calls would be interpreted as a get-append-put to the underlying RocksDB store.
+ * Range iterators would also flatten the value lists and return the values one-by-one.
+ *
+ * This store is used for cases where we do not want to de-duplicate values of the same keys but want to retain all such values.
+ */
+@SuppressWarnings("unchecked")
+public class ListValueStore
+    extends WrappedStateStore<KeyValueStore<Bytes, byte[]>, Bytes, byte[]>
+    implements KeyValueStore<Bytes, byte[]> {
+
+    static private final Serde<List<byte[]>> LIST_SERDE = Serdes.ListSerde(ArrayList.class, Serdes.ByteArray());
+
+    ListValueStore(final KeyValueStore<Bytes, byte[]> bytesStore) {
+        super(bytesStore);
+    }
+
+    @Override
+    public void put(final Bytes key, final byte[] value) {
+        // if the value is null we can skip the get and blind delete
+        if (value == null) {
+            wrapped().put(key, null);
+        } else {
+            final byte[] oldValue = wrapped().get(key);
+
+            if (oldValue == null) {
+                wrapped().put(key, LIST_SERDE.serializer().serialize(null, Collections.singletonList(value)));
+            } else {
+                final List<byte[]> list = LIST_SERDE.deserializer().deserialize(null, oldValue);
+                list.add(value);
+
+                wrapped().put(key, LIST_SERDE.serializer().serialize(null, list));
+            }
+        }
+    }
+
+    @Override
+    public byte[] putIfAbsent(final Bytes key, final byte[] value) {
+        throw new UnsupportedOperationException("putIfAbsent not supported");
+    }
+
+    @Override
+    public void putAll(final List<KeyValue<Bytes, byte[]>> entries) {
+        throw new UnsupportedOperationException("putAll not supported");
+    }
+
+    @Override
+    public byte[] delete(final Bytes key) {
+        // we intentionally disable delete calls since the returned bytes would
+        // represent a list, not a single value; we need to have a new API for delete if we do need it
+        throw new UnsupportedOperationException("delete not supported");
+    }
+
+    @Override
+    public byte[] get(final Bytes key) {
+        return wrapped().get(key);
+    }
+
+    @Override
+    public KeyValueIterator<Bytes, byte[]> range(final Bytes from, final Bytes to) {
+        throw new UnsupportedOperationException("range not supported");
+    }
+
+    @Override
+    public KeyValueIterator<Bytes, byte[]> all() {
+        return new WrappedStoreIterator(wrapped().all());
+    }
+
+    @Override
+    public long approximateNumEntries() {
+        return wrapped().approximateNumEntries();
+    }
+
+    private static class WrappedStoreIterator extends AbstractIterator<KeyValue<Bytes, byte[]>>

Review comment:
       Should we call this class `ValueListIterator` (or similar) -- "Wrapped" is not very descriptive.

##########
File path: streams/src/test/java/org/apache/kafka/streams/state/internals/TimestampedTimestampedKeyAndJoinSideSerializerTest.java
##########
@@ -24,49 +24,49 @@
 import static org.hamcrest.Matchers.notNullValue;
 import static org.junit.Assert.assertThrows;
 
-public class KeyAndJoinSideSerializerTest {
+public class TimestampedTimestampedKeyAndJoinSideSerializerTest {

Review comment:
       This comment was not addressed yet.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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