vcrfxia commented on code in PR #13264:
URL: https://github.com/apache/kafka/pull/13264#discussion_r1117829563


##########
streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueStoreWrapperTest.java:
##########
@@ -0,0 +1,356 @@
+/*
+ * 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 static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.assertThrows;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.apache.kafka.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.query.PositionBound;
+import org.apache.kafka.streams.query.Query;
+import org.apache.kafka.streams.query.QueryConfig;
+import org.apache.kafka.streams.query.QueryResult;
+import org.apache.kafka.streams.state.KeyValueStore;
+import org.apache.kafka.streams.state.TimestampedKeyValueStore;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+@RunWith(MockitoJUnitRunner.StrictStubs.class)
+public class KeyValueStoreWrapperTest {
+
+    private static final String STORE_NAME = "kvStore";
+    private static final String KEY = "k";
+    private static final ValueAndTimestamp<String> VALUE_AND_TIMESTAMP
+        = ValueAndTimestamp.make("v", 8L);
+
+    @Mock
+    private TimestampedKeyValueStore<String, String> timestampedStore;
+    @Mock
+    private VersionedKeyValueStore<String, String> versionedStore;
+    @Mock
+    private ProcessorContext context;
+    @Mock
+    private Query query;
+    @Mock
+    private PositionBound positionBound;
+    @Mock
+    private QueryConfig queryConfig;
+    @Mock
+    private QueryResult result;
+    @Mock
+    private Position position;
+
+    private KeyValueStoreWrapper<String, String> wrapper;
+
+    @Test
+    public void shouldThrowOnNonTimestampedOrVersionedStore() {
+        
when(context.getStateStore(STORE_NAME)).thenReturn(mock(KeyValueStore.class));
+
+        assertThrows(InvalidStateStoreException.class, () -> new 
KeyValueStoreWrapper<>(context, STORE_NAME));
+    }
+
+    @Test
+    public void shouldGetFromTimestampedStore() {
+        givenWrapperWithTimestampedStore();
+        when(timestampedStore.get(KEY)).thenReturn(VALUE_AND_TIMESTAMP);
+
+        assertThat(wrapper.get(KEY), equalTo(VALUE_AND_TIMESTAMP));
+    }
+
+    @Test
+    public void shouldGetFromVersionedStore() {
+        givenWrapperWithVersionedStore();
+        when(versionedStore.get(KEY)).thenReturn(
+            new VersionedRecord<>(
+                VALUE_AND_TIMESTAMP.value(),
+                VALUE_AND_TIMESTAMP.timestamp())
+        );
+
+        assertThat(wrapper.get(KEY), equalTo(VALUE_AND_TIMESTAMP));
+    }
+
+    @Test
+    public void shouldGetNullFromTimestampedStore() {
+        givenWrapperWithTimestampedStore();
+        when(timestampedStore.get(KEY)).thenReturn(null);
+
+        assertThat(wrapper.get(KEY), nullValue());
+    }
+
+    @Test
+    public void shouldGetNullFromVersionedStore() {
+        givenWrapperWithVersionedStore();
+        when(versionedStore.get(KEY)).thenReturn(null);
+
+        assertThat(wrapper.get(KEY), nullValue());
+    }
+
+    @Test
+    public void shouldPutToTimestampedStore() {
+        givenWrapperWithTimestampedStore();
+
+        wrapper.put(KEY, VALUE_AND_TIMESTAMP.value(), 
VALUE_AND_TIMESTAMP.timestamp());
+
+        verify(timestampedStore).put(KEY, VALUE_AND_TIMESTAMP);
+    }
+
+    @Test
+    public void shouldPutToVersionedStore() {
+        givenWrapperWithVersionedStore();
+
+        wrapper.put(KEY, VALUE_AND_TIMESTAMP.value(), 
VALUE_AND_TIMESTAMP.timestamp());
+
+        verify(versionedStore).put(KEY, VALUE_AND_TIMESTAMP.value(), 
VALUE_AND_TIMESTAMP.timestamp());
+    }
+
+    @Test
+    public void shouldPutNullToTimestampedStore() {
+        givenWrapperWithTimestampedStore();
+
+        wrapper.put(KEY, null, VALUE_AND_TIMESTAMP.timestamp());
+
+        verify(timestampedStore).put(KEY, null);
+    }
+
+    @Test
+    public void shouldPutNullToVersionedStore() {
+        givenWrapperWithVersionedStore();
+
+        wrapper.put(KEY, null, VALUE_AND_TIMESTAMP.timestamp());
+
+        verify(versionedStore).put(KEY, null, VALUE_AND_TIMESTAMP.timestamp());
+    }
+
+    @Test
+    public void shouldGetTimestampedStore() {

Review Comment:
   I added the `Store store` member which indeed simplified the code in 
`KeyValueStoreWrapper`, but have chosen to keep these tests here for coverage.



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimestampedKeyValueStoreMaterializer.java:
##########
@@ -33,7 +32,7 @@ public TimestampedKeyValueStoreMaterializer(final 
MaterializedInternal<K, V, Key
     /**
      * @return  StoreBuilder
      */
-    public StoreBuilder<TimestampedKeyValueStore<K, V>> materialize() {
+    public StoreBuilder<?> materialize() {

Review Comment:
   You're right; we don't need to make this change yet but it will be required 
as part of the changes in https://github.com/apache/kafka/pull/13274. I can 
back it out here and reapply in https://github.com/apache/kafka/pull/13274 
instead.



##########
streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueStoreWrapper.java:
##########
@@ -0,0 +1,204 @@
+/*
+ * 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.streams.errors.InvalidStateStoreException;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.StateStoreContext;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
+import org.apache.kafka.streams.query.Position;
+import org.apache.kafka.streams.query.PositionBound;
+import org.apache.kafka.streams.query.Query;
+import org.apache.kafka.streams.query.QueryConfig;
+import org.apache.kafka.streams.query.QueryResult;
+import org.apache.kafka.streams.state.TimestampedKeyValueStore;
+import org.apache.kafka.streams.state.ValueAndTimestamp;
+import org.apache.kafka.streams.state.VersionedKeyValueStore;
+import org.apache.kafka.streams.state.VersionedRecord;
+
+/**
+ * A wrapper class for non-windowed key-value stores used within the DSL. All 
such stores are
+ * instances of either {@link TimestampedKeyValueStore} or {@link 
VersionedKeyValueStore}.
+ *
+ * @param <K> The key type
+ * @param <V> The value type
+ */
+public class KeyValueStoreWrapper<K, V> implements StateStore {
+
+    private TimestampedKeyValueStore<K, V> timestampedStore = null;
+    private VersionedKeyValueStore<K, V> versionedStore = null;
+
+    public KeyValueStoreWrapper(final ProcessorContext<?, ?> context, final 
String storeName) {
+        try {
+            // first try timestamped store
+            timestampedStore = context.getStateStore(storeName);
+            return;
+        } catch (final ClassCastException e) {
+            // ignore since could be versioned store instead
+        }
+
+        try {
+            // next try versioned store
+            versionedStore = context.getStateStore(storeName);
+        } catch (final ClassCastException e) {
+            throw new InvalidStateStoreException("KTable source state store 
must implement either TimestampedKeyValueStore or VersionedKeyValueStore.");
+        }
+    }
+
+    public ValueAndTimestamp<V> get(final K key) {
+        if (timestampedStore != null) {
+            return timestampedStore.get(key);
+        }
+        if (versionedStore != null) {
+            final VersionedRecord<V> versionedRecord = versionedStore.get(key);
+            return versionedRecord == null
+                ? null
+                : ValueAndTimestamp.make(versionedRecord.value(), 
versionedRecord.timestamp());
+        }
+        throw new IllegalStateException("KeyValueStoreWrapper must be 
initialized with either timestamped or versioned store");
+    }
+
+    public void put(final K key, final V value, final long timestamp) {
+        if (timestampedStore != null) {
+            timestampedStore.put(key, ValueAndTimestamp.make(value, 
timestamp));
+            return;
+        }
+        if (versionedStore != null) {
+            versionedStore.put(key, value, timestamp);
+            return;
+        }
+        throw new IllegalStateException("KeyValueStoreWrapper must be 
initialized with either timestamped or versioned store");
+    }
+
+    public StateStore getStore() {

Review Comment:
   I prefer to keep this method because otherwise either (1) both the caller 
and `KeyValueStoreWrapper` would have to call 
`context.getStateStore(storeName)` when the caller needs the state store in 
addition to the wrapper, which feels redundant, or (2) if we avoid the 
duplication by not having the wrapper call `context.getStateStore(storeName)` 
then the caller would always have to call `context.getStateStore(storeName)` 
even when it doesn't need the state store itself (only the wrapper), which 
feels clunky.
   
   > If we think it's convenient to keep this helper, should we add a Store 
store member variable, and implement this one a simple return store?
   
   We can do that. Inside `KeyValueStoreWrapper` we still have to track whether 
the store is a timestamped store or a versioned store, in order to use the 
appropriate interfaces in `put()` and `get()`. I could introduce a private enum 
for this purpose; feels a little overkill but I can't think of anything better. 
I'll go ahead and make the change.



-- 
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-unsubscr...@kafka.apache.org

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

Reply via email to