You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "vcrfxia (via GitHub)" <gi...@apache.org> on 2023/02/16 21:27:49 UTC

[GitHub] [kafka] vcrfxia opened a new pull request, #13264: KAFKA-14491: [12/N] Relax requirement that KTable stores must be TimestampedKVStores

vcrfxia opened a new pull request, #13264:
URL: https://github.com/apache/kafka/pull/13264

   As part of introducing versioned key-value stores in [KIP-889](https://cwiki.apache.org/confluence/display/KAFKA/KIP-889%3A+Versioned+State+Stores), we want to lift the existing DSL restriction that KTable stores are always TimestampedKeyValueStores to allow for KTable stores which are VersionedKeyValueStores instead. This PR lifts the restriction by replacing raw usages of TimestampedKeyValueStore with a new KeyValueStoreWrapper which supports either TimestampedKeyValueStore or VersionedKeyValueStore. (Until versioned stores are actually exposed to users in a follow-up PR, all KTables stores will continue to be TimestampedKeyValueStores.)
   
   The loss of type-safety from `StoreBuilder<TimestampedKeyValueStore<K, VR>>` to `StoreBuilder<?>` is unfortunate but necessary in order to allow for top-level VersionedKeyValueStores, so that users may access versioned stores for use in `transform()` operations and interactive queries.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


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


[GitHub] [kafka] mjsax commented on a diff in pull request #13264: KAFKA-14491: [12/N] Relax requirement that KTable stores must be TimestampedKVStores

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13264:
URL: https://github.com/apache/kafka/pull/13264#discussion_r1122527645


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueStoreWrapper.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 final StateStore store;
+    private final StoreType storeType;
+
+    private enum StoreType {
+        TIMESTAMPED,
+        VERSIONED;
+    }
+
+    public KeyValueStoreWrapper(final ProcessorContext<?, ?> context, final String storeName) {
+        store = context.getStateStore(storeName);
+
+        if (store instanceof TimestampedKeyValueStore) {

Review Comment:
   I think this is fine.



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


[GitHub] [kafka] mjsax commented on pull request #13264: KAFKA-14491: [12/N] Relax requirement that KTable stores must be TimestampedKVStores

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on PR #13264:
URL: https://github.com/apache/kafka/pull/13264#issuecomment-1451176639

   There is test failures. Could it be related to this PR?
   ```
   org.apache.kafka.streams.kstream.internals.KTableKTableInnerJoinTest.shouldLogAndMeterSkippedRecordsDueToNullLeftKey
   java.lang.NullPointerException: Cannot invoke "Object.getClass()" because "this.store" is null
   ```


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


[GitHub] [kafka] mjsax merged pull request #13264: KAFKA-14491: [12/N] Relax requirement that KTable stores must be TimestampedKVStores

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax merged PR #13264:
URL: https://github.com/apache/kafka/pull/13264


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


[GitHub] [kafka] mjsax commented on a diff in pull request #13264: KAFKA-14491: [12/N] Relax requirement that KTable stores must be TimestampedKVStores

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13264:
URL: https://github.com/apache/kafka/pull/13264#discussion_r1117760309


##########
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.");

Review Comment:
   Should we include the type of the store we found in the error message?



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


[GitHub] [kafka] mjsax commented on a diff in pull request #13264: KAFKA-14491: [12/N] Relax requirement that KTable stores must be TimestampedKVStores

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13264:
URL: https://github.com/apache/kafka/pull/13264#discussion_r1117773343


##########
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:
   Why can't we keep the generic typer here? I understand that we need to make it `<?>` in `KTableImpl` but not sure why we need `<?>` here, too?



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


[GitHub] [kafka] vcrfxia commented on a diff in pull request #13264: KAFKA-14491: [12/N] Relax requirement that KTable stores must be TimestampedKVStores

Posted by "vcrfxia (via GitHub)" <gi...@apache.org>.
vcrfxia commented on code in PR #13264:
URL: https://github.com/apache/kafka/pull/13264#discussion_r1117838013


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueStoreWrapper.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 final StateStore store;
+    private final StoreType storeType;
+
+    private enum StoreType {
+        TIMESTAMPED,
+        VERSIONED;
+    }
+
+    public KeyValueStoreWrapper(final ProcessorContext<?, ?> context, final String storeName) {
+        store = context.getStateStore(storeName);
+
+        if (store instanceof TimestampedKeyValueStore) {

Review Comment:
   It's interesting that these checks don't verify the generic types (i.e., `K` and `V`) but IIUC the code in `ProcessorContextImpl#getStateStore()` for casting state stores to the requested type does not check generic types either, so I don't believe there's a visible change in behavior here.



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


[GitHub] [kafka] mjsax commented on a diff in pull request #13264: KAFKA-14491: [12/N] Relax requirement that KTable stores must be TimestampedKVStores

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13264:
URL: https://github.com/apache/kafka/pull/13264#discussion_r1122525554


##########
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:
   My proposal was actually to have three member, the previously existing `KeyValueStore keyValueStore`, `VersionedStore versionStore` and a new `Store store`. So we just simplify the code for the case when we don't care about the store type, but when we care we check if `keyValueStore != null` or `versionStore != null` and do the right thing.
   
   Sorry for not explaining it good enough -- using an enum works, too, I guess, but also tend to agree it a little overkill. But you made the change already... So maybe also not worth to revert it again to sue `KeyValueStore keyValueStore` and `VersionedStore versionStore` as you did originally.
   
   The only question (by might be pre-mature optimization): we know need to cast on very single get/put call using the enum. Sounds like more overhead that using KeyValueStore keyValueStore` / `VersionedStore versionStore` and check for `null` as originally?



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


[GitHub] [kafka] vcrfxia commented on pull request #13264: KAFKA-14491: [12/N] Relax requirement that KTable stores must be TimestampedKVStores

Posted by "vcrfxia (via GitHub)" <gi...@apache.org>.
vcrfxia commented on PR #13264:
URL: https://github.com/apache/kafka/pull/13264#issuecomment-1451404037

   > There is test failures. Could it be related to this PR?
   
   Good catch. These failures only appeared when I switched from the original try-catch approach for casting stores to either TimestampedKeyValueStore or VersionedKeyValueStore, to fetching the store as a generic StateStore and then performing `instanceof` checks. The failures happened because the unit tests use a mock context which returns null for the state store, which failed the `instanceof` checks. I'm pretty sure in the actual code (non-mocked contexts) it's not possible for `context.getStateStore()` to return null and therefore this "issue" is limited to unit tests only, but I've updated the PR to return to the original try-catch approach for casting in order to be safe (and in light of your other comment above, regarding type casts on every `put/get` call).


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


[GitHub] [kafka] mjsax commented on a diff in pull request #13264: KAFKA-14491: [12/N] Relax requirement that KTable stores must be TimestampedKVStores

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13264:
URL: https://github.com/apache/kafka/pull/13264#discussion_r1117769530


##########
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:
   If we add the `Store store` member, it seem we can cut 50% of the tests below?



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


[GitHub] [kafka] mjsax commented on a diff in pull request #13264: KAFKA-14491: [12/N] Relax requirement that KTable stores must be TimestampedKVStores

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13264:
URL: https://github.com/apache/kafka/pull/13264#discussion_r1117773343


##########
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:
   Why can't we keep the generic typer here? I understand that we need to make it `<?>` in `KTableImpl` (and the "graph node" code below) but not sure why we need `<?>` here, too?



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


[GitHub] [kafka] mjsax commented on a diff in pull request #13264: KAFKA-14491: [12/N] Relax requirement that KTable stores must be TimestampedKVStores

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13264:
URL: https://github.com/apache/kafka/pull/13264#discussion_r1122525901


##########
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:
   I we need it later, it's fine to do it right away.



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


[GitHub] [kafka] vcrfxia commented on a diff in pull request #13264: KAFKA-14491: [12/N] Relax requirement that KTable stores must be TimestampedKVStores

Posted by "vcrfxia (via GitHub)" <gi...@apache.org>.
vcrfxia commented on code in PR #13264:
URL: https://github.com/apache/kafka/pull/13264#discussion_r1122676573


##########
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:
   Yeah, your point about casting overhead is valid. I had hoped it'd be fairly minimal, but that's moot now since I went ahead and updated the PR to use your proposal (three store pointers) in response to your comment about test failures (see below).



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


[GitHub] [kafka] mjsax commented on a diff in pull request #13264: KAFKA-14491: [12/N] Relax requirement that KTable stores must be TimestampedKVStores

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13264:
URL: https://github.com/apache/kafka/pull/13264#discussion_r1122525554


##########
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:
   My proposal was actually to have three member, the previously existing `KeyValueStore keyValueStore`, `VersionedStore versionStore` and a new `Store store`. So we just simplify the code for the case when we don't care about the store type, but when we care we check if `keyValueStore != null` or `versionStore != null` and do the right thing.
   
   Sorry for not explaining it good enough -- using an enum works, too, I guess, but also tend to agree it a little overkill. But you made the change already... So maybe also not worth to revert it again to sue `KeyValueStore keyValueStore` and `VersionedStore versionStore` as you did originally.



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


[GitHub] [kafka] mjsax commented on a diff in pull request #13264: KAFKA-14491: [12/N] Relax requirement that KTable stores must be TimestampedKVStores

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13264:
URL: https://github.com/apache/kafka/pull/13264#discussion_r1122525901


##########
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:
   I we need it later, it's fine to do it right away. Maybe a little cleaner from a "commit history" POV do only to it later.



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


[GitHub] [kafka] mjsax commented on a diff in pull request #13264: KAFKA-14491: [12/N] Relax requirement that KTable stores must be TimestampedKVStores

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13264:
URL: https://github.com/apache/kafka/pull/13264#discussion_r1122527645


##########
streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueStoreWrapper.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 final StateStore store;
+    private final StoreType storeType;
+
+    private enum StoreType {
+        TIMESTAMPED,
+        VERSIONED;
+    }
+
+    public KeyValueStoreWrapper(final ProcessorContext<?, ?> context, final String storeName) {
+        store = context.getStateStore(storeName);
+
+        if (store instanceof TimestampedKeyValueStore) {

Review Comment:
   I think this is fine. -- Even if we have the cast as in the original code, I don't think it would be safer. The compile check cannot verify the generic types anyway, because `getStateStore` does not contain any information about it -- and at runtime, the generic types are gone and the actually "cast" if necessary from `Object` to `K` (or `V`) would happen somewhere else in the code.



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


[GitHub] [kafka] mjsax commented on a diff in pull request #13264: KAFKA-14491: [12/N] Relax requirement that KTable stores must be TimestampedKVStores

Posted by "mjsax (via GitHub)" <gi...@apache.org>.
mjsax commented on code in PR #13264:
URL: https://github.com/apache/kafka/pull/13264#discussion_r1117766784


##########
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:
   Do we really need this method? It seem whoever uses `KeyValueStoreWrapper` can keep a handle of the store themselves?
   
   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`? (Similar for `name()`, `init()` etc below?)
   
   If we don't get the correct stores passed into the constructor we fail anyway, and it seem a lot of unncessary boiler plate code in those "non-functional helpers"



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


[GitHub] [kafka] vcrfxia commented on a diff in pull request #13264: KAFKA-14491: [12/N] Relax requirement that KTable stores must be TimestampedKVStores

Posted by "vcrfxia (via GitHub)" <gi...@apache.org>.
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-unsubscribe@kafka.apache.org

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