satishd commented on a change in pull request #10579: URL: https://github.com/apache/kafka/pull/10579#discussion_r671415369
########## File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManager.java ########## @@ -0,0 +1,484 @@ +/* + * 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.admin.TopicDescription; +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.internals.FatalExitError; +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.ExecutionException; +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 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 final Set<TopicIdPartition> pendingAssignPartitions = Collections.synchronizedSet(new HashSet<>()); + private volatile boolean initializationFailed; + + @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); Review comment: That is a good point. We need better semantics here. Earlier, we plan to introduce RetriableException for RLMM and RSM so that callers can have an option to know whether they can retry or not. In the case of initialization is not yet complete, RetriableException can be thrown caller can retry based on backoff. RLMM can send non retriable exception if it is in closing state and there will not be any retries. Another way to handle is to take these events and store them in in-memory queue and return Future. These Futures will be considered successful if initialization is successful and the events are published to the topic. I plan to address these in a followup PR while these APIs are integrated with RLM, filed https://issues.apache.org/jira/browse/KAFKA-13097. -- 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