-
Notifications
You must be signed in to change notification settings - Fork 190
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Added Kafka config to support acknowledgments and MSK arn (#2976)
* Added Kafka config to support acknowledgments and MSK arn Signed-off-by: Krishna Kondaka <[email protected]> * Modified to use data-prepper-core in testImplementation Signed-off-by: Krishna Kondaka <[email protected]> * Addressed review comments Signed-off-by: Krishna Kondaka <[email protected]> * Addressed failing test Signed-off-by: Krishna Kondaka <[email protected]> --------- Signed-off-by: Krishna Kondaka <[email protected]> Co-authored-by: Krishna Kondaka <[email protected]> Signed-off-by: rajeshLovesToCode <[email protected]>
- Loading branch information
1 parent
10eee70
commit 657e89c
Showing
11 changed files
with
468 additions
and
46 deletions.
There are no files selected for viewing
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
19 changes: 19 additions & 0 deletions
19
...ugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/AwsConfig.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,19 @@ | ||
/* | ||
* Copyright OpenSearch Contributors | ||
* SPDX-License-Identifier: Apache-2.0 | ||
*/ | ||
|
||
package org.opensearch.dataprepper.plugins.kafka.configuration; | ||
|
||
import com.fasterxml.jackson.annotation.JsonProperty; | ||
import jakarta.validation.constraints.Size; | ||
|
||
public class AwsConfig { | ||
@JsonProperty("msk_arn") | ||
@Size(min = 20, max = 2048, message = "mskArn length should be between 20 and 2048 characters") | ||
private String awsMskArn; | ||
|
||
public String getAwsMskArn() { | ||
return awsMskArn; | ||
} | ||
} |
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
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
79 changes: 79 additions & 0 deletions
79
...n/java/org/opensearch/dataprepper/plugins/kafka/consumer/TopicPartitionCommitTracker.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,79 @@ | ||
/* | ||
* Copyright OpenSearch Contributors | ||
* SPDX-License-Identifier: Apache-2.0 | ||
*/ | ||
|
||
package org.opensearch.dataprepper.plugins.kafka.consumer; | ||
|
||
import java.util.HashMap; | ||
import java.util.Map; | ||
import java.util.Objects; | ||
|
||
import org.apache.commons.lang3.Range; | ||
import org.apache.kafka.clients.consumer.OffsetAndMetadata; | ||
import org.apache.kafka.common.TopicPartition; | ||
|
||
public class TopicPartitionCommitTracker { | ||
private long committedOffset; | ||
private final TopicPartition topicPartition; | ||
private final Map<Long, Range<Long>> offsetMaxMap; | ||
private final Map<Long, Range<Long>> offsetMinMap; | ||
|
||
public TopicPartitionCommitTracker(final TopicPartition topicPartition, Long committedOffset) { | ||
this.topicPartition = topicPartition; | ||
this.committedOffset = Objects.nonNull(committedOffset) ? committedOffset : -1L; | ||
this.offsetMaxMap = new HashMap<>(); | ||
this.offsetMinMap = new HashMap<>(); | ||
this.offsetMaxMap.put(this.committedOffset, Range.between(this.committedOffset, this.committedOffset)); | ||
} | ||
|
||
public TopicPartitionCommitTracker(final String topic, final int partition, Long committedOffset) { | ||
this(new TopicPartition(topic, partition), committedOffset); | ||
} | ||
|
||
public OffsetAndMetadata addCompletedOffsets(final Range<Long> offsetRange) { | ||
Long min = offsetRange.getMinimum(); | ||
Long max = offsetRange.getMaximum(); | ||
boolean merged = false; | ||
if (offsetMaxMap.containsKey(min - 1)) { | ||
Range<Long> entry = offsetMaxMap.get(min - 1); | ||
offsetMaxMap.remove(min - 1); | ||
offsetMinMap.remove(entry.getMinimum()); | ||
min = entry.getMinimum(); | ||
Range<Long> newEntry = Range.between(min, max); | ||
offsetMaxMap.put(max, newEntry); | ||
offsetMinMap.put(min, newEntry); | ||
merged = true; | ||
} | ||
if (offsetMinMap.containsKey(max + 1)) { | ||
Range<Long> entry = offsetMinMap.get(max + 1); | ||
offsetMinMap.remove(max + 1); | ||
if (merged) { | ||
offsetMinMap.remove(min); | ||
offsetMaxMap.remove(max); | ||
} | ||
max = entry.getMaximum(); | ||
offsetMaxMap.remove(max); | ||
Range<Long> newEntry = Range.between(min, max); | ||
offsetMaxMap.put(max, newEntry); | ||
offsetMinMap.put(min, newEntry); | ||
merged = true; | ||
} | ||
if (!merged) { | ||
offsetMaxMap.put(max, offsetRange); | ||
offsetMinMap.put(min, offsetRange); | ||
return null; | ||
} | ||
if (offsetMinMap.containsKey(committedOffset)) { | ||
Long maxValue = offsetMinMap.get(committedOffset).getMaximum(); | ||
if (maxValue != committedOffset) { | ||
offsetMinMap.remove(committedOffset); | ||
committedOffset = maxValue; | ||
offsetMaxMap.put(committedOffset, Range.between(committedOffset, committedOffset)); | ||
return new OffsetAndMetadata(committedOffset + 1); | ||
} | ||
} | ||
return null; | ||
} | ||
|
||
} |
Oops, something went wrong.