forked from confluentinc/parallel-consumer
-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
feature: Choose between Consumer commit or Producer transactional com…
…mits - Choose either Consumer sync or async commits - Fixes confluentinc#25 confluentinc#25: -- Sometimes a a transaction error occurs - Cannot call send in state COMMITTING_TRANSACTION confluentinc#25 - ReentrantReadWrite lock protects non-thread safe transactional producer from incorrect multithreaded use - Wider lock to prevent transaction's containing produced messages that they shouldn't - Implement non transactional synchronous commit sync properly - Select tests adapted to non transactional as well - Must start tx in MockProducer as well - Adds supervision to poller - Fixes a performance issue with the async committer not being woken up - Enhances tests to run under multiple commit modes - Fixes example app tests - incorrectly testing wrong thing and MockProducer not configured to auto complete - Make committer thread revoke partitions and commit - Have onPartitionsRevoked be responsible for committing on close, instead of an explicit call to commit by controller - Make sure Broker Poller now drains properly, committing any waiting work - Add missing revoke flow to MockConsumer wrapper - Add missing latch timeout check
- Loading branch information
1 parent
0b96d5e
commit ac393f2
Showing
41 changed files
with
2,036 additions
and
533 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
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -29,4 +29,7 @@ hs_err_pid* | |
*.iml | ||
target/ | ||
.DS_Store | ||
*.versionsBackup | ||
*.versionsBackup | ||
|
||
# JENV | ||
.java-version |
Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.
Oops, something went wrong.
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
59 changes: 59 additions & 0 deletions
59
...el-consumer-core/src/main/java/io/confluent/parallelconsumer/AbstractOffsetCommitter.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,59 @@ | ||
package io.confluent.parallelconsumer; | ||
|
||
import lombok.RequiredArgsConstructor; | ||
import lombok.extern.slf4j.Slf4j; | ||
import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; | ||
import org.apache.kafka.clients.consumer.OffsetAndMetadata; | ||
import org.apache.kafka.common.TopicPartition; | ||
|
||
import java.util.Map; | ||
|
||
@Slf4j | ||
@RequiredArgsConstructor | ||
public abstract class AbstractOffsetCommitter<K, V> implements OffsetCommitter { | ||
|
||
protected final ConsumerManager<K, V> consumerMgr; | ||
protected final WorkManager<K, V> wm; | ||
|
||
/** | ||
* Get offsets from {@link WorkManager} that are ready to commit | ||
*/ | ||
@Override | ||
public void retrieveOffsetsAndCommit() { | ||
log.debug("Commit starting - find completed work to commit offsets"); | ||
// todo shouldn't be removed until commit succeeds (there's no harm in committing the same offset twice) | ||
preAcquireWork(); | ||
try { | ||
Map<TopicPartition, OffsetAndMetadata> offsetsToSend = wm.findCompletedEligibleOffsetsAndRemove(); | ||
if (offsetsToSend.isEmpty()) { | ||
log.trace("No offsets ready"); | ||
} else { | ||
log.debug("Will commit offsets for {} partition(s): {}", offsetsToSend.size(), offsetsToSend); | ||
ConsumerGroupMetadata groupMetadata = consumerMgr.groupMetadata(); | ||
|
||
log.debug("Begin commit"); | ||
commitOffsets(offsetsToSend, groupMetadata); | ||
|
||
log.debug("On commit success"); | ||
onOffsetCommitSuccess(offsetsToSend); | ||
} | ||
} finally { | ||
postCommit(); | ||
} | ||
} | ||
|
||
protected void postCommit() { | ||
// default noop | ||
} | ||
|
||
protected void preAcquireWork() { | ||
// default noop | ||
} | ||
|
||
private void onOffsetCommitSuccess(final Map<TopicPartition, OffsetAndMetadata> offsetsToSend) { | ||
wm.onOffsetCommitSuccess(offsetsToSend); | ||
} | ||
|
||
protected abstract void commitOffsets(final Map<TopicPartition, OffsetAndMetadata> offsetsToSend, final ConsumerGroupMetadata groupMetadata); | ||
|
||
} |
Oops, something went wrong.