junrao commented on a change in pull request #10579: URL: https://github.com/apache/kafka/pull/10579#discussion_r664866782
########## File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/ConsumerTask.java ########## @@ -0,0 +1,242 @@ +/* + * 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 are 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 = 100; + + private final RemoteLogMetadataSerde serde = new RemoteLogMetadataSerde(); + private final KafkaConsumer<byte[], byte[]> consumer; + private final RemotePartitionMetadataEventHandler remotePartitionMetadataEventHandler; + private final RemoteLogMetadataTopicPartitioner topicPartitioner; + + // It indicates whether the closing process has been started or not. If it is set as true, + // consumer will stop consuming messages and it will not allow partition assignments to be updated. + private volatile boolean closing = false; + // It indicates whether the consumer needs to assign the partitions or not. This is set when it is + // determined that the consumer needs to be assigned with the updated partitions. + 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 consumed offsets. + private final Map<Integer, Long> partitionToConsumedOffsets = new ConcurrentHashMap<>(); + + 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; + } + + @Override + public void run() { + log.info("Started Consumer task thread."); + 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)); + for (ConsumerRecord<byte[], byte[]> record : consumerRecords) { + handleRemoteLogMetadata(serde.deserialize(record.value())); + partitionToConsumedOffsets.put(record.partition(), record.offset()); + } + } + } catch (Exception e) { + log.error("Error occurred in consumer task, close:[{}]", closing, e); + } finally { + closeConsumer(); + log.info("Exiting from consumer task thread"); + } + } + + private void closeConsumer() { + log.info("Closing the consumer instance"); + try { + consumer.close(Duration.ofSeconds(30)); + } catch (Exception e) { + log.error("Error encountered while closing the consumer", e); + } + } + + private void maybeWaitForPartitionsAssignment() { + Set<Integer> assignedMetaPartitionsSnapshot = Collections.emptySet(); + synchronized (assignPartitionsLock) { + // If it is closing, return immediately. This should be inside the assignPartitionsLock as the closing is updated + // in close() method with in the same lock to avoid any race conditions. + if (closing) { + return; + } + + while (assignedMetaPartitions.isEmpty() && !closing) { Review comment: The test for closing seem unnecessary since closing can't change while synchronized on assignPartitionsLock. ########## 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: Is this comment addressed? ########## File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManager.java ########## @@ -0,0 +1,432 @@ +/* + * 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.admin.AdminClient; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.config.TopicConfig; +import org.apache.kafka.common.errors.RetriableException; +import org.apache.kafka.common.errors.TopicExistsException; +import org.apache.kafka.common.utils.KafkaThread; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.server.log.remote.storage.RemoteLogMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteLogMetadataManager; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState; +import org.apache.kafka.server.log.remote.storage.RemotePartitionDeleteMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteStorageException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.time.Duration; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * This is the {@link RemoteLogMetadataManager} implementation with storage as an internal topic with name {@link TopicBasedRemoteLogMetadataManagerConfig#REMOTE_LOG_METADATA_TOPIC_NAME}. + * This is used to publish and fetch {@link RemoteLogMetadata} for the registered user topic partitions with + * {@link #onPartitionLeadershipChanges(Set, Set)}. Each broker will have an instance of this class and it subscribes + * to metadata updates for the registered user topic partitions. + */ +public class TopicBasedRemoteLogMetadataManager implements RemoteLogMetadataManager { + private static final Logger log = LoggerFactory.getLogger(TopicBasedRemoteLogMetadataManager.class); + private static final long INITIALIZATION_RETRY_INTERVAL_MS = 30000L; + + private volatile boolean configured = false; + + // It indicates whether the close process of this instance is started or not via #close() method. + // Using AtomicBoolean instead of volatile as it may encounter http://findbugs.sourceforge.net/bugDescriptions.html#SP_SPIN_ON_FIELD + // if the field is read but not updated in a spin loop like in #initializeResources() method. + private final AtomicBoolean closing = new AtomicBoolean(false); + private final AtomicBoolean initialized = new AtomicBoolean(false); + private final Time time = Time.SYSTEM; + + private Thread initializationThread; + private volatile ProducerManager producerManager; + private volatile ConsumerManager consumerManager; + + // This allows to gracefully close this instance using {@link #close()} method while there are some pending or new + // requests calling different methods which use the resources like producer/consumer managers. + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + + private final RemotePartitionMetadataStore remotePartitionMetadataStore = new RemotePartitionMetadataStore(); + private volatile TopicBasedRemoteLogMetadataManagerConfig rlmmConfig; + private volatile RemoteLogMetadataTopicPartitioner rlmmTopicPartitioner; + private volatile Set<TopicIdPartition> pendingAssignPartitions = Collections.synchronizedSet(new HashSet<>()); + + @Override + public void addRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata) + throws RemoteStorageException { + Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null"); + + // This allows gracefully rejecting the requests while closing of this instance is in progress, which triggers + // closing the producer/consumer manager instances. + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + // This method is allowed only to add remote log segment with the initial state(which is RemoteLogSegmentState.COPY_SEGMENT_STARTED) + // but not to update the existing remote log segment metadata. + if (remoteLogSegmentMetadata.state() != RemoteLogSegmentState.COPY_SEGMENT_STARTED) { + throw new IllegalArgumentException( + "Given remoteLogSegmentMetadata should have state as " + RemoteLogSegmentState.COPY_SEGMENT_STARTED + + " but it contains state as: " + remoteLogSegmentMetadata.state()); + } + + // Publish the message to the topic. + doPublishMetadata(remoteLogSegmentMetadata.remoteLogSegmentId().topicIdPartition(), + remoteLogSegmentMetadata); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate segmentMetadataUpdate) + throws RemoteStorageException { + Objects.requireNonNull(segmentMetadataUpdate, "segmentMetadataUpdate can not be null"); + + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + // Callers should use addRemoteLogSegmentMetadata to add RemoteLogSegmentMetadata with state as + // RemoteLogSegmentState.COPY_SEGMENT_STARTED. + if (segmentMetadataUpdate.state() == RemoteLogSegmentState.COPY_SEGMENT_STARTED) { + throw new IllegalArgumentException("Given remoteLogSegmentMetadata should not have the state as: " + + RemoteLogSegmentState.COPY_SEGMENT_STARTED); + } + + // Publish the message to the topic. + doPublishMetadata(segmentMetadataUpdate.remoteLogSegmentId().topicIdPartition(), segmentMetadataUpdate); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public void putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata) + throws RemoteStorageException { + Objects.requireNonNull(remotePartitionDeleteMetadata, "remotePartitionDeleteMetadata can not be null"); + + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + doPublishMetadata(remotePartitionDeleteMetadata.topicIdPartition(), remotePartitionDeleteMetadata); + } finally { + lock.readLock().unlock(); + } + } + + private void doPublishMetadata(TopicIdPartition topicIdPartition, RemoteLogMetadata remoteLogMetadata) + throws RemoteStorageException { + log.debug("Publishing metadata for partition: [{}] with context: [{}]", topicIdPartition, remoteLogMetadata); + + try { + // Publish the message to the topic. + RecordMetadata recordMetadata = producerManager.publishMessage(remoteLogMetadata); + // Wait until the consumer catches up with this offset. This will ensure read-after-write consistency + // semantics. + consumerManager.waitTillConsumptionCatchesUp(recordMetadata); + } catch (KafkaException e) { + if (e instanceof RetriableException) { + throw e; + } else { + throw new RemoteStorageException(e); + } + } + } + + @Override + public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(TopicIdPartition topicIdPartition, + int epochForOffset, + long offset) + throws RemoteStorageException { + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + return remotePartitionMetadataStore.remoteLogSegmentMetadata(topicIdPartition, offset, epochForOffset); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public Optional<Long> highestOffsetForEpoch(TopicIdPartition topicIdPartition, + int leaderEpoch) + throws RemoteStorageException { + lock.readLock().lock(); + try { + + ensureInitializedAndNotClosed(); + + return remotePartitionMetadataStore.highestLogOffset(topicIdPartition, leaderEpoch); + } finally { + lock.readLock().unlock(); + } + + } + + @Override + public Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicIdPartition topicIdPartition) + throws RemoteStorageException { + Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null"); + + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + return remotePartitionMetadataStore.listRemoteLogSegments(topicIdPartition); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicIdPartition topicIdPartition, int leaderEpoch) + throws RemoteStorageException { + Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null"); + + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + return remotePartitionMetadataStore.listRemoteLogSegments(topicIdPartition, leaderEpoch); + } finally { + lock.readLock().unlock(); + } + } + + public int metadataPartition(TopicIdPartition topicIdPartition) { + return rlmmTopicPartitioner.metadataPartition(topicIdPartition); + } + + // Visible For Testing + public Optional<Long> receivedOffsetForPartition(int metadataPartition) { + return consumerManager.receivedOffsetForPartition(metadataPartition); + } + + @Override + public void onPartitionLeadershipChanges(Set<TopicIdPartition> leaderPartitions, + Set<TopicIdPartition> followerPartitions) { + Objects.requireNonNull(leaderPartitions, "leaderPartitions can not be null"); + Objects.requireNonNull(followerPartitions, "followerPartitions can not be null"); + + log.info("Received leadership notifications with leader partitions {} and follower partitions {}", + leaderPartitions, followerPartitions); + + HashSet<TopicIdPartition> allPartitions = new HashSet<>(leaderPartitions); + allPartitions.addAll(followerPartitions); + lock.readLock().lock(); + try { + if (closing.get()) { + throw new IllegalStateException("This instance is in closing state"); + } + + if (!initialized.get()) { + // If it is not yet initialized, then keep them as pending partitions and assign them + // when it is initialized successfully in initializeResources(). + this.pendingAssignPartitions.addAll(allPartitions); + } else { + this.pendingAssignPartitions.clear(); + consumerManager.addAssignmentsForPartitions(allPartitions); + } + } finally { + lock.readLock().unlock(); + } + } + + @Override + public void onStopPartitions(Set<TopicIdPartition> partitions) { + lock.readLock().lock(); + try { + if (closing.get()) { + throw new IllegalStateException("This instance is in closing state"); + } + + if (!initialized.get()) { + // If it is not yet initialized, then remove them from the pending partitions if any. + if (!pendingAssignPartitions.isEmpty()) { + pendingAssignPartitions.removeAll(partitions); + } + } else { + consumerManager.removeAssignmentsForPartitions(partitions); + } + } finally { + lock.readLock().unlock(); + } + } + + @Override + public void configure(Map<String, ?> configs) { + Objects.requireNonNull(configs, "configs can not be null."); + + lock.writeLock().lock(); + try { + if (configured) { + log.info("Skipping configure as it is already configured."); + return; + } + + log.info("Started initializing with configs: {}", configs); + + rlmmConfig = new TopicBasedRemoteLogMetadataManagerConfig(configs); + rlmmTopicPartitioner = new RemoteLogMetadataTopicPartitioner(rlmmConfig.metadataTopicPartitionsCount()); + configured = true; + log.info("Successfully initialized with rlmmConfig: {}", rlmmConfig); + + // Scheduling the initialization producer/consumer managers in a separate thread. Required resources may + // not yet be available now. This thread makes sure that it is retried at regular intervals until it is + // successful. + initializationThread = KafkaThread.nonDaemon("RLMMInitializationThread", () -> initializeResources()); Review comment: > It was required to be retried until the topic is successfully created. I added the logic to check for topic creation too. I am still not sure why we need to initialize in a separate thread. If we can't create the metadata topic or instantiate the producer/consumer due to wrong configurations, we want to fail fast by throwing an error to shut down the broker. ########## File path: storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentLifecycleManager.java ########## @@ -0,0 +1,60 @@ +/* + * 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.common.TopicIdPartition; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate; +import org.apache.kafka.server.log.remote.storage.RemoteStorageException; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Iterator; +import java.util.Optional; + +/** + * This interface defines the lifecycle methods for {@code RemoteLogSegmentMetadata}. {@link RemoteLogSegmentLifecycleTest} tests + * different implementations of this interface. This is responsible for managing all the segments for a given {@code topicIdPartition} + * registered with {@link #initialize(TopicIdPartition)}. + */ +public interface RemoteLogSegmentLifecycleManager extends Closeable { Review comment: Is RemoteLogSegmentLifecycleManager used for tests only? If so, could we move it under tests? ########## File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManager.java ########## @@ -0,0 +1,432 @@ +/* + * 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.admin.AdminClient; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.config.TopicConfig; +import org.apache.kafka.common.errors.RetriableException; +import org.apache.kafka.common.errors.TopicExistsException; +import org.apache.kafka.common.utils.KafkaThread; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.server.log.remote.storage.RemoteLogMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteLogMetadataManager; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState; +import org.apache.kafka.server.log.remote.storage.RemotePartitionDeleteMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteStorageException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.time.Duration; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * This is the {@link RemoteLogMetadataManager} implementation with storage as an internal topic with name {@link TopicBasedRemoteLogMetadataManagerConfig#REMOTE_LOG_METADATA_TOPIC_NAME}. + * This is used to publish and fetch {@link RemoteLogMetadata} for the registered user topic partitions with + * {@link #onPartitionLeadershipChanges(Set, Set)}. Each broker will have an instance of this class and it subscribes + * to metadata updates for the registered user topic partitions. + */ +public class TopicBasedRemoteLogMetadataManager implements RemoteLogMetadataManager { + private static final Logger log = LoggerFactory.getLogger(TopicBasedRemoteLogMetadataManager.class); + private static final long INITIALIZATION_RETRY_INTERVAL_MS = 30000L; + + private volatile boolean configured = false; + + // It indicates whether the close process of this instance is started or not via #close() method. + // Using AtomicBoolean instead of volatile as it may encounter http://findbugs.sourceforge.net/bugDescriptions.html#SP_SPIN_ON_FIELD + // if the field is read but not updated in a spin loop like in #initializeResources() method. + private final AtomicBoolean closing = new AtomicBoolean(false); + private final AtomicBoolean initialized = new AtomicBoolean(false); + private final Time time = Time.SYSTEM; + + private Thread initializationThread; + private volatile ProducerManager producerManager; + private volatile ConsumerManager consumerManager; + + // This allows to gracefully close this instance using {@link #close()} method while there are some pending or new + // requests calling different methods which use the resources like producer/consumer managers. + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + + private final RemotePartitionMetadataStore remotePartitionMetadataStore = new RemotePartitionMetadataStore(); + private volatile TopicBasedRemoteLogMetadataManagerConfig rlmmConfig; + private volatile RemoteLogMetadataTopicPartitioner rlmmTopicPartitioner; + private volatile Set<TopicIdPartition> pendingAssignPartitions = Collections.synchronizedSet(new HashSet<>()); + + @Override + public void addRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata) + throws RemoteStorageException { + Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null"); + + // This allows gracefully rejecting the requests while closing of this instance is in progress, which triggers + // closing the producer/consumer manager instances. + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + // This method is allowed only to add remote log segment with the initial state(which is RemoteLogSegmentState.COPY_SEGMENT_STARTED) + // but not to update the existing remote log segment metadata. + if (remoteLogSegmentMetadata.state() != RemoteLogSegmentState.COPY_SEGMENT_STARTED) { + throw new IllegalArgumentException( + "Given remoteLogSegmentMetadata should have state as " + RemoteLogSegmentState.COPY_SEGMENT_STARTED + + " but it contains state as: " + remoteLogSegmentMetadata.state()); + } + + // Publish the message to the topic. + doPublishMetadata(remoteLogSegmentMetadata.remoteLogSegmentId().topicIdPartition(), + remoteLogSegmentMetadata); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate segmentMetadataUpdate) + throws RemoteStorageException { + Objects.requireNonNull(segmentMetadataUpdate, "segmentMetadataUpdate can not be null"); + + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + // Callers should use addRemoteLogSegmentMetadata to add RemoteLogSegmentMetadata with state as + // RemoteLogSegmentState.COPY_SEGMENT_STARTED. + if (segmentMetadataUpdate.state() == RemoteLogSegmentState.COPY_SEGMENT_STARTED) { + throw new IllegalArgumentException("Given remoteLogSegmentMetadata should not have the state as: " + + RemoteLogSegmentState.COPY_SEGMENT_STARTED); + } + + // Publish the message to the topic. + doPublishMetadata(segmentMetadataUpdate.remoteLogSegmentId().topicIdPartition(), segmentMetadataUpdate); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public void putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata) + throws RemoteStorageException { + Objects.requireNonNull(remotePartitionDeleteMetadata, "remotePartitionDeleteMetadata can not be null"); + + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + doPublishMetadata(remotePartitionDeleteMetadata.topicIdPartition(), remotePartitionDeleteMetadata); + } finally { + lock.readLock().unlock(); + } + } + + private void doPublishMetadata(TopicIdPartition topicIdPartition, RemoteLogMetadata remoteLogMetadata) + throws RemoteStorageException { + log.debug("Publishing metadata for partition: [{}] with context: [{}]", topicIdPartition, remoteLogMetadata); + + try { + // Publish the message to the topic. + RecordMetadata recordMetadata = producerManager.publishMessage(remoteLogMetadata); + // Wait until the consumer catches up with this offset. This will ensure read-after-write consistency + // semantics. + consumerManager.waitTillConsumptionCatchesUp(recordMetadata); + } catch (KafkaException e) { + if (e instanceof RetriableException) { + throw e; + } else { + throw new RemoteStorageException(e); + } + } + } + + @Override + public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(TopicIdPartition topicIdPartition, + int epochForOffset, + long offset) + throws RemoteStorageException { + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + return remotePartitionMetadataStore.remoteLogSegmentMetadata(topicIdPartition, offset, epochForOffset); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public Optional<Long> highestOffsetForEpoch(TopicIdPartition topicIdPartition, + int leaderEpoch) + throws RemoteStorageException { + lock.readLock().lock(); + try { + + ensureInitializedAndNotClosed(); + + return remotePartitionMetadataStore.highestLogOffset(topicIdPartition, leaderEpoch); + } finally { + lock.readLock().unlock(); + } + + } + + @Override + public Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicIdPartition topicIdPartition) + throws RemoteStorageException { + Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null"); + + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + return remotePartitionMetadataStore.listRemoteLogSegments(topicIdPartition); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicIdPartition topicIdPartition, int leaderEpoch) + throws RemoteStorageException { + Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null"); + + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + return remotePartitionMetadataStore.listRemoteLogSegments(topicIdPartition, leaderEpoch); + } finally { + lock.readLock().unlock(); + } + } + + public int metadataPartition(TopicIdPartition topicIdPartition) { + return rlmmTopicPartitioner.metadataPartition(topicIdPartition); + } + + // Visible For Testing + public Optional<Long> receivedOffsetForPartition(int metadataPartition) { + return consumerManager.receivedOffsetForPartition(metadataPartition); + } + + @Override + public void onPartitionLeadershipChanges(Set<TopicIdPartition> leaderPartitions, + Set<TopicIdPartition> followerPartitions) { + Objects.requireNonNull(leaderPartitions, "leaderPartitions can not be null"); + Objects.requireNonNull(followerPartitions, "followerPartitions can not be null"); + + log.info("Received leadership notifications with leader partitions {} and follower partitions {}", + leaderPartitions, followerPartitions); + + HashSet<TopicIdPartition> allPartitions = new HashSet<>(leaderPartitions); + allPartitions.addAll(followerPartitions); + lock.readLock().lock(); + try { + if (closing.get()) { + throw new IllegalStateException("This instance is in closing state"); + } + + if (!initialized.get()) { + // If it is not yet initialized, then keep them as pending partitions and assign them + // when it is initialized successfully in initializeResources(). + this.pendingAssignPartitions.addAll(allPartitions); + } else { + this.pendingAssignPartitions.clear(); Review comment: Hmm, why is this needed since initializeResources() does this already? ########## File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManager.java ########## @@ -0,0 +1,432 @@ +/* + * 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.admin.AdminClient; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.config.TopicConfig; +import org.apache.kafka.common.errors.RetriableException; +import org.apache.kafka.common.errors.TopicExistsException; +import org.apache.kafka.common.utils.KafkaThread; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.server.log.remote.storage.RemoteLogMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteLogMetadataManager; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState; +import org.apache.kafka.server.log.remote.storage.RemotePartitionDeleteMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteStorageException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.time.Duration; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * This is the {@link RemoteLogMetadataManager} implementation with storage as an internal topic with name {@link TopicBasedRemoteLogMetadataManagerConfig#REMOTE_LOG_METADATA_TOPIC_NAME}. + * This is used to publish and fetch {@link RemoteLogMetadata} for the registered user topic partitions with + * {@link #onPartitionLeadershipChanges(Set, Set)}. Each broker will have an instance of this class and it subscribes + * to metadata updates for the registered user topic partitions. + */ +public class TopicBasedRemoteLogMetadataManager implements RemoteLogMetadataManager { + private static final Logger log = LoggerFactory.getLogger(TopicBasedRemoteLogMetadataManager.class); + private static final long INITIALIZATION_RETRY_INTERVAL_MS = 30000L; + + private volatile boolean configured = false; + + // It indicates whether the close process of this instance is started or not via #close() method. + // Using AtomicBoolean instead of volatile as it may encounter http://findbugs.sourceforge.net/bugDescriptions.html#SP_SPIN_ON_FIELD + // if the field is read but not updated in a spin loop like in #initializeResources() method. + private final AtomicBoolean closing = new AtomicBoolean(false); + private final AtomicBoolean initialized = new AtomicBoolean(false); + private final Time time = Time.SYSTEM; + + private Thread initializationThread; + private volatile ProducerManager producerManager; + private volatile ConsumerManager consumerManager; + + // This allows to gracefully close this instance using {@link #close()} method while there are some pending or new + // requests calling different methods which use the resources like producer/consumer managers. + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + + private final RemotePartitionMetadataStore remotePartitionMetadataStore = new RemotePartitionMetadataStore(); + private volatile TopicBasedRemoteLogMetadataManagerConfig rlmmConfig; + private volatile RemoteLogMetadataTopicPartitioner rlmmTopicPartitioner; + private volatile Set<TopicIdPartition> pendingAssignPartitions = Collections.synchronizedSet(new HashSet<>()); + + @Override + public void addRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata) + throws RemoteStorageException { + Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null"); + + // This allows gracefully rejecting the requests while closing of this instance is in progress, which triggers + // closing the producer/consumer manager instances. + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + // This method is allowed only to add remote log segment with the initial state(which is RemoteLogSegmentState.COPY_SEGMENT_STARTED) + // but not to update the existing remote log segment metadata. + if (remoteLogSegmentMetadata.state() != RemoteLogSegmentState.COPY_SEGMENT_STARTED) { + throw new IllegalArgumentException( + "Given remoteLogSegmentMetadata should have state as " + RemoteLogSegmentState.COPY_SEGMENT_STARTED + + " but it contains state as: " + remoteLogSegmentMetadata.state()); + } + + // Publish the message to the topic. + doPublishMetadata(remoteLogSegmentMetadata.remoteLogSegmentId().topicIdPartition(), + remoteLogSegmentMetadata); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate segmentMetadataUpdate) + throws RemoteStorageException { + Objects.requireNonNull(segmentMetadataUpdate, "segmentMetadataUpdate can not be null"); + + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + // Callers should use addRemoteLogSegmentMetadata to add RemoteLogSegmentMetadata with state as + // RemoteLogSegmentState.COPY_SEGMENT_STARTED. + if (segmentMetadataUpdate.state() == RemoteLogSegmentState.COPY_SEGMENT_STARTED) { + throw new IllegalArgumentException("Given remoteLogSegmentMetadata should not have the state as: " + + RemoteLogSegmentState.COPY_SEGMENT_STARTED); + } + + // Publish the message to the topic. + doPublishMetadata(segmentMetadataUpdate.remoteLogSegmentId().topicIdPartition(), segmentMetadataUpdate); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public void putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata) + throws RemoteStorageException { + Objects.requireNonNull(remotePartitionDeleteMetadata, "remotePartitionDeleteMetadata can not be null"); + + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + doPublishMetadata(remotePartitionDeleteMetadata.topicIdPartition(), remotePartitionDeleteMetadata); + } finally { + lock.readLock().unlock(); + } + } + + private void doPublishMetadata(TopicIdPartition topicIdPartition, RemoteLogMetadata remoteLogMetadata) + throws RemoteStorageException { + log.debug("Publishing metadata for partition: [{}] with context: [{}]", topicIdPartition, remoteLogMetadata); + + try { + // Publish the message to the topic. + RecordMetadata recordMetadata = producerManager.publishMessage(remoteLogMetadata); + // Wait until the consumer catches up with this offset. This will ensure read-after-write consistency + // semantics. + consumerManager.waitTillConsumptionCatchesUp(recordMetadata); + } catch (KafkaException e) { + if (e instanceof RetriableException) { + throw e; + } else { + throw new RemoteStorageException(e); + } + } + } + + @Override + public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(TopicIdPartition topicIdPartition, + int epochForOffset, + long offset) + throws RemoteStorageException { + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + return remotePartitionMetadataStore.remoteLogSegmentMetadata(topicIdPartition, offset, epochForOffset); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public Optional<Long> highestOffsetForEpoch(TopicIdPartition topicIdPartition, + int leaderEpoch) + throws RemoteStorageException { + lock.readLock().lock(); + try { + + ensureInitializedAndNotClosed(); + + return remotePartitionMetadataStore.highestLogOffset(topicIdPartition, leaderEpoch); + } finally { + lock.readLock().unlock(); + } + + } + + @Override + public Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicIdPartition topicIdPartition) + throws RemoteStorageException { + Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null"); + + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + return remotePartitionMetadataStore.listRemoteLogSegments(topicIdPartition); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicIdPartition topicIdPartition, int leaderEpoch) + throws RemoteStorageException { + Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null"); + + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + return remotePartitionMetadataStore.listRemoteLogSegments(topicIdPartition, leaderEpoch); + } finally { + lock.readLock().unlock(); + } + } + + public int metadataPartition(TopicIdPartition topicIdPartition) { + return rlmmTopicPartitioner.metadataPartition(topicIdPartition); + } + + // Visible For Testing + public Optional<Long> receivedOffsetForPartition(int metadataPartition) { + return consumerManager.receivedOffsetForPartition(metadataPartition); + } + + @Override + public void onPartitionLeadershipChanges(Set<TopicIdPartition> leaderPartitions, + Set<TopicIdPartition> followerPartitions) { + Objects.requireNonNull(leaderPartitions, "leaderPartitions can not be null"); + Objects.requireNonNull(followerPartitions, "followerPartitions can not be null"); + + log.info("Received leadership notifications with leader partitions {} and follower partitions {}", + leaderPartitions, followerPartitions); + + HashSet<TopicIdPartition> allPartitions = new HashSet<>(leaderPartitions); + allPartitions.addAll(followerPartitions); + lock.readLock().lock(); + try { + if (closing.get()) { + throw new IllegalStateException("This instance is in closing state"); + } + + if (!initialized.get()) { + // If it is not yet initialized, then keep them as pending partitions and assign them + // when it is initialized successfully in initializeResources(). + this.pendingAssignPartitions.addAll(allPartitions); + } else { + this.pendingAssignPartitions.clear(); + consumerManager.addAssignmentsForPartitions(allPartitions); + } + } finally { + lock.readLock().unlock(); + } + } + + @Override + public void onStopPartitions(Set<TopicIdPartition> partitions) { + lock.readLock().lock(); + try { + if (closing.get()) { + throw new IllegalStateException("This instance is in closing state"); + } + + if (!initialized.get()) { + // If it is not yet initialized, then remove them from the pending partitions if any. + if (!pendingAssignPartitions.isEmpty()) { + pendingAssignPartitions.removeAll(partitions); + } + } else { + consumerManager.removeAssignmentsForPartitions(partitions); + } + } finally { + lock.readLock().unlock(); + } + } + + @Override + public void configure(Map<String, ?> configs) { + Objects.requireNonNull(configs, "configs can not be null."); + + lock.writeLock().lock(); + try { + if (configured) { + log.info("Skipping configure as it is already configured."); + return; + } + + log.info("Started initializing with configs: {}", configs); + + rlmmConfig = new TopicBasedRemoteLogMetadataManagerConfig(configs); + rlmmTopicPartitioner = new RemoteLogMetadataTopicPartitioner(rlmmConfig.metadataTopicPartitionsCount()); + configured = true; + log.info("Successfully initialized with rlmmConfig: {}", rlmmConfig); + + // Scheduling the initialization producer/consumer managers in a separate thread. Required resources may + // not yet be available now. This thread makes sure that it is retried at regular intervals until it is + // successful. + initializationThread = KafkaThread.nonDaemon("RLMMInitializationThread", () -> initializeResources()); + initializationThread.start(); + } finally { + lock.writeLock().unlock(); + } + } + + private void initializeResources() { + log.info("Initializing the resources."); + final NewTopic remoteLogMetadataTopicRequest = createRemoteLogMetadataTopicRequest(); + + // Stop if it is already initialized or closing. + while (!(initialized.get() || closing.get())) { + // There were dead locks observed when the remote log metadata topic created as part of on internal + // topic creation(with auto create enabled) when multiple brokers were getting started and RLMM creating + // the respective producer and consumer instances. + if (!createTopic(remoteLogMetadataTopicRequest)) { + // try to create the topic again if it could not be created. + log.info("Sleep for : {} ms before it is retried again.", INITIALIZATION_RETRY_INTERVAL_MS); + Utils.sleep(INITIALIZATION_RETRY_INTERVAL_MS); + } else { + // Create producer and consumer managers. + lock.writeLock().lock(); + try { + producerManager = new ProducerManager(rlmmConfig, rlmmTopicPartitioner); + consumerManager = new ConsumerManager(rlmmConfig, remotePartitionMetadataStore, rlmmTopicPartitioner, time); + consumerManager.startConsumerThread(); + + if (!pendingAssignPartitions.isEmpty()) { + consumerManager.addAssignmentsForPartitions(pendingAssignPartitions); + pendingAssignPartitions.clear(); + } + + initialized.set(true); + log.info("Initialized resources successfully."); + } catch (Exception e) { + log.error("Encountered error while initializing producer/consumer", e); + return; + } finally { + lock.writeLock().unlock(); + } + } + } + } + + private NewTopic createRemoteLogMetadataTopicRequest() { + Map<String, String> topicConfigs = new HashMap<>(); + topicConfigs.put(TopicConfig.RETENTION_MS_CONFIG, Long.toString(rlmmConfig.metadataTopicRetentionMs())); + topicConfigs.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE); + return new NewTopic(rlmmConfig.remoteLogMetadataTopicName(), + rlmmConfig.metadataTopicPartitionsCount(), + rlmmConfig.metadataTopicReplicationFactor()).configs(topicConfigs); + } + + /** + * @param topic topic to be created. + * @return Returns true if the topic already exists or it is created successfully. + */ + private boolean createTopic(NewTopic topic) { + boolean topicCreated = false; + AdminClient adminClient = null; + try { + adminClient = AdminClient.create(rlmmConfig.consumerProperties()); + adminClient.createTopics(Collections.singleton(topic)).all().get(); + topicCreated = true; + } catch (Exception e) { + if (e.getCause() instanceof TopicExistsException) { + log.info("Topic [{}] already exists", topic.name()); + topicCreated = true; Review comment: Should we verify that the number of partitions in the existing topic matches the configuration? ########## File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManager.java ########## @@ -0,0 +1,432 @@ +/* + * 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.admin.AdminClient; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.config.TopicConfig; +import org.apache.kafka.common.errors.RetriableException; +import org.apache.kafka.common.errors.TopicExistsException; +import org.apache.kafka.common.utils.KafkaThread; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.server.log.remote.storage.RemoteLogMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteLogMetadataManager; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState; +import org.apache.kafka.server.log.remote.storage.RemotePartitionDeleteMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteStorageException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.time.Duration; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * This is the {@link RemoteLogMetadataManager} implementation with storage as an internal topic with name {@link TopicBasedRemoteLogMetadataManagerConfig#REMOTE_LOG_METADATA_TOPIC_NAME}. + * This is used to publish and fetch {@link RemoteLogMetadata} for the registered user topic partitions with + * {@link #onPartitionLeadershipChanges(Set, Set)}. Each broker will have an instance of this class and it subscribes + * to metadata updates for the registered user topic partitions. + */ +public class TopicBasedRemoteLogMetadataManager implements RemoteLogMetadataManager { + private static final Logger log = LoggerFactory.getLogger(TopicBasedRemoteLogMetadataManager.class); + private static final long INITIALIZATION_RETRY_INTERVAL_MS = 30000L; + + private volatile boolean configured = false; + + // It indicates whether the close process of this instance is started or not via #close() method. + // Using AtomicBoolean instead of volatile as it may encounter http://findbugs.sourceforge.net/bugDescriptions.html#SP_SPIN_ON_FIELD + // if the field is read but not updated in a spin loop like in #initializeResources() method. + private final AtomicBoolean closing = new AtomicBoolean(false); + private final AtomicBoolean initialized = new AtomicBoolean(false); + private final Time time = Time.SYSTEM; + + private Thread initializationThread; + private volatile ProducerManager producerManager; + private volatile ConsumerManager consumerManager; + + // This allows to gracefully close this instance using {@link #close()} method while there are some pending or new + // requests calling different methods which use the resources like producer/consumer managers. + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + + private final RemotePartitionMetadataStore remotePartitionMetadataStore = new RemotePartitionMetadataStore(); + private volatile TopicBasedRemoteLogMetadataManagerConfig rlmmConfig; + private volatile RemoteLogMetadataTopicPartitioner rlmmTopicPartitioner; + private volatile Set<TopicIdPartition> pendingAssignPartitions = Collections.synchronizedSet(new HashSet<>()); + + @Override + public void addRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata) + throws RemoteStorageException { + Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null"); + + // This allows gracefully rejecting the requests while closing of this instance is in progress, which triggers + // closing the producer/consumer manager instances. + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + // This method is allowed only to add remote log segment with the initial state(which is RemoteLogSegmentState.COPY_SEGMENT_STARTED) + // but not to update the existing remote log segment metadata. + if (remoteLogSegmentMetadata.state() != RemoteLogSegmentState.COPY_SEGMENT_STARTED) { + throw new IllegalArgumentException( + "Given remoteLogSegmentMetadata should have state as " + RemoteLogSegmentState.COPY_SEGMENT_STARTED + + " but it contains state as: " + remoteLogSegmentMetadata.state()); + } + + // Publish the message to the topic. + doPublishMetadata(remoteLogSegmentMetadata.remoteLogSegmentId().topicIdPartition(), + remoteLogSegmentMetadata); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate segmentMetadataUpdate) + throws RemoteStorageException { + Objects.requireNonNull(segmentMetadataUpdate, "segmentMetadataUpdate can not be null"); + + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + // Callers should use addRemoteLogSegmentMetadata to add RemoteLogSegmentMetadata with state as + // RemoteLogSegmentState.COPY_SEGMENT_STARTED. + if (segmentMetadataUpdate.state() == RemoteLogSegmentState.COPY_SEGMENT_STARTED) { + throw new IllegalArgumentException("Given remoteLogSegmentMetadata should not have the state as: " + + RemoteLogSegmentState.COPY_SEGMENT_STARTED); + } + + // Publish the message to the topic. + doPublishMetadata(segmentMetadataUpdate.remoteLogSegmentId().topicIdPartition(), segmentMetadataUpdate); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public void putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata) + throws RemoteStorageException { + Objects.requireNonNull(remotePartitionDeleteMetadata, "remotePartitionDeleteMetadata can not be null"); + + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + doPublishMetadata(remotePartitionDeleteMetadata.topicIdPartition(), remotePartitionDeleteMetadata); + } finally { + lock.readLock().unlock(); + } + } + + private void doPublishMetadata(TopicIdPartition topicIdPartition, RemoteLogMetadata remoteLogMetadata) + throws RemoteStorageException { + log.debug("Publishing metadata for partition: [{}] with context: [{}]", topicIdPartition, remoteLogMetadata); + + try { + // Publish the message to the topic. + RecordMetadata recordMetadata = producerManager.publishMessage(remoteLogMetadata); + // Wait until the consumer catches up with this offset. This will ensure read-after-write consistency + // semantics. + consumerManager.waitTillConsumptionCatchesUp(recordMetadata); + } catch (KafkaException e) { + if (e instanceof RetriableException) { + throw e; + } else { + throw new RemoteStorageException(e); + } + } + } + + @Override + public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(TopicIdPartition topicIdPartition, + int epochForOffset, + long offset) + throws RemoteStorageException { + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + return remotePartitionMetadataStore.remoteLogSegmentMetadata(topicIdPartition, offset, epochForOffset); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public Optional<Long> highestOffsetForEpoch(TopicIdPartition topicIdPartition, + int leaderEpoch) + throws RemoteStorageException { + lock.readLock().lock(); + try { + + ensureInitializedAndNotClosed(); + + return remotePartitionMetadataStore.highestLogOffset(topicIdPartition, leaderEpoch); + } finally { + lock.readLock().unlock(); + } + + } + + @Override + public Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicIdPartition topicIdPartition) + throws RemoteStorageException { + Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null"); + + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + return remotePartitionMetadataStore.listRemoteLogSegments(topicIdPartition); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicIdPartition topicIdPartition, int leaderEpoch) + throws RemoteStorageException { + Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null"); + + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + return remotePartitionMetadataStore.listRemoteLogSegments(topicIdPartition, leaderEpoch); + } finally { + lock.readLock().unlock(); + } + } + + public int metadataPartition(TopicIdPartition topicIdPartition) { + return rlmmTopicPartitioner.metadataPartition(topicIdPartition); + } + + // Visible For Testing + public Optional<Long> receivedOffsetForPartition(int metadataPartition) { + return consumerManager.receivedOffsetForPartition(metadataPartition); + } + + @Override + public void onPartitionLeadershipChanges(Set<TopicIdPartition> leaderPartitions, + Set<TopicIdPartition> followerPartitions) { + Objects.requireNonNull(leaderPartitions, "leaderPartitions can not be null"); + Objects.requireNonNull(followerPartitions, "followerPartitions can not be null"); + + log.info("Received leadership notifications with leader partitions {} and follower partitions {}", + leaderPartitions, followerPartitions); + + HashSet<TopicIdPartition> allPartitions = new HashSet<>(leaderPartitions); + allPartitions.addAll(followerPartitions); + lock.readLock().lock(); + try { + if (closing.get()) { + throw new IllegalStateException("This instance is in closing state"); + } + + if (!initialized.get()) { + // If it is not yet initialized, then keep them as pending partitions and assign them + // when it is initialized successfully in initializeResources(). + this.pendingAssignPartitions.addAll(allPartitions); + } else { + this.pendingAssignPartitions.clear(); + consumerManager.addAssignmentsForPartitions(allPartitions); + } + } finally { + lock.readLock().unlock(); + } + } + + @Override + public void onStopPartitions(Set<TopicIdPartition> partitions) { + lock.readLock().lock(); + try { + if (closing.get()) { + throw new IllegalStateException("This instance is in closing state"); + } + + if (!initialized.get()) { + // If it is not yet initialized, then remove them from the pending partitions if any. + if (!pendingAssignPartitions.isEmpty()) { + pendingAssignPartitions.removeAll(partitions); + } + } else { + consumerManager.removeAssignmentsForPartitions(partitions); + } + } finally { + lock.readLock().unlock(); + } + } + + @Override + public void configure(Map<String, ?> configs) { + Objects.requireNonNull(configs, "configs can not be null."); + + lock.writeLock().lock(); + try { + if (configured) { + log.info("Skipping configure as it is already configured."); + return; + } + + log.info("Started initializing with configs: {}", configs); + + rlmmConfig = new TopicBasedRemoteLogMetadataManagerConfig(configs); + rlmmTopicPartitioner = new RemoteLogMetadataTopicPartitioner(rlmmConfig.metadataTopicPartitionsCount()); + configured = true; + log.info("Successfully initialized with rlmmConfig: {}", rlmmConfig); + + // Scheduling the initialization producer/consumer managers in a separate thread. Required resources may + // not yet be available now. This thread makes sure that it is retried at regular intervals until it is + // successful. + initializationThread = KafkaThread.nonDaemon("RLMMInitializationThread", () -> initializeResources()); + initializationThread.start(); + } finally { + lock.writeLock().unlock(); + } + } + + private void initializeResources() { + log.info("Initializing the resources."); + final NewTopic remoteLogMetadataTopicRequest = createRemoteLogMetadataTopicRequest(); + + // Stop if it is already initialized or closing. + while (!(initialized.get() || closing.get())) { + // There were dead locks observed when the remote log metadata topic created as part of on internal + // topic creation(with auto create enabled) when multiple brokers were getting started and RLMM creating + // the respective producer and consumer instances. + if (!createTopic(remoteLogMetadataTopicRequest)) { + // try to create the topic again if it could not be created. + log.info("Sleep for : {} ms before it is retried again.", INITIALIZATION_RETRY_INTERVAL_MS); + Utils.sleep(INITIALIZATION_RETRY_INTERVAL_MS); Review comment: Hmm, we don't want to retry forever. If there is a configuration error, we want to fail fast. ########## File path: storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerHarness.java ########## @@ -0,0 +1,123 @@ +/* + * 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 kafka.api.IntegrationTestHarness; +import kafka.utils.TestUtils; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.config.TopicConfig; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.TimeoutException; + +import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_PARTITIONS_PROP; +import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_REPLICATION_FACTOR_PROP; +import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_RETENTION_MILLIS_PROP; + +public class TopicBasedRemoteLogMetadataManagerHarness extends IntegrationTestHarness { + private static final Logger log = LoggerFactory.getLogger(TopicBasedRemoteLogMetadataManagerHarness.class); + + protected static final int METADATA_TOPIC_PARTITIONS_COUNT = 3; + protected static final int METADATA_TOPIC_REPLICATION_FACTOR = 2; + protected static final long METADATA_TOPIC_RETENTION_MS = 24 * 60 * 60 * 1000L; + + private final Time time = new MockTime(1); + private TopicBasedRemoteLogMetadataManager topicBasedRemoteLogMetadataManager; + + protected Map<String, Object> overrideRemoteLogMetadataManagerProps() { + return Collections.emptyMap(); + } + + public void initialize(Set<TopicIdPartition> topicIdPartitions) { + // Call setup to start the cluster. + super.setUp(); + + // Make sure the remote log metadata topic is created before it is used. + createMetadataTopic(); + + topicBasedRemoteLogMetadataManager = new TopicBasedRemoteLogMetadataManager(time); + + // Initialize TopicBasedRemoteLogMetadataManager. + Map<String, Object> configs = new HashMap<>(); + configs.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerList()); + configs.put("broker.id", 0); + configs.put(REMOTE_LOG_METADATA_TOPIC_PARTITIONS_PROP, METADATA_TOPIC_PARTITIONS_COUNT); + configs.put(REMOTE_LOG_METADATA_TOPIC_REPLICATION_FACTOR_PROP, METADATA_TOPIC_REPLICATION_FACTOR); + configs.put(REMOTE_LOG_METADATA_TOPIC_RETENTION_MILLIS_PROP, METADATA_TOPIC_RETENTION_MS); + + log.debug("TopicBasedRemoteLogMetadataManager configs before adding overridden properties: {}", configs); + // Add override properties. + configs.putAll(overrideRemoteLogMetadataManagerProps()); + log.debug("TopicBasedRemoteLogMetadataManager configs after adding overridden properties: {}", configs); + + topicBasedRemoteLogMetadataManager.configure(configs); + try { + waitUntilInitialized(120_000); + } catch (TimeoutException e) { + throw new RuntimeException(e); + } + + topicBasedRemoteLogMetadataManager.onPartitionLeadershipChanges(topicIdPartitions, Collections.emptySet()); + } + + // Visible for testing. + public void waitUntilInitialized(long waitTimeMs) throws TimeoutException { + long startMs = System.currentTimeMillis(); + while (!topicBasedRemoteLogMetadataManager.isInitialized()) { + long currentTimeMs = System.currentTimeMillis(); + if (currentTimeMs > startMs + waitTimeMs) { + throw new TimeoutException("Time out reached before it is initialized successfully"); + } + + Utils.sleep(1000); + } + } + + @Override + public int brokerCount() { + return 3; + } + + protected TopicBasedRemoteLogMetadataManager topicBasedRlmm() { + return topicBasedRemoteLogMetadataManager; + } + + private void createMetadataTopic() { Review comment: createMetadataTopic() is no longer used. ########## File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManager.java ########## @@ -0,0 +1,432 @@ +/* + * 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.admin.AdminClient; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.config.TopicConfig; +import org.apache.kafka.common.errors.RetriableException; +import org.apache.kafka.common.errors.TopicExistsException; +import org.apache.kafka.common.utils.KafkaThread; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.server.log.remote.storage.RemoteLogMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteLogMetadataManager; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState; +import org.apache.kafka.server.log.remote.storage.RemotePartitionDeleteMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteStorageException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.time.Duration; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * This is the {@link RemoteLogMetadataManager} implementation with storage as an internal topic with name {@link TopicBasedRemoteLogMetadataManagerConfig#REMOTE_LOG_METADATA_TOPIC_NAME}. + * This is used to publish and fetch {@link RemoteLogMetadata} for the registered user topic partitions with + * {@link #onPartitionLeadershipChanges(Set, Set)}. Each broker will have an instance of this class and it subscribes + * to metadata updates for the registered user topic partitions. + */ +public class TopicBasedRemoteLogMetadataManager implements RemoteLogMetadataManager { + private static final Logger log = LoggerFactory.getLogger(TopicBasedRemoteLogMetadataManager.class); + private static final long INITIALIZATION_RETRY_INTERVAL_MS = 30000L; + + private volatile boolean configured = false; + + // It indicates whether the close process of this instance is started or not via #close() method. + // Using AtomicBoolean instead of volatile as it may encounter http://findbugs.sourceforge.net/bugDescriptions.html#SP_SPIN_ON_FIELD + // if the field is read but not updated in a spin loop like in #initializeResources() method. + private final AtomicBoolean closing = new AtomicBoolean(false); + private final AtomicBoolean initialized = new AtomicBoolean(false); + private final Time time = Time.SYSTEM; + + private Thread initializationThread; + private volatile ProducerManager producerManager; + private volatile ConsumerManager consumerManager; + + // This allows to gracefully close this instance using {@link #close()} method while there are some pending or new + // requests calling different methods which use the resources like producer/consumer managers. + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + + private final RemotePartitionMetadataStore remotePartitionMetadataStore = new RemotePartitionMetadataStore(); + private volatile TopicBasedRemoteLogMetadataManagerConfig rlmmConfig; + private volatile RemoteLogMetadataTopicPartitioner rlmmTopicPartitioner; + private volatile Set<TopicIdPartition> pendingAssignPartitions = Collections.synchronizedSet(new HashSet<>()); + + @Override + public void addRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata) + throws RemoteStorageException { + Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null"); + + // This allows gracefully rejecting the requests while closing of this instance is in progress, which triggers + // closing the producer/consumer manager instances. + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + // This method is allowed only to add remote log segment with the initial state(which is RemoteLogSegmentState.COPY_SEGMENT_STARTED) + // but not to update the existing remote log segment metadata. + if (remoteLogSegmentMetadata.state() != RemoteLogSegmentState.COPY_SEGMENT_STARTED) { + throw new IllegalArgumentException( + "Given remoteLogSegmentMetadata should have state as " + RemoteLogSegmentState.COPY_SEGMENT_STARTED + + " but it contains state as: " + remoteLogSegmentMetadata.state()); + } + + // Publish the message to the topic. + doPublishMetadata(remoteLogSegmentMetadata.remoteLogSegmentId().topicIdPartition(), + remoteLogSegmentMetadata); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate segmentMetadataUpdate) + throws RemoteStorageException { + Objects.requireNonNull(segmentMetadataUpdate, "segmentMetadataUpdate can not be null"); + + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + // Callers should use addRemoteLogSegmentMetadata to add RemoteLogSegmentMetadata with state as + // RemoteLogSegmentState.COPY_SEGMENT_STARTED. + if (segmentMetadataUpdate.state() == RemoteLogSegmentState.COPY_SEGMENT_STARTED) { + throw new IllegalArgumentException("Given remoteLogSegmentMetadata should not have the state as: " + + RemoteLogSegmentState.COPY_SEGMENT_STARTED); + } + + // Publish the message to the topic. + doPublishMetadata(segmentMetadataUpdate.remoteLogSegmentId().topicIdPartition(), segmentMetadataUpdate); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public void putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata) + throws RemoteStorageException { + Objects.requireNonNull(remotePartitionDeleteMetadata, "remotePartitionDeleteMetadata can not be null"); + + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + doPublishMetadata(remotePartitionDeleteMetadata.topicIdPartition(), remotePartitionDeleteMetadata); + } finally { + lock.readLock().unlock(); + } + } + + private void doPublishMetadata(TopicIdPartition topicIdPartition, RemoteLogMetadata remoteLogMetadata) + throws RemoteStorageException { + log.debug("Publishing metadata for partition: [{}] with context: [{}]", topicIdPartition, remoteLogMetadata); + + try { + // Publish the message to the topic. + RecordMetadata recordMetadata = producerManager.publishMessage(remoteLogMetadata); + // Wait until the consumer catches up with this offset. This will ensure read-after-write consistency + // semantics. + consumerManager.waitTillConsumptionCatchesUp(recordMetadata); + } catch (KafkaException e) { + if (e instanceof RetriableException) { + throw e; + } else { + throw new RemoteStorageException(e); + } + } + } + + @Override + public Optional<RemoteLogSegmentMetadata> remoteLogSegmentMetadata(TopicIdPartition topicIdPartition, + int epochForOffset, + long offset) + throws RemoteStorageException { + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + return remotePartitionMetadataStore.remoteLogSegmentMetadata(topicIdPartition, offset, epochForOffset); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public Optional<Long> highestOffsetForEpoch(TopicIdPartition topicIdPartition, + int leaderEpoch) + throws RemoteStorageException { + lock.readLock().lock(); + try { + + ensureInitializedAndNotClosed(); + + return remotePartitionMetadataStore.highestLogOffset(topicIdPartition, leaderEpoch); + } finally { + lock.readLock().unlock(); + } + + } + + @Override + public Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicIdPartition topicIdPartition) + throws RemoteStorageException { + Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null"); + + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + return remotePartitionMetadataStore.listRemoteLogSegments(topicIdPartition); + } finally { + lock.readLock().unlock(); + } + } + + @Override + public Iterator<RemoteLogSegmentMetadata> listRemoteLogSegments(TopicIdPartition topicIdPartition, int leaderEpoch) + throws RemoteStorageException { + Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null"); + + lock.readLock().lock(); + try { + ensureInitializedAndNotClosed(); + + return remotePartitionMetadataStore.listRemoteLogSegments(topicIdPartition, leaderEpoch); + } finally { + lock.readLock().unlock(); + } + } + + public int metadataPartition(TopicIdPartition topicIdPartition) { + return rlmmTopicPartitioner.metadataPartition(topicIdPartition); + } + + // Visible For Testing + public Optional<Long> receivedOffsetForPartition(int metadataPartition) { + return consumerManager.receivedOffsetForPartition(metadataPartition); + } + + @Override + public void onPartitionLeadershipChanges(Set<TopicIdPartition> leaderPartitions, + Set<TopicIdPartition> followerPartitions) { + Objects.requireNonNull(leaderPartitions, "leaderPartitions can not be null"); + Objects.requireNonNull(followerPartitions, "followerPartitions can not be null"); + + log.info("Received leadership notifications with leader partitions {} and follower partitions {}", + leaderPartitions, followerPartitions); + + HashSet<TopicIdPartition> allPartitions = new HashSet<>(leaderPartitions); + allPartitions.addAll(followerPartitions); + lock.readLock().lock(); + try { + if (closing.get()) { + throw new IllegalStateException("This instance is in closing state"); + } + + if (!initialized.get()) { + // If it is not yet initialized, then keep them as pending partitions and assign them + // when it is initialized successfully in initializeResources(). + this.pendingAssignPartitions.addAll(allPartitions); + } else { + this.pendingAssignPartitions.clear(); + consumerManager.addAssignmentsForPartitions(allPartitions); + } + } finally { + lock.readLock().unlock(); + } + } + + @Override + public void onStopPartitions(Set<TopicIdPartition> partitions) { + lock.readLock().lock(); + try { + if (closing.get()) { + throw new IllegalStateException("This instance is in closing state"); + } + + if (!initialized.get()) { + // If it is not yet initialized, then remove them from the pending partitions if any. + if (!pendingAssignPartitions.isEmpty()) { + pendingAssignPartitions.removeAll(partitions); + } + } else { + consumerManager.removeAssignmentsForPartitions(partitions); + } + } finally { + lock.readLock().unlock(); + } + } + + @Override + public void configure(Map<String, ?> configs) { + Objects.requireNonNull(configs, "configs can not be null."); + + lock.writeLock().lock(); + try { + if (configured) { + log.info("Skipping configure as it is already configured."); + return; + } + + log.info("Started initializing with configs: {}", configs); + + rlmmConfig = new TopicBasedRemoteLogMetadataManagerConfig(configs); + rlmmTopicPartitioner = new RemoteLogMetadataTopicPartitioner(rlmmConfig.metadataTopicPartitionsCount()); + configured = true; + log.info("Successfully initialized with rlmmConfig: {}", rlmmConfig); + + // Scheduling the initialization producer/consumer managers in a separate thread. Required resources may + // not yet be available now. This thread makes sure that it is retried at regular intervals until it is + // successful. + initializationThread = KafkaThread.nonDaemon("RLMMInitializationThread", () -> initializeResources()); + initializationThread.start(); + } finally { + lock.writeLock().unlock(); + } + } + + private void initializeResources() { + log.info("Initializing the resources."); + final NewTopic remoteLogMetadataTopicRequest = createRemoteLogMetadataTopicRequest(); + + // Stop if it is already initialized or closing. + while (!(initialized.get() || closing.get())) { + // There were dead locks observed when the remote log metadata topic created as part of on internal + // topic creation(with auto create enabled) when multiple brokers were getting started and RLMM creating + // the respective producer and consumer instances. + if (!createTopic(remoteLogMetadataTopicRequest)) { + // try to create the topic again if it could not be created. + log.info("Sleep for : {} ms before it is retried again.", INITIALIZATION_RETRY_INTERVAL_MS); + Utils.sleep(INITIALIZATION_RETRY_INTERVAL_MS); + } else { + // Create producer and consumer managers. + lock.writeLock().lock(); + try { + producerManager = new ProducerManager(rlmmConfig, rlmmTopicPartitioner); + consumerManager = new ConsumerManager(rlmmConfig, remotePartitionMetadataStore, rlmmTopicPartitioner, time); + consumerManager.startConsumerThread(); + + if (!pendingAssignPartitions.isEmpty()) { + consumerManager.addAssignmentsForPartitions(pendingAssignPartitions); + pendingAssignPartitions.clear(); + } + + initialized.set(true); + log.info("Initialized resources successfully."); + } catch (Exception e) { + log.error("Encountered error while initializing producer/consumer", e); + return; + } finally { + lock.writeLock().unlock(); + } + } + } + } + + private NewTopic createRemoteLogMetadataTopicRequest() { + Map<String, String> topicConfigs = new HashMap<>(); + topicConfigs.put(TopicConfig.RETENTION_MS_CONFIG, Long.toString(rlmmConfig.metadataTopicRetentionMs())); + topicConfigs.put(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE); + return new NewTopic(rlmmConfig.remoteLogMetadataTopicName(), + rlmmConfig.metadataTopicPartitionsCount(), + rlmmConfig.metadataTopicReplicationFactor()).configs(topicConfigs); + } + + /** + * @param topic topic to be created. + * @return Returns true if the topic already exists or it is created successfully. + */ + private boolean createTopic(NewTopic topic) { + boolean topicCreated = false; + AdminClient adminClient = null; + try { + adminClient = AdminClient.create(rlmmConfig.consumerProperties()); Review comment: It seems producer credentials are closer for the admin client. -- 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: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org