Github user StephanEwen commented on a diff in the pull request:
https://github.com/apache/flink/pull/1341#discussion_r45462057
--- Diff:
flink-streaming-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer.java
---
@@ -567,6 +588,76 @@ public void notifyCheckpointComplete(long
checkpointId) throws Exception {
}
return partitionsToSub;
}
+
+ /**
+ * Thread to periodically commit the current read offset into Zookeeper.
+ */
+ private static class PeriodicOffsetCommitter extends Thread {
+ private long commitInterval;
+ private volatile boolean running = true;
+ private volatile Thread self;
+ private FlinkKafkaConsumer consumer;
+ private final Object stateUpdateLock = new Object();
+
+ public PeriodicOffsetCommitter(long commitInterval,
FlinkKafkaConsumer consumer) {
+ this.commitInterval = commitInterval;
+ this.consumer = consumer;
+ this.self = this;
+ }
+
+ @Override
+ public void run() {
+ try {
+ while (running) {
+ try {
+ Thread.sleep(commitInterval);
+
+ // ------------ commit
current offsets ----------------
+
+ // create copy of current
offsets
+ long[] currentOffsets;
+ synchronized (stateUpdateLock) {
+ currentOffsets =
Arrays.copyOf(consumer.lastOffsets, consumer.lastOffsets.length);
+ }
+
+ Map<TopicPartition, Long>
offsetsToCommit = new HashMap<>();
+ for (TopicPartition tp :
(List<TopicPartition>)consumer.subscribedPartitions) {
+ int partition =
tp.partition();
+ long offset =
currentOffsets[partition];
+ long lastCommitted =
consumer.commitedOffsets[partition];
+
+ if (offset !=
OFFSET_NOT_SET) {
+ if (offset >
lastCommitted) {
+
offsetsToCommit.put(tp, offset);
+
LOG.debug("Committing offset {} for partition {}", offset, partition);
+ } else {
+
LOG.debug("Ignoring offset {} for partition {} because it is already
committed", offset, partition);
+ }
+ }
+ }
+
+
consumer.offsetHandler.commit(offsetsToCommit);
+ } catch (InterruptedException e) {
+ if(running) {
+ // throw unexpected
interruption
+ throw e;
+ }
+ // looks like the thread is
being closed. Leave loop
+ break;
+ }
+ }
+ } catch (Throwable t) {
+ LOG.warn("Periodic checkpoint committer is
stopping the fetcher because of an error", t);
+ consumer.fetcher.stopWithError(t);
+ }
+ }
+
+ public void close() {
+ this.running = false;
+ this.self.interrupt();
--- End diff --
Same as if you do 'this.interrupt()'. Seems there is a confusion about
`this` vs `Thread.currentThread()`.
`this` is still the object reference, the thread represented by that
reference, not the calling thread.
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---