satishd commented on a change in pull request #10579: URL: https://github.com/apache/kafka/pull/10579#discussion_r642095350
########## File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java ########## @@ -0,0 +1,253 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.server.log.remote.metadata.storage; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde; +import org.apache.kafka.server.log.remote.storage.RemoteLogMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.time.Duration; +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_NAME; + +/** + * This class is responsible for consuming messages from remote log metadata topic ({@link TopicBasedRemoteLogMetadataManagerConfig#REMOTE_LOG_METADATA_TOPIC_NAME}) + * partitions and maintain the state of the remote log segment metadata. It gives an API to add or remove + * for what topic partition's metadata should be consumed by this instance using + * {{@link #addAssignmentsForPartitions(Set)}} and {@link #removeAssignmentsForPartitions(Set)} respectively. + * <p> + * When a broker is started, controller sends topic partitions that this broker is leader or follower for and the + * partitions to be deleted. This class receives those notifications with + * {@link #addAssignmentsForPartitions(Set)} and {@link #removeAssignmentsForPartitions(Set)} assigns consumer for the + * respective remote log metadata partitions by using {@link RemoteLogMetadataTopicPartitioner#metadataPartition(TopicIdPartition)}. + * Any leadership changes later are called through the same API. We will remove the partitions that ard deleted from + * this broker which are received through {@link #removeAssignmentsForPartitions(Set)}. + * <p> + * After receiving these events it invokes {@link RemotePartitionMetadataEventHandler#handleRemoteLogSegmentMetadata(RemoteLogSegmentMetadata)}, + * which maintains in-memory representation of the state of {@link RemoteLogSegmentMetadata}. + */ +class ConsumerTask implements Runnable, Closeable { + private static final Logger log = LoggerFactory.getLogger(ConsumerTask.class); + + private static final long POLL_INTERVAL_MS = 30L; + + private final RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde(); + private final KafkaConsumer<byte[], byte[]> consumer; + private final RemotePartitionMetadataEventHandler remotePartitionMetadataEventHandler; + private final RemoteLogMetadataTopicPartitioner topicPartitioner; + + private volatile boolean close = false; + private volatile boolean assignPartitions = false; + + private final Object assignPartitionsLock = new Object(); + + // Remote log metadata topic partitions that consumer is assigned to. + private volatile Set<Integer> assignedMetaPartitions = Collections.emptySet(); + + // 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 target end offsets to be consumed. + private final Map<Integer, Long> partitionToTargetEndOffsets = new ConcurrentHashMap<>(); + + // Map of remote log metadata topic partition to consumed offsets. + private final Map<Integer, Long> partitionToConsumedOffsets = new ConcurrentHashMap<>(); + + public ConsumerTask(KafkaConsumer<byte[], byte[]> consumer, + RemotePartitionMetadataEventHandler remotePartitionMetadataEventHandler, + RemoteLogMetadataTopicPartitioner topicPartitioner) { + this.consumer = consumer; + this.remotePartitionMetadataEventHandler = remotePartitionMetadataEventHandler; + this.topicPartitioner = topicPartitioner; + } + + @Override + public void run() { + log.info("Started Consumer task thread."); + try { + while (!close) { + Set<Integer> assignedMetaPartitionsSnapshot = maybeWaitForPartitionsAssignment(); + + if (!assignedMetaPartitionsSnapshot.isEmpty()) { + executeReassignment(assignedMetaPartitionsSnapshot); + } + + log.info("Polling consumer to receive remote log metadata topic records"); + ConsumerRecords<byte[], byte[]> consumerRecords + = consumer.poll(Duration.ofSeconds(POLL_INTERVAL_MS)); + for (ConsumerRecord<byte[], byte[]> record : consumerRecords) { + handleRemoteLogMetadata(serde.deserialize(record.value())); + partitionToConsumedOffsets.put(record.partition(), record.offset()); + } + + // Check whether messages are received till end offsets or not for the assigned metadata partitions. + if (!partitionToTargetEndOffsets.isEmpty()) { + for (Map.Entry<Integer, Long> entry : partitionToTargetEndOffsets.entrySet()) { + final Long offset = partitionToConsumedOffsets.getOrDefault(entry.getKey(), 0L); + if (offset >= entry.getValue()) { + partitionToTargetEndOffsets.remove(entry.getKey()); + } + } + } + } + } catch (Exception e) { + log.error("Error occurred in consumer task, close:[{}]", close, e); + } + + closeConsumer(); + log.info("Exiting from consumer task thread"); + } + + private void closeConsumer() { + log.info("Closing the consumer instance"); + if (consumer != null) { + try { + consumer.close(Duration.ofSeconds(30)); + } catch (Exception e) { + log.error("Error encountered while closing the consumer", e); + } + } + } + + private Set<Integer> maybeWaitForPartitionsAssignment() { + Set<Integer> assignedMetaPartitionsSnapshot = Collections.emptySet(); + synchronized (assignPartitionsLock) { + while (assignedMetaPartitions.isEmpty()) { + // If no partitions are assigned, wait until they are assigned. + log.info("Waiting for assigned remote log metadata partitions.."); + try { + assignPartitionsLock.wait(); + } catch (InterruptedException e) { + throw new KafkaException(e); + } + } + + if (assignPartitions) { + assignedMetaPartitionsSnapshot = new HashSet<>(assignedMetaPartitions); + assignPartitions = false; + } + } + return assignedMetaPartitionsSnapshot; + } + + private void handleRemoteLogMetadata(RemoteLogMetadata remoteLogMetadata) { + remotePartitionMetadataEventHandler.handleRemoteLogMetadata(remoteLogMetadata); + } + + private void executeReassignment(Set<Integer> assignedMetaPartitionsSnapshot) { + Set<TopicPartition> assignedMetaTopicPartitions = assignedMetaPartitionsSnapshot.stream() + .map(partitionNum -> new TopicPartition(REMOTE_LOG_METADATA_TOPIC_NAME, partitionNum)) + .collect(Collectors.toSet()); + log.info("Reassigning partitions to consumer task [{}]", assignedMetaTopicPartitions); + consumer.assign(assignedMetaTopicPartitions); + + log.debug("Fetching end offsets to consumer task [{}]", assignedMetaTopicPartitions); + Map<TopicPartition, Long> endOffsets; + while (true) { + try { + endOffsets = consumer.endOffsets(assignedMetaTopicPartitions, Duration.ofSeconds(30)); Review comment: We have not yet added the code to store the consumed offset and start from those offsets whenever the consumer starts fetching from those partitions. We plan to add that in a subsequent PR. ########## File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java ########## @@ -0,0 +1,253 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.server.log.remote.metadata.storage; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde; +import org.apache.kafka.server.log.remote.storage.RemoteLogMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.time.Duration; +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_NAME; + +/** + * This class is responsible for consuming messages from remote log metadata topic ({@link TopicBasedRemoteLogMetadataManagerConfig#REMOTE_LOG_METADATA_TOPIC_NAME}) + * partitions and maintain the state of the remote log segment metadata. It gives an API to add or remove + * for what topic partition's metadata should be consumed by this instance using + * {{@link #addAssignmentsForPartitions(Set)}} and {@link #removeAssignmentsForPartitions(Set)} respectively. + * <p> + * When a broker is started, controller sends topic partitions that this broker is leader or follower for and the + * partitions to be deleted. This class receives those notifications with + * {@link #addAssignmentsForPartitions(Set)} and {@link #removeAssignmentsForPartitions(Set)} assigns consumer for the + * respective remote log metadata partitions by using {@link RemoteLogMetadataTopicPartitioner#metadataPartition(TopicIdPartition)}. + * Any leadership changes later are called through the same API. We will remove the partitions that ard deleted from + * this broker which are received through {@link #removeAssignmentsForPartitions(Set)}. + * <p> + * After receiving these events it invokes {@link RemotePartitionMetadataEventHandler#handleRemoteLogSegmentMetadata(RemoteLogSegmentMetadata)}, + * which maintains in-memory representation of the state of {@link RemoteLogSegmentMetadata}. + */ +class ConsumerTask implements Runnable, Closeable { + private static final Logger log = LoggerFactory.getLogger(ConsumerTask.class); + + private static final long POLL_INTERVAL_MS = 30L; + + private final RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde(); + private final KafkaConsumer<byte[], byte[]> consumer; + private final RemotePartitionMetadataEventHandler remotePartitionMetadataEventHandler; + private final RemoteLogMetadataTopicPartitioner topicPartitioner; + + private volatile boolean close = false; + private volatile boolean assignPartitions = false; + + private final Object assignPartitionsLock = new Object(); + + // Remote log metadata topic partitions that consumer is assigned to. + private volatile Set<Integer> assignedMetaPartitions = Collections.emptySet(); + + // 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 target end offsets to be consumed. + private final Map<Integer, Long> partitionToTargetEndOffsets = new ConcurrentHashMap<>(); + + // Map of remote log metadata topic partition to consumed offsets. + private final Map<Integer, Long> partitionToConsumedOffsets = new ConcurrentHashMap<>(); + + public ConsumerTask(KafkaConsumer<byte[], byte[]> consumer, + RemotePartitionMetadataEventHandler remotePartitionMetadataEventHandler, + RemoteLogMetadataTopicPartitioner topicPartitioner) { + this.consumer = consumer; + this.remotePartitionMetadataEventHandler = remotePartitionMetadataEventHandler; + this.topicPartitioner = topicPartitioner; + } + + @Override + public void run() { + log.info("Started Consumer task thread."); + try { + while (!close) { + Set<Integer> assignedMetaPartitionsSnapshot = maybeWaitForPartitionsAssignment(); + + if (!assignedMetaPartitionsSnapshot.isEmpty()) { + executeReassignment(assignedMetaPartitionsSnapshot); + } + + log.info("Polling consumer to receive remote log metadata topic records"); + ConsumerRecords<byte[], byte[]> consumerRecords + = consumer.poll(Duration.ofSeconds(POLL_INTERVAL_MS)); + for (ConsumerRecord<byte[], byte[]> record : consumerRecords) { + handleRemoteLogMetadata(serde.deserialize(record.value())); Review comment: Right, I will add that check. ########## File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java ########## @@ -0,0 +1,253 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.server.log.remote.metadata.storage; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde; +import org.apache.kafka.server.log.remote.storage.RemoteLogMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.time.Duration; +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_NAME; + +/** + * This class is responsible for consuming messages from remote log metadata topic ({@link TopicBasedRemoteLogMetadataManagerConfig#REMOTE_LOG_METADATA_TOPIC_NAME}) + * partitions and maintain the state of the remote log segment metadata. It gives an API to add or remove + * for what topic partition's metadata should be consumed by this instance using + * {{@link #addAssignmentsForPartitions(Set)}} and {@link #removeAssignmentsForPartitions(Set)} respectively. + * <p> + * When a broker is started, controller sends topic partitions that this broker is leader or follower for and the + * partitions to be deleted. This class receives those notifications with + * {@link #addAssignmentsForPartitions(Set)} and {@link #removeAssignmentsForPartitions(Set)} assigns consumer for the + * respective remote log metadata partitions by using {@link RemoteLogMetadataTopicPartitioner#metadataPartition(TopicIdPartition)}. + * Any leadership changes later are called through the same API. We will remove the partitions that ard deleted from + * this broker which are received through {@link #removeAssignmentsForPartitions(Set)}. + * <p> + * After receiving these events it invokes {@link RemotePartitionMetadataEventHandler#handleRemoteLogSegmentMetadata(RemoteLogSegmentMetadata)}, + * which maintains in-memory representation of the state of {@link RemoteLogSegmentMetadata}. + */ +class ConsumerTask implements Runnable, Closeable { + private static final Logger log = LoggerFactory.getLogger(ConsumerTask.class); + + private static final long POLL_INTERVAL_MS = 30L; + + private final RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde(); + private final KafkaConsumer<byte[], byte[]> consumer; + private final RemotePartitionMetadataEventHandler remotePartitionMetadataEventHandler; + private final RemoteLogMetadataTopicPartitioner topicPartitioner; + + private volatile boolean close = false; + private volatile boolean assignPartitions = false; + + private final Object assignPartitionsLock = new Object(); + + // Remote log metadata topic partitions that consumer is assigned to. + private volatile Set<Integer> assignedMetaPartitions = Collections.emptySet(); + + // 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 target end offsets to be consumed. + private final Map<Integer, Long> partitionToTargetEndOffsets = new ConcurrentHashMap<>(); + + // Map of remote log metadata topic partition to consumed offsets. + private final Map<Integer, Long> partitionToConsumedOffsets = new ConcurrentHashMap<>(); + + public ConsumerTask(KafkaConsumer<byte[], byte[]> consumer, + RemotePartitionMetadataEventHandler remotePartitionMetadataEventHandler, + RemoteLogMetadataTopicPartitioner topicPartitioner) { + this.consumer = consumer; + this.remotePartitionMetadataEventHandler = remotePartitionMetadataEventHandler; + this.topicPartitioner = topicPartitioner; + } + + @Override + public void run() { + log.info("Started Consumer task thread."); + try { + while (!close) { + Set<Integer> assignedMetaPartitionsSnapshot = maybeWaitForPartitionsAssignment(); + + if (!assignedMetaPartitionsSnapshot.isEmpty()) { + executeReassignment(assignedMetaPartitionsSnapshot); + } + + log.info("Polling consumer to receive remote log metadata topic records"); + ConsumerRecords<byte[], byte[]> consumerRecords + = consumer.poll(Duration.ofSeconds(POLL_INTERVAL_MS)); + for (ConsumerRecord<byte[], byte[]> record : consumerRecords) { + handleRemoteLogMetadata(serde.deserialize(record.value())); + partitionToConsumedOffsets.put(record.partition(), record.offset()); + } + + // Check whether messages are received till end offsets or not for the assigned metadata partitions. + if (!partitionToTargetEndOffsets.isEmpty()) { + for (Map.Entry<Integer, Long> entry : partitionToTargetEndOffsets.entrySet()) { + final Long offset = partitionToConsumedOffsets.getOrDefault(entry.getKey(), 0L); + if (offset >= entry.getValue()) { + partitionToTargetEndOffsets.remove(entry.getKey()); + } + } + } + } + } catch (Exception e) { + log.error("Error occurred in consumer task, close:[{}]", close, e); + } + + closeConsumer(); + log.info("Exiting from consumer task thread"); + } + + private void closeConsumer() { + log.info("Closing the consumer instance"); + if (consumer != null) { + try { + consumer.close(Duration.ofSeconds(30)); + } catch (Exception e) { + log.error("Error encountered while closing the consumer", e); + } + } + } + + private Set<Integer> maybeWaitForPartitionsAssignment() { + Set<Integer> assignedMetaPartitionsSnapshot = Collections.emptySet(); + synchronized (assignPartitionsLock) { + while (assignedMetaPartitions.isEmpty()) { + // If no partitions are assigned, wait until they are assigned. + log.info("Waiting for assigned remote log metadata partitions.."); + try { + assignPartitionsLock.wait(); + } catch (InterruptedException e) { + throw new KafkaException(e); + } + } + + if (assignPartitions) { + assignedMetaPartitionsSnapshot = new HashSet<>(assignedMetaPartitions); + assignPartitions = false; + } + } + return assignedMetaPartitionsSnapshot; + } + + private void handleRemoteLogMetadata(RemoteLogMetadata remoteLogMetadata) { + remotePartitionMetadataEventHandler.handleRemoteLogMetadata(remoteLogMetadata); + } + + private void executeReassignment(Set<Integer> assignedMetaPartitionsSnapshot) { + Set<TopicPartition> assignedMetaTopicPartitions = assignedMetaPartitionsSnapshot.stream() + .map(partitionNum -> new TopicPartition(REMOTE_LOG_METADATA_TOPIC_NAME, partitionNum)) + .collect(Collectors.toSet()); + log.info("Reassigning partitions to consumer task [{}]", assignedMetaTopicPartitions); + consumer.assign(assignedMetaTopicPartitions); + + log.debug("Fetching end offsets to consumer task [{}]", assignedMetaTopicPartitions); + Map<TopicPartition, Long> endOffsets; + while (true) { + try { + endOffsets = consumer.endOffsets(assignedMetaTopicPartitions, Duration.ofSeconds(30)); + break; + } catch (Exception e) { + // ignore exception + log.debug("Error encountered in fetching end offsets", e); + } + } + log.debug("Fetched end offsets to consumer task [{}]", endOffsets); + + for (Map.Entry<TopicPartition, Long> entry : endOffsets.entrySet()) { + if (entry.getValue() > 0) { + partitionToTargetEndOffsets.put(entry.getKey().partition(), entry.getValue()); + } + } + } + + public void addAssignmentsForPartitions(Set<TopicIdPartition> updatedPartitions) { + updateAssignmentsForPartitions(updatedPartitions, Collections.emptySet()); + } + + public void removeAssignmentsForPartitions(Set<TopicIdPartition> partitions) { + updateAssignmentsForPartitions(Collections.emptySet(), partitions); + } + + private void updateAssignmentsForPartitions(Set<TopicIdPartition> addedPartitions, + Set<TopicIdPartition> removedPartitions) { + log.info("Updating assignments for addedPartitions: {} and removedPartition: {}", addedPartitions, removedPartitions); + ensureNotClosed(); + + Objects.requireNonNull(addedPartitions, "addedPartitions must not be null"); + Objects.requireNonNull(removedPartitions, "removedPartitions must not be null"); + + if (addedPartitions.isEmpty() && removedPartitions.isEmpty()) { + return; + } + + synchronized (assignPartitionsLock) { + Set<TopicIdPartition> updatedReassignedPartitions = new HashSet<>(assignedTopicPartitions); + updatedReassignedPartitions.addAll(addedPartitions); + updatedReassignedPartitions.removeAll(removedPartitions); + Set<Integer> updatedAssignedMetaPartitions = new HashSet<>(); + for (TopicIdPartition tp : updatedReassignedPartitions) { + updatedAssignedMetaPartitions.add(topicPartitioner.metadataPartition(tp)); + } + + if (!updatedAssignedMetaPartitions.equals(assignedMetaPartitions)) { + assignedTopicPartitions = Collections.unmodifiableSet(updatedReassignedPartitions); + assignedMetaPartitions = Collections.unmodifiableSet(updatedAssignedMetaPartitions); Review comment: This is not addressed in this PR. I planned to have a followup PR for these changes. I may use a different consumer for the newly subscribed partitions to build the state. ########## File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerConfig.java ########## @@ -0,0 +1,197 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.server.log.remote.metadata.storage; + +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +import static org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.kafka.common.config.ConfigDef.Importance.LOW; +import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; +import static org.apache.kafka.common.config.ConfigDef.Type.INT; +import static org.apache.kafka.common.config.ConfigDef.Type.LONG; + +public final class TopicBasedRemoteLogMetadataManagerConfig { + private static final Logger log = LoggerFactory.getLogger(TopicBasedRemoteLogMetadataManagerConfig.class.getName()); + + public static final String REMOTE_LOG_METADATA_TOPIC_NAME = "__remote_log_metadata"; + + public static final String REMOTE_LOG_METADATA_TOPIC_REPLICATION_FACTOR_PROP = "remote.log.metadata.topic.replication.factor"; + public static final String REMOTE_LOG_METADATA_TOPIC_PARTITIONS_PROP = "remote.log.metadata.topic.num.partitions"; + public static final String REMOTE_LOG_METADATA_TOPIC_RETENTION_MILLIS_PROP = "remote.log.metadata.topic.retention.ms"; + public static final String REMOTE_LOG_METADATA_CONSUME_WAIT_MS_PROP = "remote.log.metadata.publish.wait.ms"; + + public static final int DEFAULT_REMOTE_LOG_METADATA_TOPIC_PARTITIONS = 50; + public static final long DEFAULT_REMOTE_LOG_METADATA_TOPIC_RETENTION_MILLIS = -1L; + public static final int DEFAULT_REMOTE_LOG_METADATA_TOPIC_REPLICATION_FACTOR = 3; + public static final long DEFAULT_REMOTE_LOG_METADATA_CONSUME_WAIT_MS = 60 * 1000L; + + public static final String REMOTE_LOG_METADATA_TOPIC_REPLICATION_FACTOR_DOC = "Replication factor of remote log metadata Topic."; + public static final String REMOTE_LOG_METADATA_TOPIC_PARTITIONS_DOC = "The number of partitions for remote log metadata Topic."; + public static final String REMOTE_LOG_METADATA_TOPIC_RETENTION_MILLIS_DOC = "Remote log metadata topic log retention in milli seconds." + + "Default: -1, that means unlimited. Users can configure this value based on their use cases. " + + "To avoid any data loss, this value should be more than the maximum retention period of any topic enabled with " + + "tiered storage in the cluster."; + public static final String REMOTE_LOG_METADATA_CONSUME_WAIT_MS_DOC = "The amount of time in milli seconds to wait for the local consumer to " + + "receive the published event."; + + public static final String REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX = "remote.log.metadata.common.client."; + public static final String REMOTE_LOG_METADATA_PRODUCER_PREFIX = "remote.log.metadata.producer."; + public static final String REMOTE_LOG_METADATA_CONSUMER_PREFIX = "remote.log.metadata.consumer."; + + private static final String REMOTE_LOG_METADATA_CLIENT_PREFIX = "__remote_log_metadata_client"; + private static final String BROKER_ID = "broker.id"; + + private static final ConfigDef CONFIG = new ConfigDef(); + static { + CONFIG.define(REMOTE_LOG_METADATA_TOPIC_REPLICATION_FACTOR_PROP, INT, DEFAULT_REMOTE_LOG_METADATA_TOPIC_REPLICATION_FACTOR, atLeast(1), LOW, + REMOTE_LOG_METADATA_TOPIC_REPLICATION_FACTOR_DOC) + .define(REMOTE_LOG_METADATA_TOPIC_PARTITIONS_PROP, INT, DEFAULT_REMOTE_LOG_METADATA_TOPIC_PARTITIONS, atLeast(1), LOW, + REMOTE_LOG_METADATA_TOPIC_PARTITIONS_DOC) + .define(REMOTE_LOG_METADATA_TOPIC_RETENTION_MILLIS_PROP, LONG, DEFAULT_REMOTE_LOG_METADATA_TOPIC_RETENTION_MILLIS, LOW, + REMOTE_LOG_METADATA_TOPIC_RETENTION_MILLIS_DOC) + .define(REMOTE_LOG_METADATA_CONSUME_WAIT_MS_PROP, LONG, DEFAULT_REMOTE_LOG_METADATA_CONSUME_WAIT_MS, atLeast(0), LOW, + REMOTE_LOG_METADATA_CONSUME_WAIT_MS_DOC); + } + + private final String clientIdPrefix; + private final int metadataTopicPartitionsCount; + private final String bootstrapServers; + private final long consumeWaitMs; + private final long metadataTopicRetentionMillis; + + private Map<String, Object> consumerProps; + private Map<String, Object> producerProps; + + public TopicBasedRemoteLogMetadataManagerConfig(Map<String, ?> props) { + log.info("Received props: [{}]", props); + Objects.requireNonNull(props, "props can not be null"); + + Map<String, Object> parsedConfigs = CONFIG.parse(props); + + bootstrapServers = (String) props.get(BOOTSTRAP_SERVERS_CONFIG); Review comment: REMOTE_LOG_METADATA_CLIENT_PREFIX is just a prefix for generating client-ids for producer and consumer. If you are talking about the remote log metadata property prefix, It is assumed that the caller would have already removed those prefixes and sent the config. These prefixes are defined [here](https://github.com/apache/kafka/pull/10733/files/f185b4f86b457da35c21f1461fe1b1e1d73e4939#diff-31933b76c0578600db8744507e5c5ff840820d5a181a2a5d1e08c50bdc64ea6fR46) ########## File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerConfig.java ########## @@ -0,0 +1,197 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.server.log.remote.metadata.storage; + +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +import static org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.kafka.common.config.ConfigDef.Importance.LOW; +import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; +import static org.apache.kafka.common.config.ConfigDef.Type.INT; +import static org.apache.kafka.common.config.ConfigDef.Type.LONG; + +public final class TopicBasedRemoteLogMetadataManagerConfig { + private static final Logger log = LoggerFactory.getLogger(TopicBasedRemoteLogMetadataManagerConfig.class.getName()); + + public static final String REMOTE_LOG_METADATA_TOPIC_NAME = "__remote_log_metadata"; + + public static final String REMOTE_LOG_METADATA_TOPIC_REPLICATION_FACTOR_PROP = "remote.log.metadata.topic.replication.factor"; + public static final String REMOTE_LOG_METADATA_TOPIC_PARTITIONS_PROP = "remote.log.metadata.topic.num.partitions"; + public static final String REMOTE_LOG_METADATA_TOPIC_RETENTION_MILLIS_PROP = "remote.log.metadata.topic.retention.ms"; + public static final String REMOTE_LOG_METADATA_CONSUME_WAIT_MS_PROP = "remote.log.metadata.publish.wait.ms"; + + public static final int DEFAULT_REMOTE_LOG_METADATA_TOPIC_PARTITIONS = 50; + public static final long DEFAULT_REMOTE_LOG_METADATA_TOPIC_RETENTION_MILLIS = -1L; + public static final int DEFAULT_REMOTE_LOG_METADATA_TOPIC_REPLICATION_FACTOR = 3; + public static final long DEFAULT_REMOTE_LOG_METADATA_CONSUME_WAIT_MS = 60 * 1000L; + + public static final String REMOTE_LOG_METADATA_TOPIC_REPLICATION_FACTOR_DOC = "Replication factor of remote log metadata Topic."; + public static final String REMOTE_LOG_METADATA_TOPIC_PARTITIONS_DOC = "The number of partitions for remote log metadata Topic."; + public static final String REMOTE_LOG_METADATA_TOPIC_RETENTION_MILLIS_DOC = "Remote log metadata topic log retention in milli seconds." + + "Default: -1, that means unlimited. Users can configure this value based on their use cases. " + + "To avoid any data loss, this value should be more than the maximum retention period of any topic enabled with " + + "tiered storage in the cluster."; + public static final String REMOTE_LOG_METADATA_CONSUME_WAIT_MS_DOC = "The amount of time in milli seconds to wait for the local consumer to " + + "receive the published event."; + + public static final String REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX = "remote.log.metadata.common.client."; + public static final String REMOTE_LOG_METADATA_PRODUCER_PREFIX = "remote.log.metadata.producer."; + public static final String REMOTE_LOG_METADATA_CONSUMER_PREFIX = "remote.log.metadata.consumer."; + + private static final String REMOTE_LOG_METADATA_CLIENT_PREFIX = "__remote_log_metadata_client"; + private static final String BROKER_ID = "broker.id"; + + private static final ConfigDef CONFIG = new ConfigDef(); + static { + CONFIG.define(REMOTE_LOG_METADATA_TOPIC_REPLICATION_FACTOR_PROP, INT, DEFAULT_REMOTE_LOG_METADATA_TOPIC_REPLICATION_FACTOR, atLeast(1), LOW, + REMOTE_LOG_METADATA_TOPIC_REPLICATION_FACTOR_DOC) + .define(REMOTE_LOG_METADATA_TOPIC_PARTITIONS_PROP, INT, DEFAULT_REMOTE_LOG_METADATA_TOPIC_PARTITIONS, atLeast(1), LOW, + REMOTE_LOG_METADATA_TOPIC_PARTITIONS_DOC) + .define(REMOTE_LOG_METADATA_TOPIC_RETENTION_MILLIS_PROP, LONG, DEFAULT_REMOTE_LOG_METADATA_TOPIC_RETENTION_MILLIS, LOW, + REMOTE_LOG_METADATA_TOPIC_RETENTION_MILLIS_DOC) + .define(REMOTE_LOG_METADATA_CONSUME_WAIT_MS_PROP, LONG, DEFAULT_REMOTE_LOG_METADATA_CONSUME_WAIT_MS, atLeast(0), LOW, + REMOTE_LOG_METADATA_CONSUME_WAIT_MS_DOC); + } + + private final String clientIdPrefix; + private final int metadataTopicPartitionsCount; + private final String bootstrapServers; + private final long consumeWaitMs; + private final long metadataTopicRetentionMillis; + + private Map<String, Object> consumerProps; + private Map<String, Object> producerProps; + + public TopicBasedRemoteLogMetadataManagerConfig(Map<String, ?> props) { + log.info("Received props: [{}]", props); + Objects.requireNonNull(props, "props can not be null"); + + Map<String, Object> parsedConfigs = CONFIG.parse(props); + + bootstrapServers = (String) props.get(BOOTSTRAP_SERVERS_CONFIG); + if (bootstrapServers == null || bootstrapServers.isEmpty()) { + throw new IllegalArgumentException(BOOTSTRAP_SERVERS_CONFIG + " config must not be null or empty."); + } + + consumeWaitMs = (long) parsedConfigs.get(REMOTE_LOG_METADATA_CONSUME_WAIT_MS_PROP); + metadataTopicPartitionsCount = (int) parsedConfigs.get(REMOTE_LOG_METADATA_TOPIC_PARTITIONS_PROP); + metadataTopicRetentionMillis = (long) parsedConfigs.get(REMOTE_LOG_METADATA_TOPIC_RETENTION_MILLIS_PROP); + if (metadataTopicRetentionMillis != -1 && metadataTopicRetentionMillis <= 0) { + throw new IllegalArgumentException("Invalid metadata topic retention in millis: " + metadataTopicRetentionMillis); + } + + clientIdPrefix = REMOTE_LOG_METADATA_CLIENT_PREFIX + "_" + props.get(BROKER_ID); + + initializeProducerConsumerProperties(props); + } + + private void initializeProducerConsumerProperties(Map<String, ?> configs) { + Map<String, Object> commonClientConfigs = new HashMap<>(); + commonClientConfigs.put(BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + + Map<String, Object> producerOnlyConfigs = new HashMap<>(); + Map<String, Object> consumerOnlyConfigs = new HashMap<>(); + + for (Map.Entry<String, ?> entry : configs.entrySet()) { + String key = entry.getKey(); + if (key.startsWith(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX)) { Review comment: This is to avoid duplicate entries for both the producer and consumer. We added that in the KIP earlier. If duplicating is the way we use at other places if any, I am fine with that. ########## File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java ########## @@ -0,0 +1,253 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.server.log.remote.metadata.storage; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde; +import org.apache.kafka.server.log.remote.storage.RemoteLogMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.time.Duration; +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_NAME; + +/** + * This class is responsible for consuming messages from remote log metadata topic ({@link TopicBasedRemoteLogMetadataManagerConfig#REMOTE_LOG_METADATA_TOPIC_NAME}) + * partitions and maintain the state of the remote log segment metadata. It gives an API to add or remove + * for what topic partition's metadata should be consumed by this instance using + * {{@link #addAssignmentsForPartitions(Set)}} and {@link #removeAssignmentsForPartitions(Set)} respectively. + * <p> + * When a broker is started, controller sends topic partitions that this broker is leader or follower for and the + * partitions to be deleted. This class receives those notifications with + * {@link #addAssignmentsForPartitions(Set)} and {@link #removeAssignmentsForPartitions(Set)} assigns consumer for the + * respective remote log metadata partitions by using {@link RemoteLogMetadataTopicPartitioner#metadataPartition(TopicIdPartition)}. + * Any leadership changes later are called through the same API. We will remove the partitions that ard deleted from + * this broker which are received through {@link #removeAssignmentsForPartitions(Set)}. + * <p> + * After receiving these events it invokes {@link RemotePartitionMetadataEventHandler#handleRemoteLogSegmentMetadata(RemoteLogSegmentMetadata)}, + * which maintains in-memory representation of the state of {@link RemoteLogSegmentMetadata}. + */ +class ConsumerTask implements Runnable, Closeable { + private static final Logger log = LoggerFactory.getLogger(ConsumerTask.class); + + private static final long POLL_INTERVAL_MS = 30L; + + private final RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde(); + private final KafkaConsumer<byte[], byte[]> consumer; + private final RemotePartitionMetadataEventHandler remotePartitionMetadataEventHandler; + private final RemoteLogMetadataTopicPartitioner topicPartitioner; + + private volatile boolean close = false; + private volatile boolean assignPartitions = false; + + private final Object assignPartitionsLock = new Object(); + + // Remote log metadata topic partitions that consumer is assigned to. + private volatile Set<Integer> assignedMetaPartitions = Collections.emptySet(); + + // 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 target end offsets to be consumed. + private final Map<Integer, Long> partitionToTargetEndOffsets = new ConcurrentHashMap<>(); + + // Map of remote log metadata topic partition to consumed offsets. + private final Map<Integer, Long> partitionToConsumedOffsets = new ConcurrentHashMap<>(); + + public ConsumerTask(KafkaConsumer<byte[], byte[]> consumer, + RemotePartitionMetadataEventHandler remotePartitionMetadataEventHandler, + RemoteLogMetadataTopicPartitioner topicPartitioner) { + this.consumer = consumer; + this.remotePartitionMetadataEventHandler = remotePartitionMetadataEventHandler; + this.topicPartitioner = topicPartitioner; + } + + @Override + public void run() { + log.info("Started Consumer task thread."); + try { + while (!close) { + Set<Integer> assignedMetaPartitionsSnapshot = maybeWaitForPartitionsAssignment(); + + if (!assignedMetaPartitionsSnapshot.isEmpty()) { + executeReassignment(assignedMetaPartitionsSnapshot); + } + + log.info("Polling consumer to receive remote log metadata topic records"); + ConsumerRecords<byte[], byte[]> consumerRecords + = consumer.poll(Duration.ofSeconds(POLL_INTERVAL_MS)); + for (ConsumerRecord<byte[], byte[]> record : consumerRecords) { + handleRemoteLogMetadata(serde.deserialize(record.value())); + partitionToConsumedOffsets.put(record.partition(), record.offset()); + } + + // Check whether messages are received till end offsets or not for the assigned metadata partitions. + if (!partitionToTargetEndOffsets.isEmpty()) { + for (Map.Entry<Integer, Long> entry : partitionToTargetEndOffsets.entrySet()) { + final Long offset = partitionToConsumedOffsets.getOrDefault(entry.getKey(), 0L); + if (offset >= entry.getValue()) { + partitionToTargetEndOffsets.remove(entry.getKey()); Review comment: This is leftover code for other changes that I was working on for handling partition moving to a new broker in another branch. I will remove it. -- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org