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/12/01 14:19:44 UTC

[GitHub] [kafka] patrickstuedi commented on a change in pull request #11557: KAFKA-13491: IQv2 framework

patrickstuedi commented on a change in pull request #11557:
URL: https://github.com/apache/kafka/pull/11557#discussion_r760218888



##########
File path: streams/src/main/java/org/apache/kafka/streams/query/Position.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.query;
+
+
+import org.apache.kafka.common.annotation.InterfaceStability.Evolving;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+
+/**
+ * A representation of a position vector with respect to a set of topic partitions. For example, in
+ * Interactive Query ({@link org.apache.kafka.streams.KafkaStreams#query(StateQueryRequest)}, a
+ * query result may contain information from multiple store partitions, each of which contains
+ * information from multiple input topics' partitions. This class can be used to summarize all of
+ * that positional information.
+ * <p>
+ * This class is threadsafe, although it is mutable. Readers are recommended to use {@link
+ * Position#copy()} to avoid seeing mutations to the Position after they get the reference. For
+ * examples, when a store executes a {@link org.apache.kafka.streams.processor.StateStore#query(Query,
+ * PositionBound, boolean)} request and returns its current position via {@link
+ * QueryResult#setPosition(Position)}, it should pass a copy of its position instead of the mutable
+ * reference.
+ */
+@Evolving
+public class Position {
+
+    private final ConcurrentHashMap<String, ConcurrentHashMap<Integer, Long>> position;
+
+    private Position(final ConcurrentHashMap<String, ConcurrentHashMap<Integer, Long>> position) {
+        this.position = position;
+    }
+
+    /**
+     * Create a new, empty Position.
+     */
+    public static Position emptyPosition() {
+        return new Position(new ConcurrentHashMap<>());
+    }
+
+    /**
+     * Create a new Position and populate it with a mapping of topic -> partition -> offset.
+     * <p>
+     * Note, the resulting Position does not share any structure with the provided map, so
+     * subsequent changes to the map or Position will not affect the other.
+     */
+    public static Position fromMap(final Map<String, ? extends Map<Integer, Long>> map) {
+        return new Position(deepCopy(map));
+    }
+
+    /**
+     * Augment an existing Position by setting a new offset for a topic and partition.
+     * <p>
+     * Note: enforces monotonicity on offsets. I.e., if there is already a component for the same
+     * topic and partition with a larger offset, the update will succeed but not overwrite the
+     * offset.
+     * <p>
+     * Returns a self-reference for chained calls. Note: this method mutates the Position.
+     */
+    public Position withComponent(final String topic, final int partition, final long offset) {
+        position
+            .computeIfAbsent(topic, k -> new ConcurrentHashMap<>())
+            .compute(
+                partition,
+                (integer, prior) -> prior == null || offset > prior ? offset : prior

Review comment:
       Nice, this avoids the use of AtomicLong I guess! 
   
   Or, what about this case, is this safe?
   Long offset = position.get("topic").get(partition);
   if (offset > X){ //let's say there was some concurrent withComponent call on the same topic/partition here
   ..do something
   }

##########
File path: streams/src/main/java/org/apache/kafka/streams/query/Position.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.query;
+
+
+import org.apache.kafka.common.annotation.InterfaceStability.Evolving;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+
+/**
+ * A representation of a position vector with respect to a set of topic partitions. For example, in
+ * Interactive Query ({@link org.apache.kafka.streams.KafkaStreams#query(StateQueryRequest)}, a
+ * query result may contain information from multiple store partitions, each of which contains
+ * information from multiple input topics' partitions. This class can be used to summarize all of
+ * that positional information.
+ * <p>
+ * This class is threadsafe, although it is mutable. Readers are recommended to use {@link
+ * Position#copy()} to avoid seeing mutations to the Position after they get the reference. For
+ * examples, when a store executes a {@link org.apache.kafka.streams.processor.StateStore#query(Query,
+ * PositionBound, boolean)} request and returns its current position via {@link
+ * QueryResult#setPosition(Position)}, it should pass a copy of its position instead of the mutable
+ * reference.
+ */
+@Evolving
+public class Position {
+
+    private final ConcurrentHashMap<String, ConcurrentHashMap<Integer, Long>> position;
+
+    private Position(final ConcurrentHashMap<String, ConcurrentHashMap<Integer, Long>> position) {
+        this.position = position;
+    }
+
+    /**
+     * Create a new, empty Position.
+     */
+    public static Position emptyPosition() {
+        return new Position(new ConcurrentHashMap<>());
+    }
+
+    /**
+     * Create a new Position and populate it with a mapping of topic -> partition -> offset.
+     * <p>
+     * Note, the resulting Position does not share any structure with the provided map, so
+     * subsequent changes to the map or Position will not affect the other.
+     */
+    public static Position fromMap(final Map<String, ? extends Map<Integer, Long>> map) {
+        return new Position(deepCopy(map));
+    }
+
+    /**
+     * Augment an existing Position by setting a new offset for a topic and partition.
+     * <p>
+     * Note: enforces monotonicity on offsets. I.e., if there is already a component for the same
+     * topic and partition with a larger offset, the update will succeed but not overwrite the
+     * offset.
+     * <p>
+     * Returns a self-reference for chained calls. Note: this method mutates the Position.
+     */
+    public Position withComponent(final String topic, final int partition, final long offset) {
+        position
+            .computeIfAbsent(topic, k -> new ConcurrentHashMap<>())
+            .compute(
+                partition,
+                (integer, prior) -> prior == null || offset > prior ? offset : prior
+            );
+        return this;
+    }
+
+    /**
+     * Create a deep copy of the Position.
+     */
+    public Position copy() {
+        return new Position(deepCopy(position));

Review comment:
       Why not implement copy using merge?
   Position pos = Position.empyPosition();
   pos.merge(this);




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