-
Notifications
You must be signed in to change notification settings - Fork 14.1k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
KAFKA-18015: Add support for duration based offset reset strategy to …
…Kafka Streams (#17973) Part of KIP-1106. Adds the public APIs to Kafka Streams, to support the the newly added "by_duration" reset policy, plus adds the missing "none" reset policy. Deprecates the enum `Topology.AutoOffsetReset` and all related methods, and replaced them with new overload using the new `AutoOffsetReset` class. Co-authored-by: Matthias J. Sax <[email protected]> Reviewers: Matthias J. Sax <[email protected]>, Bill Bejeck <[email protected]>, Manikumar Reddy <[email protected]>
- Loading branch information
1 parent
156d551
commit d83f09d
Showing
13 changed files
with
615 additions
and
39 deletions.
There are no files selected for viewing
103 changes: 103 additions & 0 deletions
103
streams/src/main/java/org/apache/kafka/streams/AutoOffsetReset.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,103 @@ | ||
/* | ||
* 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.clients.consumer.internals.AutoOffsetResetStrategy.StrategyType; | ||
import org.apache.kafka.streams.kstream.KStream; | ||
import org.apache.kafka.streams.kstream.KTable; | ||
|
||
import java.time.Duration; | ||
import java.util.Optional; | ||
|
||
/** | ||
* Sets the {@code auto.offset.reset} configuration when | ||
* {@link Topology#addSource(AutoOffsetReset, String, String...) adding a source processor} | ||
* or when creating {@link KStream} or {@link KTable} via {@link StreamsBuilder}. | ||
*/ | ||
public class AutoOffsetReset { | ||
protected final StrategyType offsetResetStrategy; | ||
protected final Optional<Duration> duration; | ||
|
||
private AutoOffsetReset(final StrategyType offsetResetStrategy, final Optional<Duration> duration) { | ||
this.offsetResetStrategy = offsetResetStrategy; | ||
this.duration = duration; | ||
} | ||
|
||
protected AutoOffsetReset(final AutoOffsetReset autoOffsetReset) { | ||
this(autoOffsetReset.offsetResetStrategy, autoOffsetReset.duration); | ||
} | ||
|
||
/** | ||
* Creates an {@code AutoOffsetReset} instance representing "none". | ||
* | ||
* @return An {@link AutoOffsetReset} instance for no reset. | ||
*/ | ||
public static AutoOffsetReset none() { | ||
return new AutoOffsetReset(StrategyType.NONE, Optional.empty()); | ||
} | ||
|
||
/** | ||
* Creates an {@code AutoOffsetReset} instance representing "earliest". | ||
* | ||
* @return An {@link AutoOffsetReset} instance for the "earliest" offset. | ||
*/ | ||
public static AutoOffsetReset earliest() { | ||
return new AutoOffsetReset(StrategyType.EARLIEST, Optional.empty()); | ||
} | ||
|
||
/** | ||
* Creates an {@code AutoOffsetReset} instance representing "latest". | ||
* | ||
* @return An {@code AutoOffsetReset} instance for the "latest" offset. | ||
*/ | ||
public static AutoOffsetReset latest() { | ||
return new AutoOffsetReset(StrategyType.LATEST, Optional.empty()); | ||
} | ||
|
||
/** | ||
* Creates an {@code AutoOffsetReset} instance for the specified reset duration. | ||
* | ||
* @param duration The duration to use for the offset reset; must be non-negative. | ||
* @return An {@code AutoOffsetReset} instance with the specified duration. | ||
* @throws IllegalArgumentException If the duration is negative. | ||
*/ | ||
public static AutoOffsetReset byDuration(final Duration duration) { | ||
if (duration.isNegative()) { | ||
throw new IllegalArgumentException("Duration cannot be negative"); | ||
} | ||
return new AutoOffsetReset(StrategyType.BY_DURATION, Optional.of(duration)); | ||
} | ||
|
||
@Override | ||
public boolean equals(final Object o) { | ||
if (this == o) { | ||
return true; | ||
} | ||
if (o == null || getClass() != o.getClass()) { | ||
return false; | ||
} | ||
final AutoOffsetReset that = (AutoOffsetReset) o; | ||
return offsetResetStrategy == that.offsetResetStrategy && duration.equals(that.duration); | ||
} | ||
|
||
@Override | ||
public int hashCode() { | ||
int result = offsetResetStrategy.hashCode(); | ||
result = 31 * result + duration.hashCode(); | ||
return result; | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
220 changes: 217 additions & 3 deletions
220
streams/src/main/java/org/apache/kafka/streams/Topology.java
Large diffs are not rendered by default.
Oops, something went wrong.
37 changes: 37 additions & 0 deletions
37
streams/src/main/java/org/apache/kafka/streams/internals/AutoOffsetResetInternal.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,37 @@ | ||
/* | ||
* 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.internals; | ||
|
||
import org.apache.kafka.clients.consumer.internals.AutoOffsetResetStrategy.StrategyType; | ||
import org.apache.kafka.streams.AutoOffsetReset; | ||
|
||
import java.time.Duration; | ||
import java.util.Optional; | ||
|
||
public class AutoOffsetResetInternal extends AutoOffsetReset { | ||
|
||
public AutoOffsetResetInternal(final AutoOffsetReset autoOffsetReset) { | ||
super(autoOffsetReset); | ||
} | ||
|
||
public StrategyType offsetResetStrategy() { | ||
return offsetResetStrategy; | ||
} | ||
public Optional<Duration> duration() { | ||
return duration; | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.