omkreddy commented on code in PR #17858:
URL: https://github.com/apache/kafka/pull/17858#discussion_r1851276930


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AutoOffsetResetStrategy.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.clients.consumer.internals;
+
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigException;
+
+import java.util.Objects;
+
+public class AutoOffsetResetStrategy {
+    public static final String EARLIEST_STRATEGY_NAME = "earliest";
+    public static final String LATEST_STRATEGY_NAME = "latest";
+    public static final String NONE_STRATEGY_NAME = "none";

Review Comment:
   Enum wont help us as we still need to parse strings like 
`by_duration:PnDTnHnMn. n>. I prefer to keep string values for now. I will 
update code avoid string comparison in SteamsThread class



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AutoOffsetResetStrategy.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.clients.consumer.internals;
+
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigException;
+
+import java.util.Objects;
+
+public class AutoOffsetResetStrategy {
+    public static final String EARLIEST_STRATEGY_NAME = "earliest";
+    public static final String LATEST_STRATEGY_NAME = "latest";
+    public static final String NONE_STRATEGY_NAME = "none";
+
+    public static final AutoOffsetResetStrategy EARLIEST = new 
AutoOffsetResetStrategy(EARLIEST_STRATEGY_NAME);
+    public static final AutoOffsetResetStrategy LATEST = new 
AutoOffsetResetStrategy(LATEST_STRATEGY_NAME);
+    public static final AutoOffsetResetStrategy NONE = new 
AutoOffsetResetStrategy(NONE_STRATEGY_NAME);

Review Comment:
   These are not boilerplate objects. These are static instances to represent 
current strategies.
   The main reason for converting enum into class is to allow us to add 
additional reset strategies with some properties. 
   
   For example, we will be adding new class like below to represent duration 
based reset  strategy: 
   ``` 
    DurationBasedOffsetResetStrategy extends AutoOffsetResetStrategy {
       DurationBasedOffsetResetStrategy(String name, String isoDuration) {
       }
    }    
   ```
   



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java:
##########
@@ -1294,19 +1294,19 @@ private void resetOffsets(final Set<TopicPartition> 
partitions, final Exception
         final Set<TopicPartition> notReset = new HashSet<>();
 
         for (final TopicPartition partition : partitions) {
-            final OffsetResetStrategy offsetResetStrategy = 
topologyMetadata.offsetResetStrategy(partition.topic());
+            final AutoOffsetResetStrategy offsetResetStrategy = 
topologyMetadata.offsetResetStrategy(partition.topic());
 
             // This may be null if the task we are currently processing was 
apart of a named topology that was just removed.
             // TODO KAFKA-13713: keep the StreamThreads and TopologyMetadata 
view of named topologies in sync until final thread has acked
             if (offsetResetStrategy != null) {
-                switch (offsetResetStrategy) {
-                    case EARLIEST:
+                switch (offsetResetStrategy.name()) {

Review Comment:
   As mentioned in other comment,  enum wont help much here. We can be having 
different instances DurationBasedStrategy class.  I have the updated code to 
avoid string comparisons.



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