diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
index fb4c05062bb..b0b39274075 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
@@ -171,9 +171,9 @@ public class ConsumerConfig extends AbstractConfig {
public static final String AUTO_OFFSET_RESET_CONFIG = "auto.offset.reset";
public static final String AUTO_OFFSET_RESET_DOC = "What to do when there is no initial offset in Kafka or if the current offset does not exist any more on the server " +
"(e.g. because that data has been deleted): " +
- "
- earliest: automatically reset the offset to the earliest offset" +
+ "
- earliest: automatically reset the offset to the earliest offset
" +
"- latest: automatically reset the offset to the latest offset
" +
- "- by_duration:: automatically reset the offset to a configured from the current timestamp. must be specified in ISO8601 format (PnDTnHnMn.nS). " +
+ "
- by_duration:<duration>: automatically reset the offset to a configured <duration> from the current timestamp. <duration> must be specified in ISO8601 format (PnDTnHnMn.nS). " +
"Negative duration is not allowed.
" +
"- none: throw exception to the consumer if no previous offset is found for the consumer's group
" +
"- anything else: throw exception to the consumer.
" +
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AutoOffsetResetStrategy.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AutoOffsetResetStrategy.java
index a692bc45ea7..e904ca3d5d6 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AutoOffsetResetStrategy.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AutoOffsetResetStrategy.java
@@ -27,6 +27,7 @@ import java.util.Arrays;
import java.util.Locale;
import java.util.Objects;
import java.util.Optional;
+import java.util.stream.Collectors;
public class AutoOffsetResetStrategy {
public enum StrategyType {
@@ -165,5 +166,17 @@ public class AutoOffsetResetStrategy {
name + ". The value must be either 'earliest', 'latest', 'none' or of the format 'by_duration:'.");
}
}
+
+ @Override
+ public String toString() {
+ String values = Arrays.stream(StrategyType.values())
+ .map(strategyType -> {
+ if (strategyType == StrategyType.BY_DURATION) {
+ return "by_duration:PnDTnHnMn.nS";
+ }
+ return strategyType.toString();
+ }).collect(Collectors.joining(", "));
+ return "[" + values + "]";
+ }
}
}