This repository has been archived by the owner on Feb 12, 2022. It is now read-only.
-
Notifications
You must be signed in to change notification settings - Fork 23
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[Maulik] batch commits from offset commit worker
Signed-off-by: mauliksoneji <[email protected]>
- Loading branch information
mauliksoneji
committed
Apr 24, 2020
1 parent
304883e
commit 19c0b8a
Showing
9 changed files
with
253 additions
and
172 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
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
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
101 changes: 68 additions & 33 deletions
101
src/main/java/com/gojek/beast/worker/OffsetCommitWorker.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 |
---|---|---|
@@ -1,84 +1,119 @@ | ||
package com.gojek.beast.worker; | ||
|
||
import com.gojek.beast.Clock; | ||
import com.gojek.beast.commiter.KafkaCommitter; | ||
import com.gojek.beast.commiter.OffsetState; | ||
import com.gojek.beast.config.QueueConfig; | ||
import com.gojek.beast.models.FailureStatus; | ||
import com.gojek.beast.models.Records; | ||
import com.gojek.beast.models.Status; | ||
import com.gojek.beast.models.OffsetMetadata; | ||
import com.gojek.beast.models.SuccessStatus; | ||
import com.gojek.beast.stats.Stats; | ||
import com.gojek.beast.models.Records; | ||
import com.gojek.beast.models.Status; | ||
import lombok.Setter; | ||
import lombok.extern.slf4j.Slf4j; | ||
import org.apache.kafka.clients.consumer.OffsetAndMetadata; | ||
import org.apache.kafka.common.TopicPartition; | ||
|
||
import java.time.Instant; | ||
import java.util.HashMap; | ||
import java.util.Map; | ||
import java.util.Set; | ||
import java.util.concurrent.BlockingQueue; | ||
|
||
@Slf4j | ||
public class OffsetCommitWorker extends Worker { | ||
private static final int DEFAULT_SLEEP_MS = 100; | ||
private final Stats statsClient = Stats.client(); | ||
private BlockingQueue<Records> commitQueue; | ||
private Set<Map<TopicPartition, OffsetAndMetadata>> partitionOffsetAck; | ||
private KafkaCommitter kafkaCommitter; | ||
private final BlockingQueue<Records> commitQueue; | ||
private final QueueConfig queueConfig; | ||
private final KafkaCommitter kafkaCommitter; | ||
@Setter | ||
private long defaultSleepMs; | ||
|
||
|
||
private boolean stopped; | ||
private OffsetState offsetState; | ||
private Clock clock; | ||
|
||
public OffsetCommitWorker(String name, Set<Map<TopicPartition, OffsetAndMetadata>> partitionOffsetAck, KafkaCommitter kafkaCommitter, OffsetState offsetState, BlockingQueue<Records> commitQueue, WorkerState workerState) { | ||
public OffsetCommitWorker(String name, QueueConfig queueConfig, KafkaCommitter kafkaCommitter, OffsetState offsetState, BlockingQueue<Records> commitQueue, WorkerState workerState, Clock clock) { | ||
super(name, workerState); | ||
this.clock = clock; | ||
this.queueConfig = queueConfig; | ||
this.commitQueue = commitQueue; | ||
this.partitionOffsetAck = partitionOffsetAck; | ||
this.kafkaCommitter = kafkaCommitter; | ||
this.defaultSleepMs = DEFAULT_SLEEP_MS; | ||
this.offsetState = offsetState; | ||
this.stopped = false; | ||
} | ||
|
||
@Override | ||
public void stop(String reason) { | ||
log.info("Closing committer: {}", reason); | ||
this.stopped = true; | ||
kafkaCommitter.wakeup(reason); | ||
} | ||
|
||
@Override | ||
public Status job() { | ||
offsetState.startTimer(); | ||
try { | ||
Instant start = Instant.now(); | ||
Records commitOffset = commitQueue.peek(); | ||
if (commitOffset == null) { | ||
return new SuccessStatus(); | ||
} | ||
Map<TopicPartition, OffsetAndMetadata> currentOffset = commitOffset.getPartitionsCommitOffset(); | ||
if (partitionOffsetAck.contains(currentOffset)) { | ||
commit(); | ||
} else { | ||
if (offsetState.shouldCloseConsumer(currentOffset)) { | ||
statsClient.increment("committer.ack.timeout"); | ||
return new FailureStatus(new RuntimeException("Acknowledgement Timeout exceeded: " + offsetState.getAcknowledgeTimeoutMs())); | ||
Instant startTime = Instant.now(); | ||
long start = clock.currentEpochMillis(); | ||
Map<TopicPartition, OffsetAndMetadata> partitionsCommitOffset = new HashMap<>(); | ||
|
||
int offsetClubbedBatches = 0; | ||
while (true) { | ||
Records commitOffset = commitQueue.poll(queueConfig.getTimeout(), queueConfig.getTimeoutUnit()); | ||
if (stopped || clock.currentEpochMillis() - start > offsetState.getOffsetBatchDuration()) { | ||
break; | ||
} | ||
log.debug("waiting for {} acknowledgement for offset {}", defaultSleepMs, currentOffset); | ||
sleep(defaultSleepMs); | ||
statsClient.gauge("committer.queue.wait.ms", defaultSleepMs); | ||
|
||
if (commitOffset == null) { | ||
continue; | ||
} | ||
|
||
Map<TopicPartition, OffsetAndMetadata> currentOffset = commitOffset.getPartitionsCommitOffset(); | ||
Instant commitQueuePollStartTime = Instant.now(); | ||
while (true) { | ||
if (offsetState.removeFromOffsetAck(currentOffset)) { | ||
|
||
currentOffset.keySet().forEach(topicPartition -> { | ||
OffsetAndMetadata offsetAndMetadata = currentOffset.get(topicPartition); | ||
OffsetMetadata previousOffset = (OffsetMetadata) partitionsCommitOffset.getOrDefault(topicPartition, new OffsetMetadata(Integer.MIN_VALUE)); | ||
OffsetMetadata newOffset = new OffsetMetadata(offsetAndMetadata.offset()); | ||
if (previousOffset.compareTo(newOffset) < 0) { | ||
partitionsCommitOffset.put(topicPartition, newOffset); | ||
} | ||
}); | ||
offsetState.resetOffset(); | ||
offsetClubbedBatches++; | ||
break; | ||
} else { | ||
if (offsetState.shouldCloseConsumer(partitionsCommitOffset)) { | ||
statsClient.increment("committer.ack.timeout"); | ||
return new FailureStatus(new RuntimeException("Acknowledgement Timeout exceeded: " + offsetState.getAcknowledgeTimeoutMs())); | ||
} | ||
|
||
log.debug("waiting for {} acknowledgement for offset {}, {}, {}", defaultSleepMs, currentOffset, offsetState.partitionOffsetAckSize(), currentOffset); | ||
sleep(defaultSleepMs); | ||
} | ||
} | ||
statsClient.timeIt("committer.queue.wait.ms", commitQueuePollStartTime); | ||
} | ||
statsClient.timeIt("committer.processing.time", start); | ||
commit(partitionsCommitOffset); | ||
|
||
statsClient.gauge("committer.clubbed.offsets", offsetClubbedBatches); | ||
statsClient.timeIt("committer.processing.time", startTime); | ||
} catch (InterruptedException | RuntimeException e) { | ||
log.info("Received {} exception: {}, resetting committer", e.getClass(), e.getMessage()); | ||
e.printStackTrace(); | ||
return new FailureStatus(new RuntimeException("Exception in offset committer: " + e.getMessage())); | ||
} | ||
return new SuccessStatus(); | ||
} | ||
|
||
private void commit() { | ||
Map<TopicPartition, OffsetAndMetadata> partitionsCommitOffset = commitQueue.remove().getPartitionsCommitOffset(); | ||
kafkaCommitter.commitSync(partitionsCommitOffset); | ||
partitionOffsetAck.remove(partitionsCommitOffset); | ||
Records nextOffset = commitQueue.peek(); | ||
if (nextOffset != null) offsetState.resetOffset(nextOffset.getPartitionsCommitOffset()); | ||
log.info("commit partition {} size {}", partitionsCommitOffset.toString(), partitionsCommitOffset.size()); | ||
private void commit(Map<TopicPartition, OffsetAndMetadata> partitionsCommitOffset) { | ||
if (partitionsCommitOffset.size() != 0) { | ||
kafkaCommitter.commitSync(partitionsCommitOffset); | ||
} | ||
log.info("committed offsets partition {} size {}", partitionsCommitOffset.toString(), partitionsCommitOffset.size()); | ||
} | ||
} |
Oops, something went wrong.