|
| 1 | +/* |
| 2 | + * Copyright 2018 Confluent Inc. |
| 3 | + * |
| 4 | + * Licensed under the Confluent Community License (the "License"); you may not use |
| 5 | + * this file except in compliance with the License. You may obtain a copy of the |
| 6 | + * License at |
| 7 | + * |
| 8 | + * http://www.confluent.io/confluent-community-license |
| 9 | + * |
| 10 | + * Unless required by applicable law or agreed to in writing, software |
| 11 | + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT |
| 12 | + * WARRANTIES OF ANY KIND, either express or implied. See the License for the |
| 13 | + * specific language governing permissions and limitations under the License. |
| 14 | + */ |
| 15 | + |
| 16 | +package io.confluent.connect.elasticsearch; |
| 17 | + |
| 18 | +import java.util.Collection; |
| 19 | +import java.util.HashMap; |
| 20 | +import java.util.Iterator; |
| 21 | +import java.util.LinkedHashMap; |
| 22 | +import java.util.Map; |
| 23 | +import java.util.concurrent.atomic.AtomicLong; |
| 24 | + |
| 25 | +import org.apache.kafka.clients.consumer.OffsetAndMetadata; |
| 26 | +import org.apache.kafka.common.TopicPartition; |
| 27 | +import org.apache.kafka.connect.errors.ConnectException; |
| 28 | +import org.apache.kafka.connect.sink.SinkRecord; |
| 29 | +import org.apache.kafka.connect.sink.SinkTaskContext; |
| 30 | +import org.slf4j.Logger; |
| 31 | +import org.slf4j.LoggerFactory; |
| 32 | + |
| 33 | +import static java.util.stream.Collectors.toMap; |
| 34 | + |
| 35 | +/** |
| 36 | + * It's an asynchronous implementation of <code>OffsetTracker</code> |
| 37 | + * |
| 38 | + * <p>Since ElasticsearchClient can potentially process multiple batches asynchronously for the same |
| 39 | + * partition, if we don't want to wait for all in-flight batches at the end of the put call |
| 40 | + * (or flush/preCommit) we need to keep track of what's the highest offset that is safe to commit. |
| 41 | + * For now, we do that at the individual record level because batching is handled by BulkProcessor, |
| 42 | + * and we don't have control over grouping/ordering. |
| 43 | + */ |
| 44 | +class AsyncOffsetTracker implements OffsetTracker { |
| 45 | + |
| 46 | + private static final Logger log = LoggerFactory.getLogger(AsyncOffsetTracker.class); |
| 47 | + |
| 48 | + private final Map<TopicPartition, Map<Long, OffsetState>> offsetsByPartition = new HashMap<>(); |
| 49 | + private final Map<TopicPartition, Long> maxOffsetByPartition = new HashMap<>(); |
| 50 | + |
| 51 | + private final AtomicLong numEntries = new AtomicLong(); |
| 52 | + private final SinkTaskContext context; |
| 53 | + |
| 54 | + public AsyncOffsetTracker(SinkTaskContext context) { |
| 55 | + this.context = context; |
| 56 | + } |
| 57 | + |
| 58 | + static class AsyncOffsetState implements OffsetState { |
| 59 | + |
| 60 | + private final long offset; |
| 61 | + private volatile boolean processed; |
| 62 | + |
| 63 | + AsyncOffsetState(long offset) { |
| 64 | + this.offset = offset; |
| 65 | + } |
| 66 | + |
| 67 | + @Override |
| 68 | + public void markProcessed() { |
| 69 | + processed = true; |
| 70 | + } |
| 71 | + |
| 72 | + @Override |
| 73 | + public boolean isProcessed() { |
| 74 | + return processed; |
| 75 | + } |
| 76 | + |
| 77 | + @Override |
| 78 | + public long offset() { |
| 79 | + return offset; |
| 80 | + } |
| 81 | + } |
| 82 | + |
| 83 | + /** |
| 84 | + * Partitions are no longer owned, we should release all related resources. |
| 85 | + * @param topicPartitions partitions to close |
| 86 | + */ |
| 87 | + @Override |
| 88 | + public synchronized void closePartitions(Collection<TopicPartition> topicPartitions) { |
| 89 | + topicPartitions.forEach(tp -> { |
| 90 | + Map<Long, OffsetState> offsets = offsetsByPartition.remove(tp); |
| 91 | + if (offsets != null) { |
| 92 | + numEntries.getAndAdd(-offsets.size()); |
| 93 | + } |
| 94 | + maxOffsetByPartition.remove(tp); |
| 95 | + }); |
| 96 | + } |
| 97 | + |
| 98 | + /** |
| 99 | + * This method assumes that new records are added in offset order. |
| 100 | + * Older records can be re-added, and the same Offset object will be return if its |
| 101 | + * offset hasn't been reported yet. |
| 102 | + * @param sinkRecord record to add |
| 103 | + * @return offset state record that can be used to mark the record as processed |
| 104 | + */ |
| 105 | + @Override |
| 106 | + public synchronized OffsetState addPendingRecord( |
| 107 | + SinkRecord sinkRecord |
| 108 | + ) { |
| 109 | + log.trace("Adding pending record"); |
| 110 | + TopicPartition tp = new TopicPartition(sinkRecord.topic(), sinkRecord.kafkaPartition()); |
| 111 | + if (!context.assignment().contains(tp)) { |
| 112 | + String msg = String.format("Found a topic name '%s' that doesn't match assigned partitions." |
| 113 | + + " Connector doesn't support topic mutating SMTs", sinkRecord.topic()); |
| 114 | + throw new ConnectException(msg); |
| 115 | + } |
| 116 | + Long partitionMax = maxOffsetByPartition.get(tp); |
| 117 | + if (partitionMax == null || sinkRecord.kafkaOffset() > partitionMax) { |
| 118 | + numEntries.incrementAndGet(); |
| 119 | + return offsetsByPartition |
| 120 | + // Insertion order needs to be maintained |
| 121 | + .computeIfAbsent(tp, key -> new LinkedHashMap<>()) |
| 122 | + .computeIfAbsent(sinkRecord.kafkaOffset(), AsyncOffsetState::new); |
| 123 | + } else { |
| 124 | + return new AsyncOffsetState(sinkRecord.kafkaOffset()); |
| 125 | + } |
| 126 | + } |
| 127 | + |
| 128 | + /** |
| 129 | + * @return overall number of entries currently in memory. |
| 130 | + */ |
| 131 | + @Override |
| 132 | + public long numOffsetStateEntries() { |
| 133 | + return numEntries.get(); |
| 134 | + } |
| 135 | + |
| 136 | + /** |
| 137 | + * Move offsets to the highest we can. |
| 138 | + */ |
| 139 | + @Override |
| 140 | + public synchronized void updateOffsets() { |
| 141 | + log.trace("Updating offsets"); |
| 142 | + offsetsByPartition.forEach(((topicPartition, offsets) -> { |
| 143 | + Long max = maxOffsetByPartition.get(topicPartition); |
| 144 | + boolean newMaxFound = false; |
| 145 | + Iterator<OffsetState> iterator = offsets.values().iterator(); |
| 146 | + while (iterator.hasNext()) { |
| 147 | + OffsetState offsetState = iterator.next(); |
| 148 | + if (offsetState.isProcessed()) { |
| 149 | + iterator.remove(); |
| 150 | + numEntries.decrementAndGet(); |
| 151 | + if (max == null || offsetState.offset() > max) { |
| 152 | + max = offsetState.offset(); |
| 153 | + newMaxFound = true; |
| 154 | + } |
| 155 | + } else { |
| 156 | + break; |
| 157 | + } |
| 158 | + } |
| 159 | + if (newMaxFound) { |
| 160 | + maxOffsetByPartition.put(topicPartition, max); |
| 161 | + } |
| 162 | + })); |
| 163 | + log.trace("Updated offsets, num entries: {}", numEntries); |
| 164 | + } |
| 165 | + |
| 166 | + /** |
| 167 | + * @param currentOffsets current offsets from a task |
| 168 | + * @return offsets to commit |
| 169 | + */ |
| 170 | + @Override |
| 171 | + public synchronized Map<TopicPartition, OffsetAndMetadata> offsets( |
| 172 | + Map<TopicPartition, OffsetAndMetadata> currentOffsets |
| 173 | + ) { |
| 174 | + return maxOffsetByPartition.entrySet().stream() |
| 175 | + .collect(toMap( |
| 176 | + Map.Entry::getKey, |
| 177 | + // The offsets you commit are the offsets of the messages you want to read next |
| 178 | + // (not the offsets of the messages you did read last) |
| 179 | + e -> new OffsetAndMetadata(e.getValue() + 1))); |
| 180 | + } |
| 181 | +} |
0 commit comments