satishd commented on a change in pull request #11058:
URL: https://github.com/apache/kafka/pull/11058#discussion_r719960058
##########
File path:
storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java
##########
@@ -82,44 +87,148 @@
// User topic partitions that this broker is a leader/follower for.
private Set<TopicIdPartition> assignedTopicPartitions =
Collections.emptySet();
- // Map of remote log metadata topic partition to consumed offsets.
+ // Map of remote log metadata topic partition to consumed offsets.
Received consumer records
+ // may or may not have been processed based on the assigned topic
partitions.
private final Map<Integer, Long> partitionToConsumedOffsets = new
ConcurrentHashMap<>();
+ // Map of remote log metadata topic partition to processed offsets.
Received consumer record is
+ // processed as the remote log metadata record's topic partition exists in
assigned topic partitions.
+ private final Map<Integer, Long> partitionToProcessedOffsets = new
ConcurrentHashMap<>();
+
+ // Map of remote log metadata topic partition to processed offsets that
were synced in committedOffsetsFile.
+ private Map<Integer, Long> lastSyncedPartitionToConsumedOffsets =
Collections.emptyMap();
+
+ private final long committedOffsetSyncIntervalMs;
+ private CommittedOffsetsFile committedOffsetsFile;
+ private long lastSyncedTimeMs;
+
public ConsumerTask(KafkaConsumer<byte[], byte[]> consumer,
RemotePartitionMetadataEventHandler
remotePartitionMetadataEventHandler,
- RemoteLogMetadataTopicPartitioner topicPartitioner) {
- Objects.requireNonNull(consumer);
- Objects.requireNonNull(remotePartitionMetadataEventHandler);
- Objects.requireNonNull(topicPartitioner);
-
- this.consumer = consumer;
- this.remotePartitionMetadataEventHandler =
remotePartitionMetadataEventHandler;
- this.topicPartitioner = topicPartitioner;
+ RemoteLogMetadataTopicPartitioner topicPartitioner,
+ Path committedOffsetsPath,
+ Time time,
+ long committedOffsetSyncIntervalMs) {
+ this.consumer = Objects.requireNonNull(consumer);
+ this.remotePartitionMetadataEventHandler =
Objects.requireNonNull(remotePartitionMetadataEventHandler);
+ this.topicPartitioner = Objects.requireNonNull(topicPartitioner);
+ this.time = Objects.requireNonNull(time);
+ this.committedOffsetSyncIntervalMs = committedOffsetSyncIntervalMs;
+
+ initializeConsumerAssignment(committedOffsetsPath);
+ }
+
+ private void initializeConsumerAssignment(Path committedOffsetsPath) {
+ try {
+ committedOffsetsFile = new
CommittedOffsetsFile(committedOffsetsPath.toFile());
+ } catch (IOException e) {
+ throw new KafkaException(e);
+ }
+
+ Map<Integer, Long> committedOffsets = Collections.emptyMap();
+ try {
+ // Load committed offset and assign them in the consumer.
+ committedOffsets = committedOffsetsFile.readEntries();
+ } catch (IOException e) {
+ // Ignore the error and consumer consumes from the earliest offset.
+ log.error("Encountered error while building committed offsets from
the file", e);
+ }
+
+ if (!committedOffsets.isEmpty()) {
+ // Assign topic partitions from the earlier committed offsets file.
+ Set<Integer> earlierAssignedPartitions = committedOffsets.keySet();
+ assignedMetaPartitions =
Collections.unmodifiableSet(earlierAssignedPartitions);
+ Set<TopicPartition> metadataTopicPartitions =
earlierAssignedPartitions.stream()
+
.map(x -> new TopicPartition(REMOTE_LOG_METADATA_TOPIC_NAME, x))
+
.collect(Collectors.toSet());
+ consumer.assign(metadataTopicPartitions);
+
+ // Seek to the committed offsets
+ for (Map.Entry<Integer, Long> entry : committedOffsets.entrySet())
{
+ partitionToConsumedOffsets.put(entry.getKey(),
entry.getValue());
+ partitionToProcessedOffsets.put(entry.getKey(),
entry.getValue());
+ consumer.seek(new
TopicPartition(REMOTE_LOG_METADATA_TOPIC_NAME, entry.getKey()),
entry.getValue());
+ }
+
+ lastSyncedPartitionToConsumedOffsets =
Collections.unmodifiableMap(committedOffsets);
+ }
}
@Override
public void run() {
log.info("Started Consumer task thread.");
+ lastSyncedTimeMs = time.milliseconds();
try {
while (!closing) {
maybeWaitForPartitionsAssignment();
log.info("Polling consumer to receive remote log metadata
topic records");
- ConsumerRecords<byte[], byte[]> consumerRecords
- = consumer.poll(Duration.ofMillis(POLL_INTERVAL_MS));
+ ConsumerRecords<byte[], byte[]> consumerRecords =
consumer.poll(Duration.ofMillis(POLL_INTERVAL_MS));
for (ConsumerRecord<byte[], byte[]> record : consumerRecords) {
- handleRemoteLogMetadata(serde.deserialize(record.value()));
- partitionToConsumedOffsets.put(record.partition(),
record.offset());
+ processConsumerRecord(record);
}
+
+ maybeSyncCommittedDataAndOffsets(false);
}
} catch (Exception e) {
log.error("Error occurred in consumer task, close:[{}]", closing,
e);
} finally {
+ maybeSyncCommittedDataAndOffsets(true);
closeConsumer();
log.info("Exiting from consumer task thread");
}
}
+ private void processConsumerRecord(ConsumerRecord<byte[], byte[]> record) {
+ // Taking assignPartitionsLock here as updateAssignmentsForPartitions
changes assignedTopicPartitions
+ // and also calls
remotePartitionMetadataEventHandler.clearTopicPartition(removedPartition) for
the removed
+ // partitions.
+ RemoteLogMetadata remoteLogMetadata =
serde.deserialize(record.value());
+ synchronized (assignPartitionsLock) {
+ if
(assignedTopicPartitions.contains(remoteLogMetadata.topicIdPartition())) {
+
remotePartitionMetadataEventHandler.handleRemoteLogMetadata(remoteLogMetadata);
+ partitionToProcessedOffsets.put(record.partition(),
record.offset());
Review comment:
As I mentioned in my earlier
[comment](https://github.com/apache/kafka/pull/11058#discussion_r719295072), we
will be passing offset and storing it as part of inmemory and snapshots. This
will be done in a followup PR. With that, we do not need to maintain here.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]