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/04/13 22:58:33 UTC

[GitHub] [kafka] vcrfxia commented on a diff in pull request #13565: KAFKA-14834: [N/9] Disable versioned-stores for unsupported operations

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


##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java:
##########
@@ -164,6 +166,11 @@ public <K, V> GlobalKTable<K, V> globalTable(final String topic,
                                                  final MaterializedInternal<K, V, KeyValueStore<Bytes, byte[]>> materialized) {
         Objects.requireNonNull(consumed, "consumed can't be null");
         Objects.requireNonNull(materialized, "materialized can't be null");
+
+        if (materialized.storeSupplier() instanceof VersionedBytesStoreSupplier) {
+            throw new IllegalArgumentException("GlobalTables cannot be versioned.");

Review Comment:
   Would TopologyException be more appropriate? 



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/SuppressNode.java:
##########
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals.graph;
+
+import org.apache.kafka.streams.state.StoreBuilder;
+
+public class SuppressNode<K, V> extends StatefulProcessorNode<K, V> {

Review Comment:
   nit: TableSuppressNode, instead of SuppressNode, for consistency with the other named node classes? I'm fine either way, since it seems unlikely that stream suppress will be introduced in the future.



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java:
##########
@@ -539,6 +540,10 @@ public <K1> KStream<K1, V> toStream(final KeyValueMapper<? super K, ? super V, ?
 
     @Override
     public KTable<K, V> suppress(final Suppressed<? super K> suppressed) {
+        if (graphNode.isOutputVersioned().isPresent() && graphNode.isOutputVersioned().get()) {

Review Comment:
   Makes sense. Could be good to leave a comment in the code so future readers know that this check is incomplete, and that the one in InternalStreamsBuilder is the "real" one.
   
   Should we also update the other error message (in InternalStreamsBuilder) to make clear to users that versioned-ness can be inherited downstream, since they will only hit that exception if that's the case? Don't think it's necessary, just a thought.
   
   EDIT: just saw your other comment below with the same question haha.



##########
streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java:
##########
@@ -369,6 +369,8 @@ public synchronized <K, V> KTable<K, V> table(final String topic,
      * <p>
      * Note that {@link GlobalKTable} always applies {@code "auto.offset.reset"} strategy {@code "earliest"}
      * regardless of the specified value in {@link StreamsConfig} or {@link Consumed}.
+     * Furthermore, {@link GlobalKTable} cannot be a {@link org.apache.kafka.streams.state.VersionedBytesStoreSupplier

Review Comment:
   Does it make more sense to point to VersionedBytesStoreSupplier in these docs notes or VersionedKeyValueStore? I see the merits of both, might be inconsistent right now -- the other docs I added, e.g., for TopologyTestDriver link to VersionedKeyValueStore, but perhaps that example is different since it's for getting the store rather than supplying one.



##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java:
##########
@@ -611,6 +621,11 @@ private <K, V> GroupedInternal<K, V> getRepartitionSerdes(final Collection<Optim
 
     private void enableVersionedSemantics() {
         versionedSemanticsNodes.forEach(node -> ((VersionedSemanticsGraphNode) node).enableVersionedSemantics(isVersionedUpstream(node)));
+        suppressNodesNodes.forEach(node -> {
+            if (isVersionedUpstream(node)) {
+                throw new UnsupportedOperationException("suppress() is only supported for non-versioned KTables");

Review Comment:
   TopologyException? (No strong opinion, just wondering.)



##########
streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java:
##########
@@ -433,6 +433,21 @@ public void shouldUseSerdesDefinedInMaterializedToConsumeGlobalTable() {
         }
     }
 
+    @Test
+    public void shouldThrowOnVersionedStoreSupplierForGlobalTable() {

Review Comment:
   Thanks for adding this. You'll also want to remove the existing integration test in VersionedKeyValueStoreIntegrationTest, and the relevant unit tests in KStreamGlobalKTableJoinTest and KStreamGlobalKTableLeftJoinTest.



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