apoorvmittal10 commented on code in PR #18096:
URL: https://github.com/apache/kafka/pull/18096#discussion_r1876851848
##########
core/src/main/java/kafka/server/share/SharePartition.java:
##########
@@ -2125,16 +2128,25 @@ private long startOffsetDuringInitialization(long
partitionDataStartOffset) thro
if (partitionDataStartOffset !=
PartitionFactory.UNINITIALIZED_START_OFFSET) {
return partitionDataStartOffset;
}
- GroupConfig.ShareGroupAutoOffsetReset offsetResetStrategy;
+ ShareGroupAutoOffsetResetStrategy offsetResetStrategy;
if (groupConfigManager.groupConfig(groupId).isPresent()) {
offsetResetStrategy =
groupConfigManager.groupConfig(groupId).get().shareAutoOffsetReset();
} else {
offsetResetStrategy = GroupConfig.defaultShareAutoOffsetReset();
}
- if (offsetResetStrategy ==
GroupConfig.ShareGroupAutoOffsetReset.EARLIEST)
+ if (offsetResetStrategy.timestamp().isEmpty()) {
+ throw new Exception("The timestamp is not available for the share
partition: " + topicIdPartition);
Review Comment:
This will translate to UnknownServerError to the client. Do we want to
return that or specific exception?
##########
core/src/main/java/kafka/server/share/SharePartition.java:
##########
@@ -2125,16 +2128,25 @@ private long startOffsetDuringInitialization(long
partitionDataStartOffset) thro
if (partitionDataStartOffset !=
PartitionFactory.UNINITIALIZED_START_OFFSET) {
return partitionDataStartOffset;
}
- GroupConfig.ShareGroupAutoOffsetReset offsetResetStrategy;
+ ShareGroupAutoOffsetResetStrategy offsetResetStrategy;
if (groupConfigManager.groupConfig(groupId).isPresent()) {
offsetResetStrategy =
groupConfigManager.groupConfig(groupId).get().shareAutoOffsetReset();
} else {
offsetResetStrategy = GroupConfig.defaultShareAutoOffsetReset();
}
- if (offsetResetStrategy ==
GroupConfig.ShareGroupAutoOffsetReset.EARLIEST)
+ if (offsetResetStrategy.timestamp().isEmpty()) {
Review Comment:
Could simplify the code with `orElseThrow` method.
##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/ShareGroupAutoOffsetResetStrategy.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.coordinator.group;
+
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.requests.ListOffsetsRequest;
+import org.apache.kafka.common.utils.Utils;
+
+import java.time.Duration;
+import java.time.Instant;
+import java.util.Arrays;
+import java.util.Locale;
+import java.util.Objects;
+import java.util.Optional;
+
+public class ShareGroupAutoOffsetResetStrategy {
+ public enum StrategyType {
+ LATEST, EARLIEST, BY_DURATION;
+
+ @Override
+ public String toString() {
+ return super.toString().toLowerCase(Locale.ROOT);
+ }
+ }
+
+ public static final ShareGroupAutoOffsetResetStrategy EARLIEST = new
ShareGroupAutoOffsetResetStrategy(StrategyType.EARLIEST);
+ public static final ShareGroupAutoOffsetResetStrategy LATEST = new
ShareGroupAutoOffsetResetStrategy(StrategyType.LATEST);
+
+ private final StrategyType type;
+ private final Optional<Duration> duration;
+
+ private ShareGroupAutoOffsetResetStrategy(StrategyType type) {
+ this.type = type;
+ this.duration = Optional.empty();
+ }
+
+ private ShareGroupAutoOffsetResetStrategy(Duration duration) {
+ this.type = StrategyType.BY_DURATION;
+ this.duration = Optional.of(duration);
+ }
+
+ /**
+ * Returns the AutoOffsetResetStrategy from the given string.
+ */
+ public static ShareGroupAutoOffsetResetStrategy fromString(String
offsetStrategy) {
+ if (offsetStrategy == null) {
+ throw new IllegalArgumentException("Auto offset reset strategy is
null");
+ }
+
+ if (StrategyType.BY_DURATION.toString().equals(offsetStrategy)) {
+ throw new IllegalArgumentException("<:duration> part is missing in
by_duration auto offset reset strategy.");
+ }
+
+ if
(Arrays.asList(Utils.enumOptions(StrategyType.class)).contains(offsetStrategy))
{
+ StrategyType type =
StrategyType.valueOf(offsetStrategy.toUpperCase(Locale.ROOT));
+ switch (type) {
+ case EARLIEST:
+ return EARLIEST;
+ case LATEST:
+ return LATEST;
+ default:
+ throw new IllegalArgumentException("Unknown auto offset
reset strategy: " + offsetStrategy);
+ }
+ }
+
+ if (offsetStrategy.startsWith(StrategyType.BY_DURATION + ":")) {
+ String isoDuration =
offsetStrategy.substring(StrategyType.BY_DURATION.toString().length() + 1);
+ try {
+ Duration duration = Duration.parse(isoDuration);
+ if (duration.isNegative()) {
+ throw new IllegalArgumentException("Negative duration is
not supported in by_duration offset reset strategy.");
+ }
+ return new ShareGroupAutoOffsetResetStrategy(duration);
+ } catch (Exception e) {
+ throw new IllegalArgumentException("Unable to parse duration
string in by_duration offset reset strategy.", e);
+ }
+ }
+
+ throw new IllegalArgumentException("Unknown auto offset reset
strategy: " + offsetStrategy);
+ }
+
+ /**
+ * Returns the offset reset strategy type.
+ */
+ public StrategyType type() {
+ return type;
+ }
+
+ /**
+ * Returns the name of the offset reset strategy.
+ */
+ public String name() {
+ return type.toString();
+ }
+
+ /**
+ * Return the timestamp to be used for the ListOffsetsRequest.
+ * @return the timestamp for the OffsetResetStrategy,
+ * if the strategy is EARLIEST or LATEST or duration is provided
+ * else return Optional.empty()
+ */
+ public Optional<Long> timestamp() {
+ if (type == StrategyType.EARLIEST)
+ return Optional.of(ListOffsetsRequest.EARLIEST_TIMESTAMP);
+ else if (type == StrategyType.LATEST)
+ return Optional.of(ListOffsetsRequest.LATEST_TIMESTAMP);
+ else if (type == StrategyType.BY_DURATION && duration.isPresent()) {
+ Instant now = Instant.now();
+ return Optional.of(now.minus(duration.get()).toEpochMilli());
+ } else
+ return Optional.empty();
Review Comment:
Would it be better to use `pattern type` switch statement, java 17, so any
change in enum and not in this method can give compile time exception?
##########
core/src/main/java/kafka/server/share/SharePartition.java:
##########
@@ -2125,16 +2128,25 @@ private long startOffsetDuringInitialization(long
partitionDataStartOffset) thro
if (partitionDataStartOffset !=
PartitionFactory.UNINITIALIZED_START_OFFSET) {
return partitionDataStartOffset;
}
- GroupConfig.ShareGroupAutoOffsetReset offsetResetStrategy;
+ ShareGroupAutoOffsetResetStrategy offsetResetStrategy;
if (groupConfigManager.groupConfig(groupId).isPresent()) {
offsetResetStrategy =
groupConfigManager.groupConfig(groupId).get().shareAutoOffsetReset();
} else {
offsetResetStrategy = GroupConfig.defaultShareAutoOffsetReset();
}
- if (offsetResetStrategy ==
GroupConfig.ShareGroupAutoOffsetReset.EARLIEST)
+ if (offsetResetStrategy.timestamp().isEmpty()) {
+ throw new Exception("The timestamp is not available for the share
partition: " + topicIdPartition);
+ }
+ final long timestamp = offsetResetStrategy.timestamp().get();
+
+ if (timestamp == ListOffsetsRequest.LATEST_TIMESTAMP) {
Review Comment:
It's bit odd to fetch the `timestamp` to know which strategy to use however,
startegy itself is used to figure out the `timestamp`. Why no to use
`offsetResetStrategy` itself?
--
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]