junrao commented on a change in pull request #11033: URL: https://github.com/apache/kafka/pull/11033#discussion_r703798708
########## File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManager.java ########## @@ -129,37 +132,44 @@ public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate segmen } // Publish the message to the topic. - doPublishMetadata(segmentMetadataUpdate.remoteLogSegmentId().topicIdPartition(), segmentMetadataUpdate); + return doPublishMetadata(segmentMetadataUpdate.remoteLogSegmentId().topicIdPartition(), segmentMetadataUpdate); } finally { lock.readLock().unlock(); } } @Override - public void putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata) + public CompletableFuture<Void> putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata) throws RemoteStorageException { Objects.requireNonNull(remotePartitionDeleteMetadata, "remotePartitionDeleteMetadata can not be null"); lock.readLock().lock(); try { ensureInitializedAndNotClosed(); - doPublishMetadata(remotePartitionDeleteMetadata.topicIdPartition(), remotePartitionDeleteMetadata); + return doPublishMetadata(remotePartitionDeleteMetadata.topicIdPartition(), remotePartitionDeleteMetadata); } finally { lock.readLock().unlock(); } } - private void doPublishMetadata(TopicIdPartition topicIdPartition, RemoteLogMetadata remoteLogMetadata) + private CompletableFuture<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); + CompletableFuture<RecordMetadata> produceFuture = new CompletableFuture<>(); + producerManager.publishMessage(remoteLogMetadata, produceFuture); + return produceFuture.thenApplyAsync((Function<RecordMetadata, Void>) recordMetadata -> { + try { + consumerManager.waitTillConsumptionCatchesUp(recordMetadata); + } catch (TimeoutException e) { + throw new KafkaException(e); + } + return null; + }).toCompletableFuture(); Review comment: Is toCompletableFuture() needed since thenApplyAsync() returns CompletableFuture already? ########## File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManager.java ########## @@ -129,37 +132,44 @@ public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate segmen } // Publish the message to the topic. - doPublishMetadata(segmentMetadataUpdate.remoteLogSegmentId().topicIdPartition(), segmentMetadataUpdate); + return doPublishMetadata(segmentMetadataUpdate.remoteLogSegmentId().topicIdPartition(), segmentMetadataUpdate); } finally { lock.readLock().unlock(); } } @Override - public void putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata) + public CompletableFuture<Void> putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata) throws RemoteStorageException { Objects.requireNonNull(remotePartitionDeleteMetadata, "remotePartitionDeleteMetadata can not be null"); lock.readLock().lock(); try { ensureInitializedAndNotClosed(); - doPublishMetadata(remotePartitionDeleteMetadata.topicIdPartition(), remotePartitionDeleteMetadata); + return doPublishMetadata(remotePartitionDeleteMetadata.topicIdPartition(), remotePartitionDeleteMetadata); } finally { lock.readLock().unlock(); } } - private void doPublishMetadata(TopicIdPartition topicIdPartition, RemoteLogMetadata remoteLogMetadata) + private CompletableFuture<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); + CompletableFuture<RecordMetadata> produceFuture = new CompletableFuture<>(); + producerManager.publishMessage(remoteLogMetadata, produceFuture); Review comment: It seems that it's more natural for producerManager to create a CompletableFuture and return it through publishMessage() rather than passing in a CompletableFuture from the caller. ########## File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/FuturePublishRecordMetadata.java ########## @@ -0,0 +1,131 @@ +/* + * 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.producer.RecordMetadata; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.utils.Time; +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.RemotePartitionDeleteMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +/** + * This class represents the {@link Future} implementation for {@link TopicBasedRemoteLogMetadataManager} + * <ul> + * <li> publishing the {@link org.apache.kafka.server.log.remote.storage.RemoteLogMetadata}, and </li> + * <li> consuming these messages from remote log metadata topic for adding them to its internal store, that makes + * available for fetching these remote log metadata with APIs like {@link TopicBasedRemoteLogMetadataManager#remoteLogSegmentMetadata(TopicIdPartition, int, long)} </li> + * </ul> + * <br> + * This class instance is returned with APIs like + * <ul> + * <li>{@link TopicBasedRemoteLogMetadataManager#addRemoteLogSegmentMetadata(RemoteLogSegmentMetadata)}</li> + * <li>{@link TopicBasedRemoteLogMetadataManager#updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate)}</li> + * <li>{@link TopicBasedRemoteLogMetadataManager#putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata)}</li> + * </ul> + */ +class FuturePublishRecordMetadata implements Future<Void> { Review comment: This class seems never used? ########## File path: storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManager.java ########## @@ -129,37 +132,44 @@ public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate segmen } // Publish the message to the topic. - doPublishMetadata(segmentMetadataUpdate.remoteLogSegmentId().topicIdPartition(), segmentMetadataUpdate); + return doPublishMetadata(segmentMetadataUpdate.remoteLogSegmentId().topicIdPartition(), segmentMetadataUpdate); } finally { lock.readLock().unlock(); } } @Override - public void putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata) + public CompletableFuture<Void> putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata) throws RemoteStorageException { Objects.requireNonNull(remotePartitionDeleteMetadata, "remotePartitionDeleteMetadata can not be null"); lock.readLock().lock(); try { ensureInitializedAndNotClosed(); - doPublishMetadata(remotePartitionDeleteMetadata.topicIdPartition(), remotePartitionDeleteMetadata); + return doPublishMetadata(remotePartitionDeleteMetadata.topicIdPartition(), remotePartitionDeleteMetadata); } finally { lock.readLock().unlock(); } } - private void doPublishMetadata(TopicIdPartition topicIdPartition, RemoteLogMetadata remoteLogMetadata) + private CompletableFuture<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); + CompletableFuture<RecordMetadata> produceFuture = new CompletableFuture<>(); + producerManager.publishMessage(remoteLogMetadata, produceFuture); + return produceFuture.thenApplyAsync((Function<RecordMetadata, Void>) recordMetadata -> { + try { + consumerManager.waitTillConsumptionCatchesUp(recordMetadata); Review comment: Hmm, this callback will be called from Producer's Sender thread and consumerManager.waitTillConsumptionCatchesUp() blocks until the timeout. This will block the Sender thread, which is not ideal. -- 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