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


##########
streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/FineGrainedAutoResetIntegrationTest.java:
##########
@@ -251,14 +263,108 @@ private void commitInvalidOffsets() {
         consumer.close();
     }
 
+    @Test
+    public void shouldFailForResetNone() throws Exception {
+        final Properties props = new Properties();
+        props.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000");
+
+        final Properties localConfig = StreamsTestUtils.getStreamsConfig(
+            "testConfigAutoOffsetWithNone",
+            CLUSTER.bootstrapServers(),
+            STRING_SERDE_CLASSNAME,
+            STRING_SERDE_CLASSNAME,
+            props);
+
+        final StreamsBuilder builder = new StreamsBuilder();
+        final KStream<String, String> exceptionStream = builder.stream(NOOP, 
Consumed.with(AutoOffsetReset.none()));
+
+        exceptionStream.to(DEFAULT_OUTPUT_TOPIC, Produced.with(stringSerde, 
stringSerde));
+
+        try (final KafkaStreams streams = new KafkaStreams(builder.build(), 
localConfig)) {
+            final TestingUncaughtExceptionHandler uncaughtExceptionHandler = 
new TestingUncaughtExceptionHandler();
+            streams.setUncaughtExceptionHandler(uncaughtExceptionHandler);
+
+            streams.start();
+
+            waitForCondition(
+                () -> uncaughtExceptionHandler.correctExceptionThrown,
+                "The expected NoOffsetForPartitionException was never thrown"
+            );
+        }
+    }
+
+    @Test
+    public void shouldResetByDuration() throws Exception {
+        final StreamsBuilder builder = new StreamsBuilder();
+
+        builder.<String, String>stream(TOPIC_DURATION_1, 
Consumed.with(AutoOffsetReset.byDuration(Duration.ofHours(6L).plus(Duration.ofMinutes(40L)))))
+            .to(OUTPUT_TOPIC_3, Produced.with(stringSerde, stringSerde));
+        builder.<String, String>stream(TOPIC_DURATION_2, 
Consumed.with(AutoOffsetReset.byDuration(Duration.ofMillis(mockTime.milliseconds()).minus(Duration.ofDays(1L)))))
+            .to(OUTPUT_TOPIC_4, Produced.with(stringSerde, stringSerde));
+        builder.<String, String>stream(TOPIC_DURATION_3, 
Consumed.with(AutoOffsetReset.byDuration(Duration.ZERO)))
+            .to(OUTPUT_TOPIC_5, Produced.with(stringSerde, stringSerde));
+
+        final Properties producerConfig = 
TestUtils.producerConfig(CLUSTER.bootstrapServers(), StringSerializer.class, 
StringSerializer.class);
+
+        for (int i = 0; i < 10; ++i) {
+            mockTime.sleep(Duration.ofHours(1L).toMillis());

Review Comment:
   How does this work? Unless I'm missing something, this looks to me like it 
should pause the test for an hour.



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java:
##########
@@ -1344,19 +1360,30 @@ private <S extends StateStore> InternalTopicConfig 
createChangelogTopicConfig(fi
     }
 
     public boolean hasOffsetResetOverrides() {
-        return !(earliestResetTopics.isEmpty() && 
earliestResetPatterns.isEmpty()
-            && latestResetTopics.isEmpty() && latestResetPatterns.isEmpty());
+        return noneResetTopics.size() + noneResetPatterns.size()
+            + earliestResetTopics.size() + earliestResetPatterns.size()
+            + latestResetTopics.size() + latestResetPatterns.size()
+            + durationResetTopics.size() + durationResetPatterns.size() > 0;
     }
 
     public AutoOffsetResetStrategy offsetResetStrategy(final String topic) {
-        if 
(maybeDecorateInternalSourceTopics(earliestResetTopics).contains(topic) ||
+        final Set<Duration> resetDuration = new HashSet<>();
+
+        if (maybeDecorateInternalSourceTopics(noneResetTopics).contains(topic) 
||
+            noneResetPatterns.stream().anyMatch(p -> 
p.matcher(topic).matches())) {
+            return AutoOffsetResetStrategy.NONE;
+        } else if 
(maybeDecorateInternalSourceTopics(earliestResetTopics).contains(topic) ||
             earliestResetPatterns.stream().anyMatch(p -> 
p.matcher(topic).matches())) {
             return AutoOffsetResetStrategy.EARLIEST;
         } else if 
(maybeDecorateInternalSourceTopics(latestResetTopics).contains(topic) ||
             latestResetPatterns.stream().anyMatch(p -> 
p.matcher(topic).matches())) {
             return AutoOffsetResetStrategy.LATEST;
+        } else if 
(maybeDecorateInternalSourceTopics(durationResetTopics.keySet()).contains(topic))
 {
+            return AutoOffsetResetStrategy.fromString("by_duration:" + 
durationResetTopics.get(topic).toString());
+        } else if (findDuration(topic, resetDuration)) {

Review Comment:
   I'm thinking this could be simplified (part 1, part 2 in next comment below)
   
   ```
    public AutoOffsetResetStrategy offsetResetStrategy(final String topic) {
          final Optional<Duration> duration;
          if (...) {
              ....
          } else if ((duration = findDuration(topic)).isPresent()) {
                    return AutoOffsetResetStrategy.fromString("by_duration:" + 
duration);
         }
       ...
   ```



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java:
##########
@@ -1366,6 +1393,19 @@ public AutoOffsetResetStrategy offsetResetStrategy(final 
String topic) {
         }
     }
 
+    private boolean findDuration(final String topic, final Set<Duration> 
resetDuration) {

Review Comment:
   Part 2 of simplification
   
   ```
   private Optional<Duration> findDuration(final String topic) {
         return  durationResetPatterns.entrySet().stream()
               .filter(e -> e.getKey().matcher(topic).matches())
               .map(Map.Entry::getValue)
                   .findAny();
       }
   ```
   



##########
streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java:
##########
@@ -144,7 +144,7 @@ public synchronized <K, V> KStream<K, V> stream(final 
String topic,
      * @return a {@link KStream} for the specified topics
      */
     public synchronized <K, V> KStream<K, V> stream(final Collection<String> 
topics) {
-        return stream(topics, Consumed.with(null, null, null, null));
+        return stream(topics, Consumed.with(null, null));

Review Comment:
   You may have explained this before - but why this change? (is this from the 
other PR and due to the compiler and overloads?)



##########
streams/src/test/java/org/apache/kafka/streams/AutoOffsetResetTest.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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 org.apache.kafka.streams.internals.AutoOffsetResetInternal;
+
+import org.junit.jupiter.api.Test;
+
+import java.time.Duration;
+import java.util.NoSuchElementException;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+class AutoOffsetResetTest {
+
+    @Test
+    void noneShouldReturnAnEmptyDuration() {
+        final AutoOffsetResetInternal none = new 
AutoOffsetResetInternal(AutoOffsetReset.none());
+        assertThrows(NoSuchElementException.class, none::duration, "None 
should not have a duration.");
+    }
+
+    @Test
+    void latestShouldReturnAnEmptyDuration() {
+        final AutoOffsetResetInternal latest = new 
AutoOffsetResetInternal(AutoOffsetReset.latest());
+        assertThrows(NoSuchElementException.class, latest::duration, "Latest 
should not have a duration.");
+    }
+
+    @Test
+    void earliestShouldReturnAnEmptyDuration() {
+        final AutoOffsetResetInternal earliest = new 
AutoOffsetResetInternal(AutoOffsetReset.earliest());

Review Comment:
   nit: these 3 tests have the name `xShouldReturnAnEmptyDuration()` but the 
actual behavior is to throw a `NoSuchElementException` (which is expected by 
calling `get` on an empty `Optional`) - it's a little confusing it made my 
think it would return `Optional.empty()`



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to