bbejeck commented on code in PR #17973:
URL: https://github.com/apache/kafka/pull/17973#discussion_r1879018744


##########
streams/src/main/java/org/apache/kafka/streams/Topology.java:
##########
@@ -252,6 +325,27 @@ public synchronized Topology addSource(final 
AutoOffsetReset offsetReset,
         return this;
     }
 
+    /**
+     * Adds a new source that consumes from topics matching the given pattern 
with a specified {@link TimestampExtractor}
+     * and forwards the records to child processor and/or sink nodes.
+     * The source will use the provided timestamp extractor to determine the 
timestamp of each record.
+     *
+     * @param offsetReset the auto offset reset policy to use if no committed 
offsets are found
+     * @param timestampExtractor the timestamp extractor to use for this source
+     * @param name the unique name of the source used to reference this node 
when {@link #addProcessor(String, ProcessorSupplier, String...) adding 
processor children}
+     * @param topicPattern the regular expression pattern to match Kafka 
topics that this source is to consume
+     * @return itself
+     * @throws TopologyException if a processor is already added or if topics 
have already been registered by another source
+     */
+    public synchronized Topology addSource(final 
org.apache.kafka.streams.AutoOffsetReset offsetReset,

Review Comment:
   nit: since the `AutoOffsetReset` enum in the `Topology` class is being 
deprecated, maybe switch to have the full path for that and import the new 
`AutoOffsetReset` class here and the other uses.



##########
streams/src/main/java/org/apache/kafka/streams/kstream/Consumed.java:
##########
@@ -55,176 +56,223 @@ public class Consumed<K, V> implements 
NamedOperation<Consumed<K, V>> {
     protected Serde<K> keySerde;
     protected Serde<V> valueSerde;
     protected TimestampExtractor timestampExtractor;
-    protected Topology.AutoOffsetReset resetPolicy;
+    @Deprecated
+    protected Topology.AutoOffsetReset legacyResetPolicy;
+    protected AutoOffsetReset resetPolicy;
     protected String processorName;
 
+    @SuppressWarnings("deprecation")
     private Consumed(final Serde<K> keySerde,
                      final Serde<V> valueSerde,
                      final TimestampExtractor timestampExtractor,
-                     final Topology.AutoOffsetReset resetPolicy,
+                     final Topology.AutoOffsetReset legacyResetPolicy,
+                     final AutoOffsetReset resetPolicy,
                      final String processorName) {
         this.keySerde = keySerde;
         this.valueSerde = valueSerde;
         this.timestampExtractor = timestampExtractor;
+        this.legacyResetPolicy = legacyResetPolicy;
         this.resetPolicy = resetPolicy;
         this.processorName = processorName;
     }
 
+    /**
+     * Create an instance of {@link Consumed} from an existing instance.
+     * @param consumed  the instance of {@link Consumed} to copy
+     */
     protected Consumed(final Consumed<K, V> consumed) {
-        this(consumed.keySerde,
-             consumed.valueSerde,
-             consumed.timestampExtractor,
-             consumed.resetPolicy,
-             consumed.processorName
+        this(
+            consumed.keySerde,
+            consumed.valueSerde,
+            consumed.timestampExtractor,
+            consumed.legacyResetPolicy,
+            consumed.resetPolicy,

Review Comment:
   Why do we need both reset policies here?



##########
streams/src/main/java/org/apache/kafka/streams/AutoOffsetReset.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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;
+
+import java.time.Duration;
+import java.util.Optional;
+
+/**
+ * Sets the {@code auto.offset.reset} configuration when
+ * {@link #addSource(AutoOffsetReset, String, String...) adding a source 
processor} or when creating {@link KStream}
+ * or {@link KTable} via {@link StreamsBuilder}.
+ */
+public class AutoOffsetReset {

Review Comment:
   I'm also good with the name, it makes sense to me as it is an internal class.



##########
streams/src/main/java/org/apache/kafka/streams/kstream/Consumed.java:
##########
@@ -55,176 +56,223 @@ public class Consumed<K, V> implements 
NamedOperation<Consumed<K, V>> {
     protected Serde<K> keySerde;
     protected Serde<V> valueSerde;
     protected TimestampExtractor timestampExtractor;
-    protected Topology.AutoOffsetReset resetPolicy;
+    @Deprecated
+    protected Topology.AutoOffsetReset legacyResetPolicy;
+    protected AutoOffsetReset resetPolicy;
     protected String processorName;
 
+    @SuppressWarnings("deprecation")
     private Consumed(final Serde<K> keySerde,
                      final Serde<V> valueSerde,
                      final TimestampExtractor timestampExtractor,
-                     final Topology.AutoOffsetReset resetPolicy,
+                     final Topology.AutoOffsetReset legacyResetPolicy,
+                     final AutoOffsetReset resetPolicy,
                      final String processorName) {
         this.keySerde = keySerde;
         this.valueSerde = valueSerde;
         this.timestampExtractor = timestampExtractor;
+        this.legacyResetPolicy = legacyResetPolicy;
         this.resetPolicy = resetPolicy;
         this.processorName = processorName;
     }
 
+    /**
+     * Create an instance of {@link Consumed} from an existing instance.
+     * @param consumed  the instance of {@link Consumed} to copy
+     */
     protected Consumed(final Consumed<K, V> consumed) {
-        this(consumed.keySerde,
-             consumed.valueSerde,
-             consumed.timestampExtractor,
-             consumed.resetPolicy,
-             consumed.processorName
+        this(
+            consumed.keySerde,
+            consumed.valueSerde,
+            consumed.timestampExtractor,
+            consumed.legacyResetPolicy,
+            consumed.resetPolicy,

Review Comment:
   NM I can see why now.



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