From 6eca897ee8438d7d0348721d2a88babb06cae2cd Mon Sep 17 00:00:00 2001 From: Sushant Mane Date: Sat, 21 Oct 2023 03:26:45 -0700 Subject: [PATCH 1/8] [controller][server] TopicManager refactoring and performance improvements --- .../davinci/config/VeniceServerConfig.java | 19 + ...lBootstrappingVeniceChangelogConsumer.java | 5 +- .../AbstractKafkaConsumerService.java | 6 +- .../ActiveActiveStoreIngestionTask.java | 8 +- .../consumer/AggKafkaConsumerService.java | 33 +- .../consumer/CachedPubSubMetadataGetter.java | 171 ---- .../kafka/consumer/KafkaConsumerService.java | 4 +- .../KafkaConsumerServiceDelegator.java | 8 +- .../consumer/KafkaStoreIngestionService.java | 31 +- .../LeaderFollowerStoreIngestionTask.java | 35 +- .../kafka/consumer/StoreIngestionTask.java | 41 +- .../consumer/StoreIngestionTaskFactory.java | 2 +- ...tstrappingVeniceChangelogConsumerTest.java | 3 - .../consumer/AggKafkaConsumerServiceTest.java | 109 +++ .../CachedPubSubMetadataGetterTest.java | 64 -- .../KafkaConsumerServiceDelegatorTest.java | 2 +- .../KafkaStoreIngestionServiceTest.java | 2 + .../consumer/StoreIngestionTaskTest.java | 53 +- .../java/com/linkedin/venice/AdminTool.java | 30 +- .../com/linkedin/venice/KafkaTopicDumper.java | 5 +- .../hadoop/input/kafka/KafkaInputFormat.java | 37 +- .../com/linkedin/venice/ConfigConstants.java | 6 - .../java/com/linkedin/venice/ConfigKeys.java | 12 + .../venice/kafka/TopicManagerRepository.java | 184 ----- .../InstrumentedPartitionOffsetFetcher.java | 101 --- .../PartitionOffsetFetcher.java | 36 - .../PartitionOffsetFetcherFactory.java | 46 -- .../PartitionOffsetFetcherImpl.java | 514 ------------ .../PartitionOffsetFetcherStats.java | 55 -- .../com/linkedin/venice/meta/ZKStore.java | 4 +- .../venice/pubsub/PubSubConstants.java | 52 +- .../pubsub/PubSubTopicPartitionInfo.java | 17 +- .../kafka/admin/ApacheKafkaAdminAdapter.java | 4 +- .../consumer/ApacheKafkaConsumerAdapter.java | 22 +- .../consumer/ApacheKafkaConsumerConfig.java | 8 + .../venice/pubsub/api/PubSubAdminAdapter.java | 2 +- .../pubsub/api/PubSubConsumerAdapter.java | 6 +- .../api/PubSubInstrumentedAdminAdapter.java | 166 ---- .../manager}/TopicManager.java | 500 ++++++------ .../pubsub/manager/TopicManagerContext.java | 222 ++++++ .../manager/TopicManagerRepository.java | 74 ++ .../pubsub/manager/TopicManagerStats.java | 58 ++ .../pubsub/manager/TopicMetadataFetcher.java | 737 ++++++++++++++++++ .../venice/stats/PubSubAdminWrapperStats.java | 65 -- .../venice/system/store/MetaStoreWriter.java | 2 +- .../linkedin/venice/utils/CachedCallable.java | 43 - .../com/linkedin/venice/utils/StoreUtils.java | 29 + .../TestHelixReadWriteSchemaRepository.java | 5 - .../partitioner/TestVenicePartitioner.java | 16 +- .../admin/ApacheKafkaAdminAdapterTest.java | 2 +- .../manager/TopicManagerContextTest.java | 146 ++++ .../manager/TopicManagerRepositoryTest.java | 63 ++ .../pubsub/manager/TopicManagerStatsTest.java | 50 ++ .../manager}/TopicManagerTest.java | 102 ++- .../manager/TopicMetadataFetcherTest.java | 494 ++++++++++++ .../stats/TestPubSubAdminWrapperStats.java | 34 - .../system/store/MetaStoreWriterTest.java | 2 +- .../controller/TestAdminToolEndToEnd.java | 2 +- ...LevelConfigForActiveActiveReplication.java | 6 +- ...lusterLevelConfigForNativeReplication.java | 6 +- .../controller/TestDelayedRebalance.java | 12 +- .../TestDeleteStoreDeletesRealtimeTopic.java | 12 +- ...stParentControllerWithMultiDataCenter.java | 2 +- .../TestTopicRequestOnHybridDelete.java | 10 +- ...VeniceHelixAdminWithSharedEnvironment.java | 10 +- .../AdminConsumptionTaskIntegrationTest.java | 8 +- .../venice/endToEnd/TestEmptyPush.java | 8 +- .../linkedin/venice/endToEnd/TestHybrid.java | 17 +- .../venice/endToEnd/TestHybridQuota.java | 8 +- .../endToEnd/TestHybridStoreDeletion.java | 8 +- .../TestPushJobWithNativeReplication.java | 7 +- .../endToEnd/TestStuckConsumerRepair.java | 8 +- .../input/kafka/TestKafkaInputFormat.java | 8 +- .../kafka/TestKafkaInputRecordReader.java | 8 +- .../utils/VeniceServerWrapper.java | 2 + .../venice/kafka/KafkaConsumptionTest.java | 11 +- .../PartitionOffsetFetcherTest.java | 64 -- .../consumer/PubSubConsumerAdapterTest.java | 27 +- .../pubsub/manager/TopicManagerE2ETest.java | 429 ++++++++++ .../manager}/TopicManagerIntegrationTest.java | 14 +- .../utils/IntegrationTestPushUtils.java | 25 +- ...ubSubSharedProducerAdapterFactoryTest.java | 16 +- .../venice/writer/VeniceWriterTest.java | 16 +- .../unit/kafka/MockInMemoryAdminAdapter.java | 2 +- .../linkedin/venice/utils/PubSubHelper.java | 120 +++ .../com/linkedin/venice/utils/TestUtils.java | 2 +- .../com/linkedin/venice/controller/Admin.java | 4 +- .../VeniceControllerClusterConfig.java | 4 +- .../controller/VeniceControllerConfig.java | 24 +- .../venice/controller/VeniceHelixAdmin.java | 62 +- .../controller/VeniceParentHelixAdmin.java | 8 +- .../controller/kafka/TopicCleanupService.java | 4 +- ...opicCleanupServiceForParentController.java | 6 +- .../kafka/consumer/AdminConsumerService.java | 2 +- .../kafka/consumer/AdminConsumptionTask.java | 4 +- .../controller/server/ControllerRoutes.java | 2 +- .../controller/server/StoresRoutes.java | 2 +- .../control/RealTimeTopicSwitcher.java | 14 +- .../AbstractTestVeniceParentHelixAdmin.java | 2 +- .../TestVeniceHelixAdminWithoutCluster.java | 2 +- .../TestVeniceParentHelixAdmin.java | 2 +- .../kafka/TestTopicCleanupService.java | 6 +- ...icCleanupServiceForMultiKafkaClusters.java | 6 +- ...opicCleanupServiceForParentController.java | 2 +- .../consumer/AdminConsumptionTaskTest.java | 2 +- .../consumer/TestAdminConsumerService.java | 2 +- .../RealTimeTopicSwitcherRewindTest.java | 2 +- .../control/RealTimeTopicSwitcherTest.java | 9 +- 108 files changed, 3370 insertions(+), 2216 deletions(-) delete mode 100644 clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/CachedPubSubMetadataGetter.java delete mode 100644 clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/CachedPubSubMetadataGetterTest.java delete mode 100644 internal/venice-common/src/main/java/com/linkedin/venice/kafka/TopicManagerRepository.java delete mode 100644 internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/InstrumentedPartitionOffsetFetcher.java delete mode 100644 internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcher.java delete mode 100644 internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherFactory.java delete mode 100644 internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherImpl.java delete mode 100644 internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherStats.java delete mode 100644 internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubInstrumentedAdminAdapter.java rename internal/venice-common/src/main/java/com/linkedin/venice/{kafka => pubsub/manager}/TopicManager.java (57%) create mode 100644 internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerContext.java create mode 100644 internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerRepository.java create mode 100644 internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerStats.java create mode 100644 internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java delete mode 100644 internal/venice-common/src/main/java/com/linkedin/venice/stats/PubSubAdminWrapperStats.java delete mode 100644 internal/venice-common/src/main/java/com/linkedin/venice/utils/CachedCallable.java create mode 100644 internal/venice-common/src/main/java/com/linkedin/venice/utils/StoreUtils.java create mode 100644 internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicManagerContextTest.java create mode 100644 internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicManagerRepositoryTest.java create mode 100644 internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicManagerStatsTest.java rename internal/venice-common/src/test/java/com/linkedin/venice/{kafka => pubsub/manager}/TopicManagerTest.java (90%) create mode 100644 internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcherTest.java delete mode 100644 internal/venice-common/src/test/java/com/linkedin/venice/stats/TestPubSubAdminWrapperStats.java delete mode 100644 internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherTest.java create mode 100644 internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/pubsub/manager/TopicManagerE2ETest.java rename internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/{kafka => pubsub/manager}/TopicManagerIntegrationTest.java (88%) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/config/VeniceServerConfig.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/config/VeniceServerConfig.java index b5298eb4e0..ecf531d416 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/config/VeniceServerConfig.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/config/VeniceServerConfig.java @@ -36,6 +36,8 @@ import static com.linkedin.venice.ConfigKeys.MIN_CONSUMER_IN_CONSUMER_POOL_PER_KAFKA_CLUSTER; import static com.linkedin.venice.ConfigKeys.OFFSET_LAG_DELTA_RELAX_FACTOR_FOR_FAST_ONLINE_TRANSITION_IN_RESTART; import static com.linkedin.venice.ConfigKeys.PARTICIPANT_MESSAGE_CONSUMPTION_DELAY_MS; +import static com.linkedin.venice.ConfigKeys.PUBSUB_TOPIC_MANAGER_METADATA_FETCHER_CONSUMER_POOL_SIZE; +import static com.linkedin.venice.ConfigKeys.PUBSUB_TOPIC_MANAGER_METADATA_FETCHER_THREAD_POOL_SIZE; import static com.linkedin.venice.ConfigKeys.PUB_SUB_ADMIN_ADAPTER_FACTORY_CLASS; import static com.linkedin.venice.ConfigKeys.PUB_SUB_CONSUMER_ADAPTER_FACTORY_CLASS; import static com.linkedin.venice.ConfigKeys.PUB_SUB_PRODUCER_ADAPTER_FACTORY_CLASS; @@ -131,6 +133,7 @@ import static com.linkedin.venice.ConfigKeys.SYSTEM_SCHEMA_INITIALIZATION_AT_START_TIME_ENABLED; import static com.linkedin.venice.ConfigKeys.UNREGISTER_METRIC_FOR_DELETED_STORE_ENABLED; import static com.linkedin.venice.ConfigKeys.UNSORTED_INPUT_DRAINER_SIZE; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_TOPIC_MANAGER_METADATA_FETCHER_CONSUMER_POOL_SIZE_DEFAULT_VALUE; import com.linkedin.davinci.helix.LeaderFollowerPartitionStateModelFactory; import com.linkedin.davinci.kafka.consumer.KafkaConsumerService; @@ -271,6 +274,9 @@ public class VeniceServerConfig extends VeniceClusterConfig { */ private final int topicOffsetCheckIntervalMs; + private final int topicManagerMetadataFetcherConsumerPoolSize; + private final int topicManagerMetadataFetcherThreadPoolSize; + /** * Graceful shutdown period. * Venice SN needs to explicitly do graceful shutdown since Netty's graceful shutdown logic @@ -511,6 +517,11 @@ public VeniceServerConfig(VeniceProperties serverProperties, Map functionToCheckWhetherStorageEngineShouldBeKep } PubSubTopicPartition topicPartition = getTopicPartition(partition); - Long earliestOffset = pubSubConsumer.beginningOffset(topicPartition, DEFAULT_KAFKA_OFFSET_API_TIMEOUT); + Long earliestOffset = + pubSubConsumer.beginningOffset(topicPartition, PUBSUB_OFFSET_API_TIMEOUT_DURATION_DEFAULT_VALUE); VeniceChangeCoordinate earliestCheckpoint = earliestOffset == null ? null : new VeniceChangeCoordinate( diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/AbstractKafkaConsumerService.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/AbstractKafkaConsumerService.java index 323b89ddf7..6ab3371793 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/AbstractKafkaConsumerService.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/AbstractKafkaConsumerService.java @@ -33,7 +33,9 @@ public abstract void startConsumptionIntoDataReceiver( long lastReadOffset, ConsumedDataReceiver>> consumedDataReceiver); - public abstract long getOffsetLagFor(PubSubTopic versionTopic, PubSubTopicPartition pubSubTopicPartition); + public abstract long getOffsetLagBasedOnMetrics(PubSubTopic versionTopic, PubSubTopicPartition pubSubTopicPartition); - public abstract long getLatestOffsetFor(PubSubTopic versionTopic, PubSubTopicPartition pubSubTopicPartition); + public abstract long getLatestOffsetBasedOnMetrics( + PubSubTopic versionTopic, + PubSubTopicPartition pubSubTopicPartition); } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java index e7ffd2b421..e6dea86639 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java @@ -28,7 +28,6 @@ import com.linkedin.venice.exceptions.PersistenceFailureException; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.exceptions.VeniceMessageException; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.kafka.protocol.ControlMessage; import com.linkedin.venice.kafka.protocol.Delete; import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; @@ -43,6 +42,7 @@ import com.linkedin.venice.meta.Store; import com.linkedin.venice.meta.Version; import com.linkedin.venice.offsets.OffsetRecord; +import com.linkedin.venice.pubsub.PubSubConstants; import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl; import com.linkedin.venice.pubsub.api.PubSubMessage; import com.linkedin.venice.pubsub.api.PubSubTopic; @@ -876,7 +876,7 @@ private long calculateRewindStartTime(PartitionConsumptionState partitionConsump long rewindTimeInMs = hybridStoreConfig.get().getRewindTimeInSeconds() * Time.MS_PER_SECOND; if (isDataRecovery) { // Override the user rewind if the version is under data recovery to avoid data loss when user have short rewind. - rewindTimeInMs = Math.max(TopicManager.BUFFER_REPLAY_MINIMAL_SAFETY_MARGIN, rewindTimeInMs); + rewindTimeInMs = Math.max(PubSubConstants.BUFFER_REPLAY_MINIMAL_SAFETY_MARGIN, rewindTimeInMs); } switch (hybridStoreConfig.get().getBufferReplayPolicy()) { case REWIND_FROM_SOP: @@ -1107,7 +1107,7 @@ protected boolean processTopicSwitch( try { PubSubTopicPartition newSourceTP = new PubSubTopicPartitionImpl(newSourceTopic, newSourceTopicPartition); upstreamStartOffset = - getTopicManager(sourceKafkaURL.toString()).getPartitionOffsetByTime(newSourceTP, rewindStartTimestamp); + getTopicManager(sourceKafkaURL.toString()).getOffsetByTime(newSourceTP, rewindStartTimestamp); numberOfContactedBrokers.getAndIncrement(); } catch (Exception e) { // TODO: Catch more specific Exception? @@ -1398,7 +1398,7 @@ public long getRegionHybridOffsetLag(int regionId) { // Consumer might not existed after the consumption state is created, but before attaching the corresponding // consumer. long offsetLagOptional = - getPartitionOffsetLag(kafkaSourceAddress, currentLeaderTopic, pcs.getUserPartition()); + getPartitionOffsetLagBasedOnMetrics(kafkaSourceAddress, currentLeaderTopic, pcs.getUserPartition()); if (offsetLagOptional >= 0) { return offsetLagOptional; } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/AggKafkaConsumerService.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/AggKafkaConsumerService.java index 5202407759..2bea3df298 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/AggKafkaConsumerService.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/AggKafkaConsumerService.java @@ -6,7 +6,6 @@ import com.linkedin.davinci.ingestion.consumption.ConsumedDataReceiver; import com.linkedin.davinci.stats.StuckConsumerRepairStats; import com.linkedin.venice.exceptions.VeniceException; -import com.linkedin.venice.kafka.TopicManagerRepository; import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; import com.linkedin.venice.message.KafkaKey; import com.linkedin.venice.meta.ReadOnlyStoreRepository; @@ -16,6 +15,8 @@ import com.linkedin.venice.pubsub.api.PubSubMessageDeserializer; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; +import com.linkedin.venice.pubsub.manager.TopicManager; +import com.linkedin.venice.pubsub.manager.TopicManagerContext.PubSubPropertiesSupplier; import com.linkedin.venice.service.AbstractVeniceService; import com.linkedin.venice.throttle.EventThrottler; import com.linkedin.venice.utils.DaemonThreadFactory; @@ -68,8 +69,8 @@ public class AggKafkaConsumerService extends AbstractVeniceService { private final Map kafkaClusterUrlToAliasMap; private final Object2IntMap kafkaClusterUrlToIdMap; private final PubSubMessageDeserializer pubSubDeserializer; - private final TopicManagerRepository.SSLPropertiesSupplier sslPropertiesSupplier; private final Function kafkaClusterUrlResolver; + private final PubSubPropertiesSupplier pubSubPropertiesSupplier; private final Map versionTopicStoreIngestionTaskMapping = new VeniceConcurrentHashMap<>(); private ScheduledExecutorService stuckConsumerRepairExecutorService; @@ -81,7 +82,7 @@ public class AggKafkaConsumerService extends AbstractVeniceService { public AggKafkaConsumerService( final PubSubConsumerAdapterFactory consumerFactory, - TopicManagerRepository.SSLPropertiesSupplier sslPropertiesSupplier, + final PubSubPropertiesSupplier pubSubPropertiesSupplier, final VeniceServerConfig serverConfig, final EventThrottler bandwidthThrottler, final EventThrottler recordsThrottler, @@ -107,10 +108,8 @@ public AggKafkaConsumerService( this.kafkaClusterUrlToIdMap = serverConfig.getKafkaClusterUrlToIdMap(); this.isKafkaConsumerOffsetCollectionEnabled = serverConfig.isKafkaConsumerOffsetCollectionEnabled(); this.pubSubDeserializer = pubSubDeserializer; - this.sslPropertiesSupplier = sslPropertiesSupplier; this.kafkaClusterUrlResolver = serverConfig.getKafkaClusterUrlResolver(); this.metadataRepository = metadataRepository; - if (serverConfig.isStuckConsumerRepairEnabled()) { this.stuckConsumerRepairExecutorService = Executors.newSingleThreadScheduledExecutor( new DaemonThreadFactory(this.getClass().getName() + "-StuckConsumerRepair")); @@ -130,7 +129,7 @@ public AggKafkaConsumerService( LOGGER.info("Started stuck consumer repair service with checking interval: {} seconds", intervalInSeconds); } this.isAAOrWCEnabledFunc = isAAOrWCEnabledFunc; - + this.pubSubPropertiesSupplier = pubSubPropertiesSupplier; LOGGER.info("Successfully initialized AggKafkaConsumerService"); } @@ -259,7 +258,7 @@ protected static Runnable getStuckConsumerDetectionAndRepairRunnable( * @return the {@link KafkaConsumerService} for a specific Kafka bootstrap url, * or null if there isn't any. */ - private AbstractKafkaConsumerService getKafkaConsumerService(final String kafkaURL) { + AbstractKafkaConsumerService getKafkaConsumerService(final String kafkaURL) { AbstractKafkaConsumerService consumerService = kafkaServerToConsumerServiceMap.get(kafkaURL); if (consumerService == null && kafkaClusterUrlResolver != null) { consumerService = kafkaServerToConsumerServiceMap.get(kafkaClusterUrlResolver.apply(kafkaURL)); @@ -276,7 +275,7 @@ private AbstractKafkaConsumerService getKafkaConsumerService(final String kafkaU */ public synchronized AbstractKafkaConsumerService createKafkaConsumerService(final Properties consumerProperties) { String kafkaUrl = consumerProperties.getProperty(KAFKA_BOOTSTRAP_SERVERS); - Properties properties = sslPropertiesSupplier.get(kafkaUrl).toProperties(); + Properties properties = pubSubPropertiesSupplier.get(kafkaUrl).toProperties(); consumerProperties.putAll(properties); if (kafkaUrl == null || kafkaUrl.isEmpty()) { throw new IllegalArgumentException("Kafka URL must be set in the consumer properties config. Got: " + kafkaUrl); @@ -393,24 +392,32 @@ public ConsumedDataReceiver> topicExistenceCache; - private final Map> offsetCache; - private final Map> lastProducerTimestampCache; - - CachedPubSubMetadataGetter(long timeToLiveMs) { - this.ttlNs = MILLISECONDS.toNanos(timeToLiveMs); - this.topicExistenceCache = new VeniceConcurrentHashMap<>(); - this.offsetCache = new VeniceConcurrentHashMap<>(); - this.lastProducerTimestampCache = new VeniceConcurrentHashMap<>(); - } - - /** - * @return Users of this method should be aware that Kafka will actually - * return the next available offset rather the latest used offset. Therefore, - * the value will be 1 offset greater than what's expected. - * - * TODO: Refactor this using PubSubTopicPartition - */ - long getOffset(TopicManager topicManager, PubSubTopic pubSubTopic, int partitionId) { - final String sourcePubSubServer = topicManager.getPubSubBootstrapServers(); - PubSubTopicPartition pubSubTopicPartition = new PubSubTopicPartitionImpl(pubSubTopic, partitionId); - try { - return fetchMetadata( - new PubSubMetadataCacheKey(sourcePubSubServer, pubSubTopicPartition), - offsetCache, - () -> topicManager.getPartitionLatestOffsetAndRetry(pubSubTopicPartition, DEFAULT_MAX_RETRY)); - } catch (PubSubTopicDoesNotExistException e) { - // It's observed in production that with java based admin client the topic may not be found temporarily, return - // error code - LOGGER.error("Failed to get offset for topic partition {}", pubSubTopicPartition, e); - return StatsErrorCode.LAG_MEASUREMENT_FAILURE.code; - } - } - - long getProducerTimestampOfLastDataMessage(TopicManager topicManager, PubSubTopicPartition pubSubTopicPartition) { - try { - return fetchMetadata( - new PubSubMetadataCacheKey(topicManager.getPubSubBootstrapServers(), pubSubTopicPartition), - lastProducerTimestampCache, - () -> topicManager.getProducerTimestampOfLastDataRecord(pubSubTopicPartition, DEFAULT_MAX_RETRY)); - } catch (PubSubTopicDoesNotExistException e) { - // It's observed in production that with java based admin client the topic may not be found temporarily, return - // error code - return StatsErrorCode.LAG_MEASUREMENT_FAILURE.code; - } - } - - boolean containsTopic(TopicManager topicManager, PubSubTopic pubSubTopic) { - return fetchMetadata( - new PubSubMetadataCacheKey( - topicManager.getPubSubBootstrapServers(), - new PubSubTopicPartitionImpl(pubSubTopic, -1)), - topicExistenceCache, - () -> topicManager.containsTopic(pubSubTopic)); - } - - /** - * Helper function to fetch metadata from cache or PubSub servers. - * @param key cache key: Topic name or TopicPartition - * @param metadataCache cache for this specific metadata - * @param valueSupplier function to fetch metadata from PubSub servers - * @param type of the metadata - * @return the cache value or the fresh metadata from PubSub servers - */ - T fetchMetadata( - PubSubMetadataCacheKey key, - Map> metadataCache, - Supplier valueSupplier) { - final long now = System.nanoTime(); - final ValueAndExpiryTime cachedValue = - metadataCache.computeIfAbsent(key, k -> new ValueAndExpiryTime<>(valueSupplier.get(), now + ttlNs)); - - // For a given key in the given cache, we will only issue one async request at the same time. - if (cachedValue.getExpiryTimeNs() <= now && cachedValue.valueUpdateInProgress.compareAndSet(false, true)) { - CompletableFuture.runAsync(() -> { - try { - T newValue = valueSupplier.get(); - metadataCache.put(key, new ValueAndExpiryTime<>(newValue, System.nanoTime() + ttlNs)); - } catch (Exception e) { - metadataCache.remove(key); - } - }); - } - return cachedValue.getValue(); - } - - static class PubSubMetadataCacheKey { - private final String pubSubServer; - private final PubSubTopicPartition pubSubTopicPartition; - - PubSubMetadataCacheKey(String pubSubServer, PubSubTopicPartition pubSubTopicPartition) { - this.pubSubServer = pubSubServer; - this.pubSubTopicPartition = pubSubTopicPartition; - } - - @Override - public int hashCode() { - int result = 1; - result = 31 * result + (pubSubServer == null ? 0 : pubSubServer.hashCode()); - result = 31 * result + (pubSubTopicPartition == null ? 0 : pubSubTopicPartition.hashCode()); - return result; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (!(o instanceof PubSubMetadataCacheKey)) { - return false; - } - - final PubSubMetadataCacheKey other = (PubSubMetadataCacheKey) o; - return pubSubTopicPartition.equals(other.pubSubTopicPartition) - && Objects.equals(pubSubServer, other.pubSubServer); - } - } - - /** - * A POJO contains a value and its expiry time in milliseconds. - * - * @param Type of the value. - */ - static class ValueAndExpiryTime { - private final T value; - private final long expiryTimeNs; - private final AtomicBoolean valueUpdateInProgress = new AtomicBoolean(false); - - ValueAndExpiryTime(T value, long expiryTimeNs) { - this.value = value; - this.expiryTimeNs = expiryTimeNs; - } - - T getValue() { - return value; - } - - long getExpiryTimeNs() { - return expiryTimeNs; - } - } -} diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/KafkaConsumerService.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/KafkaConsumerService.java index 985d127791..f94d80bda6 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/KafkaConsumerService.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/KafkaConsumerService.java @@ -435,7 +435,7 @@ final void recordPartitionsPerConsumerSensor() { aggStats.recordTotalMinPartitionsPerConsumer(minPartitionsPerConsumer); } - public long getOffsetLagFor(PubSubTopic versionTopic, PubSubTopicPartition pubSubTopicPartition) { + public long getOffsetLagBasedOnMetrics(PubSubTopic versionTopic, PubSubTopicPartition pubSubTopicPartition) { return getSomeOffsetFor( versionTopic, pubSubTopicPartition, @@ -444,7 +444,7 @@ public long getOffsetLagFor(PubSubTopic versionTopic, PubSubTopicPartition pubSu aggStats::recordTotalOffsetLagIsPresent); } - public long getLatestOffsetFor(PubSubTopic versionTopic, PubSubTopicPartition pubSubTopicPartition) { + public long getLatestOffsetBasedOnMetrics(PubSubTopic versionTopic, PubSubTopicPartition pubSubTopicPartition) { return getSomeOffsetFor( versionTopic, pubSubTopicPartition, diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/KafkaConsumerServiceDelegator.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/KafkaConsumerServiceDelegator.java index dc11acf877..8a88e0d7d2 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/KafkaConsumerServiceDelegator.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/KafkaConsumerServiceDelegator.java @@ -128,15 +128,15 @@ public void startConsumptionIntoDataReceiver( } @Override - public long getOffsetLagFor(PubSubTopic versionTopic, PubSubTopicPartition pubSubTopicPartition) { + public long getOffsetLagBasedOnMetrics(PubSubTopic versionTopic, PubSubTopicPartition pubSubTopicPartition) { return getKafkaConsumerService(versionTopic, pubSubTopicPartition) - .getOffsetLagFor(versionTopic, pubSubTopicPartition); + .getOffsetLagBasedOnMetrics(versionTopic, pubSubTopicPartition); } @Override - public long getLatestOffsetFor(PubSubTopic versionTopic, PubSubTopicPartition pubSubTopicPartition) { + public long getLatestOffsetBasedOnMetrics(PubSubTopic versionTopic, PubSubTopicPartition pubSubTopicPartition) { return getKafkaConsumerService(versionTopic, pubSubTopicPartition) - .getLatestOffsetFor(versionTopic, pubSubTopicPartition); + .getLatestOffsetBasedOnMetrics(versionTopic, pubSubTopicPartition); } @Override diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/KafkaStoreIngestionService.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/KafkaStoreIngestionService.java index a0a48e3b3a..caac067c50 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/KafkaStoreIngestionService.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/KafkaStoreIngestionService.java @@ -1,6 +1,5 @@ package com.linkedin.davinci.kafka.consumer; -import static com.linkedin.venice.ConfigConstants.DEFAULT_TOPIC_DELETION_STATUS_POLL_INTERVAL_MS; import static com.linkedin.venice.ConfigKeys.KAFKA_AUTO_OFFSET_RESET_CONFIG; import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; import static com.linkedin.venice.ConfigKeys.KAFKA_CLIENT_ID_CONFIG; @@ -11,8 +10,6 @@ import static com.linkedin.venice.ConfigKeys.KAFKA_GROUP_ID_CONFIG; import static com.linkedin.venice.ConfigKeys.KAFKA_MAX_PARTITION_FETCH_BYTES_CONFIG; import static com.linkedin.venice.ConfigKeys.KAFKA_MAX_POLL_RECORDS_CONFIG; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_MIN_LOG_COMPACTION_LAG_MS; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_OPERATION_TIMEOUT_MS; import static java.lang.Thread.currentThread; import static java.lang.Thread.sleep; @@ -45,7 +42,6 @@ import com.linkedin.venice.helix.HelixCustomizedViewOfflinePushRepository; import com.linkedin.venice.helix.HelixInstanceConfigRepository; import com.linkedin.venice.helix.HelixReadOnlyZKSharedSchemaRepository; -import com.linkedin.venice.kafka.TopicManagerRepository; import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; import com.linkedin.venice.kafka.protocol.state.PartitionState; import com.linkedin.venice.meta.ClusterInfoProvider; @@ -70,6 +66,8 @@ import com.linkedin.venice.pubsub.adapter.kafka.producer.ApacheKafkaProducerConfig; import com.linkedin.venice.pubsub.adapter.kafka.producer.SharedKafkaProducerAdapterFactory; import com.linkedin.venice.pubsub.api.PubSubMessageDeserializer; +import com.linkedin.venice.pubsub.manager.TopicManagerContext; +import com.linkedin.venice.pubsub.manager.TopicManagerRepository; import com.linkedin.venice.schema.SchemaEntry; import com.linkedin.venice.schema.SchemaReader; import com.linkedin.venice.security.SSLFactory; @@ -313,17 +311,18 @@ public KafkaStoreIngestionService( KafkaClusterBasedRecordThrottler kafkaClusterBasedRecordThrottler = new KafkaClusterBasedRecordThrottler(kafkaUrlToRecordsThrottler); - this.topicManagerRepository = TopicManagerRepository.builder() - .setPubSubTopicRepository(pubSubTopicRepository) - .setMetricsRepository(metricsRepository) - .setLocalKafkaBootstrapServers(serverConfig.getKafkaBootstrapServers()) - .setPubSubConsumerAdapterFactory(pubSubClientsFactory.getConsumerAdapterFactory()) - .setTopicDeletionStatusPollIntervalMs(DEFAULT_TOPIC_DELETION_STATUS_POLL_INTERVAL_MS) - .setTopicMinLogCompactionLagMs(DEFAULT_KAFKA_MIN_LOG_COMPACTION_LAG_MS) - .setKafkaOperationTimeoutMs(DEFAULT_KAFKA_OPERATION_TIMEOUT_MS) - .setPubSubProperties(this::getPubSubSSLPropertiesFromServerConfig) - .setPubSubAdminAdapterFactory(pubSubClientsFactory.getAdminAdapterFactory()) - .build(); + TopicManagerContext topicManagerContext = + new TopicManagerContext.Builder().setPubSubTopicRepository(pubSubTopicRepository) + .setMetricsRepository(metricsRepository) + .setTopicOffsetCheckIntervalMs(serverConfig.getTopicOffsetCheckIntervalMs()) + .setPubSubPropertiesSupplier(this::getPubSubSSLPropertiesFromServerConfig) + .setPubSubAdminAdapterFactory(pubSubClientsFactory.getAdminAdapterFactory()) + .setPubSubConsumerAdapterFactory(pubSubClientsFactory.getConsumerAdapterFactory()) + .setTopicMetadataFetcherThreadPoolSize(serverConfig.getTopicManagerMetadataFetcherThreadPoolSize()) + .setTopicMetadataFetcherConsumerPoolSize(serverConfig.getTopicManagerMetadataFetcherConsumerPoolSize()) + .build(); + this.topicManagerRepository = + new TopicManagerRepository(topicManagerContext, serverConfig.getKafkaBootstrapServers()); VeniceNotifier notifier = new LogNotifier(); this.leaderFollowerNotifiers.add(notifier); @@ -334,7 +333,7 @@ public KafkaStoreIngestionService( */ if (zkSharedSchemaRepository.isPresent()) { this.metaStoreWriter = new MetaStoreWriter( - topicManagerRepository.getTopicManager(), + topicManagerRepository.getLocalTopicManager(), veniceWriterFactoryForMetaStoreWriter, zkSharedSchemaRepository.get(), pubSubTopicRepository, diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index 992709d88c..2921a64d1a 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -39,7 +39,6 @@ import com.linkedin.venice.exceptions.validation.DuplicateDataException; import com.linkedin.venice.exceptions.validation.FatalDataValidationException; import com.linkedin.venice.guid.GuidUtils; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.kafka.protocol.ControlMessage; import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; import com.linkedin.venice.kafka.protocol.Put; @@ -52,6 +51,7 @@ import com.linkedin.venice.meta.Store; import com.linkedin.venice.meta.Version; import com.linkedin.venice.offsets.OffsetRecord; +import com.linkedin.venice.pubsub.PubSubConstants; import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl; import com.linkedin.venice.pubsub.api.PubSubMessage; import com.linkedin.venice.pubsub.api.PubSubMessageHeader; @@ -249,7 +249,7 @@ public LeaderFollowerStoreIngestionTask( isDataRecovery = true; dataRecoverySourceVersionNumber = version.getDataRecoveryVersionConfig().getDataRecoverySourceVersionNumber(); if (isHybridMode()) { - dataRecoveryCompletionTimeLagThresholdInMs = TopicManager.BUFFER_REPLAY_MINIMAL_SAFETY_MARGIN / 2; + dataRecoveryCompletionTimeLagThresholdInMs = PubSubConstants.BUFFER_REPLAY_MINIMAL_SAFETY_MARGIN / 2; LOGGER.info( "Data recovery info for topic: {}, source kafka url: {}, time lag threshold for completion: {}", getVersionTopic(), @@ -938,10 +938,10 @@ protected long getTopicPartitionOffsetByKafkaURL( PubSubTopicPartition pubSubTopicPartition, long rewindStartTimestamp) { long topicPartitionOffset = - getTopicManager(kafkaURL.toString()).getPartitionOffsetByTime(pubSubTopicPartition, rewindStartTimestamp); + getTopicManager(kafkaURL.toString()).getOffsetByTime(pubSubTopicPartition, rewindStartTimestamp); /** - * {@link com.linkedin.venice.kafka.TopicManager#getPartitionOffsetByTime} will always return the next offset - * to consume, but {@link ApacheKafkaConsumer#subscribe} is always + * {@link com.linkedin.venice.pubsub.manager.TopicManager#getOffsetByTime} will always + * return the next offset to consume, but {@link ApacheKafkaConsumer#subscribe} is always * seeking the next offset, so we will deduct 1 from the returned offset here. */ return topicPartitionOffset - 1; @@ -1175,8 +1175,8 @@ protected boolean processTopicSwitch( int newSourceTopicPartitionId = partitionConsumptionState.getSourceTopicPartitionNumber(newSourceTopic); PubSubTopicPartition newSourceTopicPartition = new PubSubTopicPartitionImpl(newSourceTopic, newSourceTopicPartitionId); - upstreamStartOffset = getTopicManager(sourceKafkaURL) - .getPartitionOffsetByTime(newSourceTopicPartition, topicSwitch.rewindStartTimestamp); + upstreamStartOffset = + getTopicManager(sourceKafkaURL).getOffsetByTime(newSourceTopicPartition, topicSwitch.rewindStartTimestamp); if (upstreamStartOffset != OffsetRecord.LOWEST_OFFSET) { upstreamStartOffset -= 1; } @@ -1606,9 +1606,9 @@ protected long measureHybridOffsetLag(PartitionConsumptionState partitionConsump /** * After END_OF_PUSH received, `isReadyToServe()` is invoked for each message until the lag is caught up (otherwise, * if we only check ready to serve periodically, the lag may never catch up); in order not to slow down the hybrid - * ingestion, {@link CachedPubSubMetadataGetter} was introduced to get the latest offset periodically; - * with this strategy, it is possible that partition could become 'ONLINE' at most - * {@link CachedPubSubMetadataGetter#ttlMs} earlier. + * ingestion, {@link com.linkedin.venice.pubsub.manager.TopicMetadataFetcher} was introduced to get the latest + * offset periodically and cache them; with this strategy, it is possible that partition could become 'ONLINE' at + * most {@link com.linkedin.venice.pubsub.manager.TopicMetadataFetcher#ttlInNs} earlier. */ PubSubTopic leaderTopic = offsetRecord.getLeaderTopic(pubSubTopicRepository); if (leaderTopic == null || !leaderTopic.isRealTime()) { @@ -2568,20 +2568,20 @@ public long getBatchReplicationLag() { String sourceKafkaURL = getSourceKafkaUrlForOffsetLagMeasurement(pcs); // Consumer might not exist after the consumption state is created, but before attaching the corresponding // consumer. - long offsetLagOptional = getPartitionOffsetLag(sourceKafkaURL, currentLeaderTopic, pcs.getUserPartition()); + long offsetLagOptional = + getPartitionOffsetLagBasedOnMetrics(sourceKafkaURL, currentLeaderTopic, pcs.getUserPartition()); if (offsetLagOptional >= 0) { return offsetLagOptional; } // Fall back to use the old way (latest VT offset in remote kafka - latest VT offset in local kafka) - long localOffset = cachedPubSubMetadataGetter - .getOffset(getTopicManager(localKafkaServer), currentLeaderTopic, pcs.getPartition()) - 1; + long localOffset = + getTopicManager(localKafkaServer).getLatestOffsetCached(currentLeaderTopic, pcs.getPartition()) - 1; return measureLagWithCallToPubSub( nativeReplicationSourceVersionTopicKafkaURL, currentLeaderTopic, pcs.getPartition(), localOffset); }).filter(VALID_LAG).sum(); - return minZeroLag(replicationLag); } @@ -2625,7 +2625,8 @@ private long getLeaderOffsetLag(Predicate par final String kafkaSourceAddress = getSourceKafkaUrlForOffsetLagMeasurement(pcs); // Consumer might not exist after the consumption state is created, but before attaching the corresponding // consumer. - long offsetLagOptional = getPartitionOffsetLag(kafkaSourceAddress, currentLeaderTopic, pcs.getPartition()); + long offsetLagOptional = + getPartitionOffsetLagBasedOnMetrics(kafkaSourceAddress, currentLeaderTopic, pcs.getPartition()); if (offsetLagOptional >= 0) { return offsetLagOptional; } @@ -2701,7 +2702,8 @@ private long getFollowerOffsetLag(Predicate p .mapToLong((pcs) -> { // Consumer might not existed after the consumption state is created, but before attaching the corresponding // consumer. - long offsetLagOptional = getPartitionOffsetLag(localKafkaServer, versionTopic, pcs.getPartition()); + long offsetLagOptional = + getPartitionOffsetLagBasedOnMetrics(localKafkaServer, versionTopic, pcs.getPartition()); if (offsetLagOptional >= 0) { return offsetLagOptional; } @@ -3298,7 +3300,6 @@ protected long measureRTOffsetLagForSingleRegion( } PubSubTopic leaderTopic = pcs.getOffsetRecord().getLeaderTopic(pubSubTopicRepository); - long lastOffsetInRealTimeTopic = getTopicPartitionEndOffSet(sourceRealTimeTopicKafkaURL, leaderTopic, partitionToGetLatestOffsetFor); diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 9d755c6fc0..3e758c9fde 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -49,8 +49,6 @@ import com.linkedin.venice.exceptions.validation.FatalDataValidationException; import com.linkedin.venice.exceptions.validation.ImproperlyStartedSegmentException; import com.linkedin.venice.exceptions.validation.UnsupportedMessageTypeException; -import com.linkedin.venice.kafka.TopicManager; -import com.linkedin.venice.kafka.TopicManagerRepository; import com.linkedin.venice.kafka.protocol.ControlMessage; import com.linkedin.venice.kafka.protocol.Delete; import com.linkedin.venice.kafka.protocol.EndOfIncrementalPush; @@ -80,6 +78,8 @@ import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; import com.linkedin.venice.pubsub.api.exceptions.PubSubUnsubscribedTopicPartitionException; +import com.linkedin.venice.pubsub.manager.TopicManager; +import com.linkedin.venice.pubsub.manager.TopicManagerRepository; import com.linkedin.venice.schema.SchemaEntry; import com.linkedin.venice.serialization.avro.AvroProtocolDefinition; import com.linkedin.venice.serialization.avro.InternalAvroSpecificSerializer; @@ -190,7 +190,6 @@ public abstract class StoreIngestionTask implements Runnable, Closeable { protected final Map partitionToPendingConsumerActionCountMap; protected final StorageMetadataService storageMetadataService; protected final TopicManagerRepository topicManagerRepository; - protected final CachedPubSubMetadataGetter cachedPubSubMetadataGetter; /** Per-partition consumption state map */ protected final ConcurrentMap partitionConsumptionStateMap; protected final AbstractStoreBufferService storeBufferService; @@ -365,8 +364,6 @@ public StoreIngestionTask( builder.getServerConfig().getDivProducerStateMaxAgeMs()); this.ingestionTaskName = String.format(CONSUMER_TASK_ID_FORMAT, kafkaVersionTopic); this.topicManagerRepository = builder.getTopicManagerRepository(); - this.cachedPubSubMetadataGetter = new CachedPubSubMetadataGetter(storeConfig.getTopicOffsetCheckIntervalMs()); - this.hostLevelIngestionStats = builder.getIngestionStats().getStoreStats(storeName); this.versionedDIVStats = builder.getVersionedDIVStats(); this.versionedIngestionStats = builder.getVersionedStorageIngestionStats(); @@ -891,8 +888,8 @@ protected boolean isReadyToServe(PartitionConsumptionState partitionConsumptionS // DaVinci and STANDBY checks the local consumption and leaderCompleteState status final String lagMeasurementKafkaUrl = (isHybridFollower(partitionConsumptionState)) ? localKafkaServer : realTimeTopicKafkaURL; - - if (!cachedPubSubMetadataGetter.containsTopic(getTopicManager(lagMeasurementKafkaUrl), realTimeTopic)) { + TopicManager topicManager = getTopicManager(lagMeasurementKafkaUrl); + if (!topicManager.containsTopicCached(realTimeTopic)) { timestampLagIsAcceptable = true; if (!REDUNDANT_LOGGING_FILTER.isRedundantException(msgIdentifier)) { LOGGER.info( @@ -901,8 +898,8 @@ protected boolean isReadyToServe(PartitionConsumptionState partitionConsumptionS lagMeasurementTopic); } } else { - long latestProducerTimestampInTopic = cachedPubSubMetadataGetter - .getProducerTimestampOfLastDataMessage(getTopicManager(lagMeasurementKafkaUrl), pubSubTopicPartition); + long latestProducerTimestampInTopic = + topicManager.getProducerTimestampOfLastDataMessageCached(pubSubTopicPartition); if (latestProducerTimestampInTopic < 0 || latestProducerTimestampInTopic <= latestConsumedProducerTimestamp) { timestampLagIsAcceptable = true; @@ -1525,6 +1522,13 @@ private void internalClose(boolean doFlush) { LOGGER.error("Error while closing venice view writer", e); } + if (topicManagerRepository != null) { + Collection topicManagers = topicManagerRepository.getAllTopicManagers(); + for (TopicManager topicManager: topicManagers) { + topicManager.invalidateCache(versionTopic); + } + } + close(); synchronized (this) { @@ -1953,7 +1957,7 @@ private void reportStoreVersionTopicOffsetRewindMetrics(PartitionConsumptionStat * N.B.: We do not want to use {@link #getTopicPartitionEndOffSet(String, PubSubTopic, int)} because it can return * a cached value which will result in a false positive in the below check. */ - long endOffset = aggKafkaConsumerService.getLatestOffsetFor( + long endOffset = aggKafkaConsumerService.getLatestOffsetBasedOnMetrics( localKafkaServer, versionTopic, new PubSubTopicPartitionImpl(versionTopic, pcs.getPartition())); @@ -1978,17 +1982,16 @@ private void reportStoreVersionTopicOffsetRewindMetrics(PartitionConsumptionStat */ protected long getTopicPartitionEndOffSet(String kafkaUrl, PubSubTopic pubSubTopic, int partition) { long offsetFromConsumer = aggKafkaConsumerService - .getLatestOffsetFor(kafkaUrl, versionTopic, new PubSubTopicPartitionImpl(pubSubTopic, partition)); + .getLatestOffsetBasedOnMetrics(kafkaUrl, versionTopic, new PubSubTopicPartitionImpl(pubSubTopic, partition)); if (offsetFromConsumer >= 0) { return offsetFromConsumer; } - - return cachedPubSubMetadataGetter.getOffset(getTopicManager(kafkaUrl), pubSubTopic, partition); + return getTopicManager(kafkaUrl).getLatestOffsetCached(pubSubTopic, partition); } - protected long getPartitionOffsetLag(String kafkaSourceAddress, PubSubTopic topic, int partition) { + protected long getPartitionOffsetLagBasedOnMetrics(String kafkaSourceAddress, PubSubTopic topic, int partition) { return aggKafkaConsumerService - .getOffsetLagFor(kafkaSourceAddress, versionTopic, new PubSubTopicPartitionImpl(topic, partition)); + .getOffsetLagBasedOnMetrics(kafkaSourceAddress, versionTopic, new PubSubTopicPartitionImpl(topic, partition)); } protected abstract void checkLongRunningTaskState() throws InterruptedException; @@ -2407,7 +2410,7 @@ protected long measureLagWithCallToPubSub( return Long.MAX_VALUE; } TopicManager tm = getTopicManager(pubSubServerName); - long endOffset = cachedPubSubMetadataGetter.getOffset(tm, topic, partition) - 1; + long endOffset = tm.getLatestOffsetCached(topic, partition) - 1; if (endOffset < 0) { return Long.MAX_VALUE; } @@ -2872,7 +2875,7 @@ protected void validateMessage( return; } Lazy tolerateMissingMsgs = Lazy.of(() -> { - TopicManager topicManager = topicManagerRepository.getTopicManager(); + TopicManager topicManager = topicManagerRepository.getLocalTopicManager(); // Tolerate missing message if store version is data recovery + hybrid and TS not received yet (due to source // topic // data may have been log compacted) or log compaction is enabled and record is old enough for log compaction. @@ -3694,7 +3697,7 @@ public void updateOffsetMetadataAndSync(String topic, int partitionId) { protected TopicManager getTopicManager(String sourceKafkaServer) { if (sourceKafkaServer.equals(localKafkaServer)) { // Use default kafka admin client (could be scala or java based) to get local topic manager - return topicManagerRepository.getTopicManager(); + return topicManagerRepository.getLocalTopicManager(); } // Use java-based kafka admin client to get remote topic manager return topicManagerRepository.getTopicManager(sourceKafkaServer); @@ -3880,7 +3883,7 @@ public boolean isProducingVersionTopicHealthy() { */ return true; } - if (!topicManagerRepository.getTopicManager().containsTopic(this.versionTopic)) { + if (!topicManagerRepository.getLocalTopicManager().containsTopic(this.versionTopic)) { return false; } return true; diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskFactory.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskFactory.java index 1301d2d208..dd7a096c27 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskFactory.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskFactory.java @@ -12,13 +12,13 @@ import com.linkedin.davinci.storage.StorageMetadataService; import com.linkedin.davinci.store.cache.backend.ObjectCacheBackend; import com.linkedin.davinci.store.view.VeniceViewWriterFactory; -import com.linkedin.venice.kafka.TopicManagerRepository; import com.linkedin.venice.kafka.protocol.state.PartitionState; import com.linkedin.venice.meta.ReadOnlySchemaRepository; import com.linkedin.venice.meta.ReadOnlyStoreRepository; import com.linkedin.venice.meta.Store; import com.linkedin.venice.meta.Version; import com.linkedin.venice.pubsub.PubSubTopicRepository; +import com.linkedin.venice.pubsub.manager.TopicManagerRepository; import com.linkedin.venice.serialization.avro.InternalAvroSpecificSerializer; import com.linkedin.venice.system.store.MetaStoreWriter; import com.linkedin.venice.utils.DiskUsage; diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/InternalLocalBootstrappingVeniceChangelogConsumerTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/InternalLocalBootstrappingVeniceChangelogConsumerTest.java index 5f84e1a7fe..4533fb6e83 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/InternalLocalBootstrappingVeniceChangelogConsumerTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/consumer/InternalLocalBootstrappingVeniceChangelogConsumerTest.java @@ -10,9 +10,6 @@ import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.any; -import static org.mockito.Mockito.anyInt; -import static org.mockito.Mockito.anyLong; -import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/AggKafkaConsumerServiceTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/AggKafkaConsumerServiceTest.java index 5af803c9cc..159cc4ef08 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/AggKafkaConsumerServiceTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/AggKafkaConsumerServiceTest.java @@ -1,20 +1,129 @@ package com.linkedin.davinci.kafka.consumer; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import com.linkedin.davinci.config.VeniceServerConfig; import com.linkedin.davinci.stats.StuckConsumerRepairStats; +import com.linkedin.venice.meta.ReadOnlyStoreRepository; +import com.linkedin.venice.pubsub.PubSubConsumerAdapterFactory; +import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl; +import com.linkedin.venice.pubsub.PubSubTopicRepository; +import com.linkedin.venice.pubsub.api.PubSubMessageDeserializer; +import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.api.PubSubTopicPartition; +import com.linkedin.venice.pubsub.manager.TopicManager; +import com.linkedin.venice.pubsub.manager.TopicManagerContext.PubSubPropertiesSupplier; +import com.linkedin.venice.throttle.EventThrottler; +import com.linkedin.venice.utils.Utils; +import io.tehuti.metrics.MetricsRepository; +import it.unimi.dsi.fastutil.objects.Object2IntMaps; import java.util.HashMap; import java.util.Map; import java.util.function.Consumer; +import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; public class AggKafkaConsumerServiceTest { + private PubSubConsumerAdapterFactory consumerFactory; + private PubSubPropertiesSupplier pubSubPropertiesSupplier; + private VeniceServerConfig serverConfig; + private EventThrottler bandwidthThrottler; + private EventThrottler recordsThrottler; + private KafkaClusterBasedRecordThrottler kafkaClusterBasedRecordThrottler; + private MetricsRepository metricsRepository; + private TopicExistenceChecker topicExistenceChecker; + private PubSubMessageDeserializer pubSubDeserializer; + private Consumer killIngestionTaskRunnable; + private ReadOnlyStoreRepository metadataRepository; + private PubSubTopicRepository topicRepository; + private AggKafkaConsumerService aggKafkaConsumerService; + private String pubSubUrl = "pubsub.venice.db"; + private PubSubTopic topic; + private PubSubTopicPartition topicPartition; + + @BeforeMethod + public void setUp() { + topicRepository = new PubSubTopicRepository(); + topic = topicRepository.getTopic(Utils.getUniqueString("topic") + "_v1"); + topicPartition = new PubSubTopicPartitionImpl(topic, 0); + consumerFactory = mock(PubSubConsumerAdapterFactory.class); + pubSubPropertiesSupplier = mock(PubSubPropertiesSupplier.class); + bandwidthThrottler = mock(EventThrottler.class); + recordsThrottler = mock(EventThrottler.class); + kafkaClusterBasedRecordThrottler = mock(KafkaClusterBasedRecordThrottler.class); + metricsRepository = mock(MetricsRepository.class); + topicExistenceChecker = mock(TopicExistenceChecker.class); + pubSubDeserializer = mock(PubSubMessageDeserializer.class); + killIngestionTaskRunnable = mock(Consumer.class); + metadataRepository = mock(ReadOnlyStoreRepository.class); + serverConfig = mock(VeniceServerConfig.class); + when(serverConfig.getKafkaClusterUrlToIdMap()).thenReturn(Object2IntMaps.EMPTY_MAP); + + aggKafkaConsumerService = new AggKafkaConsumerService( + consumerFactory, + pubSubPropertiesSupplier, + serverConfig, + bandwidthThrottler, + recordsThrottler, + kafkaClusterBasedRecordThrottler, + metricsRepository, + topicExistenceChecker, + pubSubDeserializer, + killIngestionTaskRunnable, + t -> false, + metadataRepository); + } + + // test subscribeConsumerFor + @Test + public void testSubscribeConsumerFor() { + AggKafkaConsumerService aggKafkaConsumerServiceSpy = spy(aggKafkaConsumerService); + StoreIngestionTask storeIngestionTask = mock(StoreIngestionTask.class); + TopicManager topicManager = mock(TopicManager.class); + + doReturn(mock(AbstractKafkaConsumerService.class)).when(aggKafkaConsumerServiceSpy).getKafkaConsumerService(any()); + when(storeIngestionTask.getVersionTopic()).thenReturn(topic); + when(storeIngestionTask.getTopicManager(pubSubUrl)).thenReturn(topicManager); + + aggKafkaConsumerServiceSpy.subscribeConsumerFor(pubSubUrl, storeIngestionTask, topicPartition, -1); + + verify(topicManager).prefetchAndCacheLatestOffset(topicPartition); + } + + @Test + public void testGetOffsetLagBasedOnMetrics() { + AggKafkaConsumerService aggKafkaConsumerServiceSpy = spy(aggKafkaConsumerService); + + doReturn(null).when(aggKafkaConsumerServiceSpy).getKafkaConsumerService(pubSubUrl); + assertEquals(aggKafkaConsumerServiceSpy.getOffsetLagBasedOnMetrics(pubSubUrl, topic, topicPartition), -1); + + AbstractKafkaConsumerService consumerService = mock(AbstractKafkaConsumerService.class); + when(consumerService.getOffsetLagBasedOnMetrics(topic, topicPartition)).thenReturn(123L); + doReturn(consumerService).when(aggKafkaConsumerServiceSpy).getKafkaConsumerService(any()); + assertEquals(aggKafkaConsumerServiceSpy.getOffsetLagBasedOnMetrics(pubSubUrl, topic, topicPartition), 123L); + } + + @Test + public void testGetLatestOffsetBasedOnMetrics() { + AggKafkaConsumerService aggKafkaConsumerServiceSpy = spy(aggKafkaConsumerService); + doReturn(null).when(aggKafkaConsumerServiceSpy).getKafkaConsumerService(pubSubUrl); + assertEquals(aggKafkaConsumerServiceSpy.getLatestOffsetBasedOnMetrics(pubSubUrl, topic, topicPartition), -1); + + AbstractKafkaConsumerService consumerService = mock(AbstractKafkaConsumerService.class); + when(consumerService.getLatestOffsetBasedOnMetrics(topic, topicPartition)).thenReturn(1234L); + doReturn(consumerService).when(aggKafkaConsumerServiceSpy).getKafkaConsumerService(any()); + assertEquals(aggKafkaConsumerServiceSpy.getLatestOffsetBasedOnMetrics(pubSubUrl, topic, topicPartition), 1234L); + } + @Test public void testGetStuckConsumerDetectionAndRepairRunnable() { Map kafkaServerToConsumerServiceMap = new HashMap<>(); diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/CachedPubSubMetadataGetterTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/CachedPubSubMetadataGetterTest.java deleted file mode 100644 index b75dca8bac..0000000000 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/CachedPubSubMetadataGetterTest.java +++ /dev/null @@ -1,64 +0,0 @@ -package com.linkedin.davinci.kafka.consumer; - -import static org.mockito.ArgumentMatchers.*; - -import com.linkedin.venice.exceptions.VeniceException; -import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl; -import com.linkedin.venice.pubsub.PubSubTopicRepository; -import com.linkedin.venice.utils.TestUtils; -import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap; -import java.util.Map; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import org.testng.Assert; -import org.testng.annotations.Test; - - -public class CachedPubSubMetadataGetterTest { - private final PubSubTopicRepository pubSubTopicRepository = new PubSubTopicRepository(); - - @Test - public void testCacheWillResetStatusWhenExceptionIsThrown() { - CachedPubSubMetadataGetter cachedPubSubMetadataGetter = new CachedPubSubMetadataGetter(1000); - CachedPubSubMetadataGetter.PubSubMetadataCacheKey key = new CachedPubSubMetadataGetter.PubSubMetadataCacheKey( - "server", - new PubSubTopicPartitionImpl(pubSubTopicRepository.getTopic(TestUtils.getUniqueTopicString("topic")), 1)); - Map> offsetCache = - new VeniceConcurrentHashMap<>(); - CachedPubSubMetadataGetter.ValueAndExpiryTime valueCache = - new CachedPubSubMetadataGetter.ValueAndExpiryTime<>(1L, System.nanoTime()); - offsetCache.put(key, valueCache); - // Successful call will update the value from 1 to 2. - TestUtils.waitForNonDeterministicAssertion(5, TimeUnit.SECONDS, () -> { - Long actualResult = cachedPubSubMetadataGetter.fetchMetadata(key, offsetCache, () -> 2L); - Long expectedResult = 2L; - Assert.assertEquals(actualResult, expectedResult); - }); - - // For persisting exception, it will be caught and thrown. - TestUtils.waitForNonDeterministicAssertion(5, TimeUnit.SECONDS, () -> { - Assert - .assertThrows(VeniceException.class, () -> cachedPubSubMetadataGetter.fetchMetadata(key, offsetCache, () -> { - throw new VeniceException("dummy exception"); - })); - }); - - // Reset the cached value to 1. - valueCache = new CachedPubSubMetadataGetter.ValueAndExpiryTime<>(1L, System.nanoTime()); - offsetCache.put(key, valueCache); - - // The first call throws a transient exception, and it should be updated to expected value after second call. - AtomicBoolean exceptionFlag = new AtomicBoolean(false); - TestUtils.waitForNonDeterministicAssertion(5, TimeUnit.SECONDS, false, true, () -> { - Long actualResult = cachedPubSubMetadataGetter.fetchMetadata(key, offsetCache, () -> { - if (exceptionFlag.compareAndSet(false, true)) { - throw new VeniceException("do not throw this exception!"); - } else { - return 2L; - } - }); - Long expectedResult = 2L; - Assert.assertEquals(actualResult, expectedResult); - }); - } -} diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/KafkaConsumerServiceDelegatorTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/KafkaConsumerServiceDelegatorTest.java index bfcbaed175..d664fa6d92 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/KafkaConsumerServiceDelegatorTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/KafkaConsumerServiceDelegatorTest.java @@ -35,7 +35,7 @@ public class KafkaConsumerServiceDelegatorTest { @DataProvider(name = "Method-List") public static Object[][] methodList() { return new Object[][] { { "getConsumerAssignedToVersionTopicPartition" }, { "assignConsumerFor" }, - { "unSubscribe" }, { "getOffsetLagFor" }, { "getLatestOffsetFor" } }; + { "unSubscribe" }, { "getOffsetLagBasedOnMetrics" }, { "getLatestOffsetBasedOnMetrics" } }; } @Test(dataProvider = "Method-List") diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/KafkaStoreIngestionServiceTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/KafkaStoreIngestionServiceTest.java index 269b7fb7ec..d0abd11a20 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/KafkaStoreIngestionServiceTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/KafkaStoreIngestionServiceTest.java @@ -133,6 +133,8 @@ private void setupMockConfig() { doReturn(1).when(mockVeniceServerConfig).getConsumerPoolSizePerKafkaCluster(); doReturn(SecurityProtocol.PLAINTEXT).when(mockVeniceServerConfig).getKafkaSecurityProtocol(dummyKafkaUrl); doReturn(10).when(mockVeniceServerConfig).getKafkaMaxPollRecords(); + doReturn(2).when(mockVeniceServerConfig).getTopicManagerMetadataFetcherConsumerPoolSize(); + doReturn(2).when(mockVeniceServerConfig).getTopicManagerMetadataFetcherThreadPoolSize(); VeniceClusterConfig mockVeniceClusterConfig = mock(VeniceClusterConfig.class); Properties properties = new Properties(); diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskTest.java index 33678340fa..bf03ebe3b4 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTaskTest.java @@ -108,8 +108,6 @@ import com.linkedin.venice.exceptions.validation.FatalDataValidationException; import com.linkedin.venice.exceptions.validation.MissingDataException; import com.linkedin.venice.guid.GuidUtils; -import com.linkedin.venice.kafka.TopicManager; -import com.linkedin.venice.kafka.TopicManagerRepository; import com.linkedin.venice.kafka.protocol.ControlMessage; import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; import com.linkedin.venice.kafka.protocol.LeaderMetadata; @@ -156,6 +154,8 @@ import com.linkedin.venice.pubsub.api.PubSubTopicPartition; import com.linkedin.venice.pubsub.api.PubSubTopicType; import com.linkedin.venice.pubsub.api.exceptions.PubSubUnsubscribedTopicPartitionException; +import com.linkedin.venice.pubsub.manager.TopicManager; +import com.linkedin.venice.pubsub.manager.TopicManagerRepository; import com.linkedin.venice.schema.SchemaEntry; import com.linkedin.venice.schema.rmd.RmdSchemaEntry; import com.linkedin.venice.schema.rmd.RmdSchemaGenerator; @@ -526,7 +526,7 @@ public void methodSetUp() throws Exception { mockTopicManager = mock(TopicManager.class); mockTopicManagerRepository = mock(TopicManagerRepository.class); - doReturn(mockTopicManager).when(mockTopicManagerRepository).getTopicManager(); + doReturn(mockTopicManager).when(mockTopicManagerRepository).getLocalTopicManager(); mockAggStoreIngestionStats = mock(AggHostLevelIngestionStats.class); mockStoreIngestionStats = mock(HostLevelIngestionStats.class); @@ -1457,14 +1457,8 @@ public void testNotifier(AAConfig aaConfig) throws Exception { long barLastOffset = getOffset(localVeniceWriter.put(putKeyBar, putValue, SCHEMA_ID)); localVeniceWriter.broadcastEndOfPush(new HashMap<>()); localVeniceWriter.broadcastEndOfPush(new HashMap<>()); - doReturn(fooLastOffset + 1).when(mockTopicManager) - .getPartitionLatestOffsetAndRetry( - argThat(argument -> argument.getPartitionNumber() == PARTITION_FOO), - anyInt()); - doReturn(barLastOffset + 1).when(mockTopicManager) - .getPartitionLatestOffsetAndRetry( - argThat(argument -> argument.getPartitionNumber() == PARTITION_BAR), - anyInt()); + doReturn(fooLastOffset + 1).when(mockTopicManager).getLatestOffsetCached(any(), eq(PARTITION_FOO)); + doReturn(barLastOffset + 1).when(mockTopicManager).getLatestOffsetCached(any(), eq(PARTITION_BAR)); runTest(Utils.setOf(PARTITION_FOO, PARTITION_BAR), () -> { /** @@ -2338,9 +2332,9 @@ public void testDelayedTransitionToOnlineInHybridMode(AAConfig aaConfig) throws BufferReplayPolicy.REWIND_FROM_EOP)); long[] messageCountPerPartition = new long[PARTITION_COUNT]; - when(mockTopicManager.getPartitionLatestOffsetAndRetry(any(), anyInt())).thenAnswer(invocation -> { - PubSubTopicPartition pt = invocation.getArgument(0); - return messageCountPerPartition[pt.getPartitionNumber()]; + when(mockTopicManager.getLatestOffsetCached(any(), anyInt())).thenAnswer(invocation -> { + int partitionNumber = invocation.getArgument(1); + return messageCountPerPartition[partitionNumber]; }); runTest(ALL_PARTITIONS, () -> { @@ -2845,8 +2839,8 @@ public void testIsReadyToServe(NodeType nodeType, AAConfig aaConfig) { doReturn(mockTopicManagerRemoteKafka).when(mockTopicManagerRepository) .getTopicManager(inMemoryRemoteKafkaBroker.getKafkaBootstrapServer()); - doReturn(true).when(mockTopicManager).containsTopic(any()); - doReturn(true).when(mockTopicManagerRemoteKafka).containsTopic(any()); + doReturn(true).when(mockTopicManager).containsTopicCached(any()); + doReturn(true).when(mockTopicManagerRemoteKafka).containsTopicCached(any()); Properties kafkaProps = new Properties(); kafkaProps.put(KAFKA_BOOTSTRAP_SERVERS, inMemoryLocalKafkaBroker.getKafkaBootstrapServer()); @@ -2928,8 +2922,8 @@ public void testIsReadyToServe(NodeType nodeType, AAConfig aaConfig) { doReturn(true).when(mockPcsBufferReplayStartedLagCaughtUp).isHybrid(); doReturn(topicSwitchWithSourceRealTimeTopicWrapper).when(mockPcsBufferReplayStartedLagCaughtUp).getTopicSwitch(); doReturn(mockOffsetRecordLagCaughtUp).when(mockPcsBufferReplayStartedLagCaughtUp).getOffsetRecord(); - doReturn(5L).when(mockTopicManager).getPartitionLatestOffsetAndRetry(any(), anyInt()); - doReturn(5L).when(mockTopicManagerRemoteKafka).getPartitionLatestOffsetAndRetry(any(), anyInt()); + doReturn(5L).when(mockTopicManager).getLatestOffsetCached(any(), anyInt()); + doReturn(5L).when(mockTopicManagerRemoteKafka).getLatestOffsetCached(any(), anyInt()); doReturn(0).when(mockPcsBufferReplayStartedLagCaughtUp).getPartition(); doReturn(0).when(mockPcsBufferReplayStartedLagCaughtUp).getUserPartition(); storeIngestionTaskUnderTest.setPartitionConsumptionState(0, mockPcsBufferReplayStartedLagCaughtUp); @@ -2964,9 +2958,9 @@ public void testIsReadyToServe(NodeType nodeType, AAConfig aaConfig) { doReturn(topicSwitchWithSourceRealTimeTopicWrapper).when(mockPcsBufferReplayStartedRemoteLagging).getTopicSwitch(); doReturn(mockOffsetRecordLagCaughtUpTimestampLagging).when(mockPcsBufferReplayStartedRemoteLagging) .getOffsetRecord(); - doReturn(5L).when(mockTopicManager).getPartitionLatestOffsetAndRetry(any(), anyInt()); - doReturn(150L).when(mockTopicManagerRemoteKafka).getPartitionLatestOffsetAndRetry(any(), anyInt()); - doReturn(150L).when(aggKafkaConsumerService).getLatestOffsetFor(anyString(), any(), any()); + doReturn(5L).when(mockTopicManager).getLatestOffsetCached(any(), anyInt()); + doReturn(150L).when(mockTopicManagerRemoteKafka).getLatestOffsetCached(any(), anyInt()); + doReturn(150L).when(aggKafkaConsumerService).getLatestOffsetBasedOnMetrics(anyString(), any(), any()); if (nodeType == NodeType.LEADER) { // case 6a: leader replica => partition is not ready to serve doReturn(LeaderFollowerStateType.LEADER).when(mockPcsBufferReplayStartedRemoteLagging).getLeaderFollowerState(); @@ -2997,12 +2991,12 @@ public void testIsReadyToServe(NodeType nodeType, AAConfig aaConfig) { doReturn(topicSwitchWithSourceRealTimeTopicWrapper).when(mockPcsOffsetLagCaughtUpTimestampLagging).getTopicSwitch(); doReturn(mockOffsetRecordLagCaughtUpTimestampLagging).when(mockPcsOffsetLagCaughtUpTimestampLagging) .getOffsetRecord(); - doReturn(5L).when(mockTopicManager).getPartitionLatestOffsetAndRetry(any(), anyInt()); - doReturn(5L).when(mockTopicManagerRemoteKafka).getPartitionLatestOffsetAndRetry(any(), anyInt()); + doReturn(5L).when(mockTopicManager).getLatestOffsetCached(any(), anyInt()); + doReturn(5L).when(mockTopicManagerRemoteKafka).getLatestOffsetCached(any(), anyInt()); doReturn(System.currentTimeMillis() - 2 * MS_PER_DAY).when(mockTopicManager) - .getProducerTimestampOfLastDataRecord(any(), anyInt()); + .getProducerTimestampOfLastDataMessageCached(any()); doReturn(System.currentTimeMillis()).when(mockTopicManagerRemoteKafka) - .getProducerTimestampOfLastDataRecord(any(), anyInt()); + .getProducerTimestampOfLastDataMessageCached(any()); if (nodeType == NodeType.LEADER) { // case 7a: leader replica => partition is not ready to serve doReturn(LeaderFollowerStateType.LEADER).when(mockPcsOffsetLagCaughtUpTimestampLagging).getLeaderFollowerState(); @@ -3111,9 +3105,9 @@ public void testActiveActiveStoreIsReadyToServe(HybridConfig hybridConfig, NodeT doReturn(hybridConfig == HYBRID).when(mockPcsMultipleSourceKafkaServers).isHybrid(); doReturn(topicSwitchWithMultipleSourceKafkaServersWrapper).when(mockPcsMultipleSourceKafkaServers).getTopicSwitch(); doReturn(mockOffsetRecord).when(mockPcsMultipleSourceKafkaServers).getOffsetRecord(); - doReturn(5L).when(mockTopicManager).getPartitionLatestOffsetAndRetry(any(), anyInt()); - doReturn(150L).when(mockTopicManagerRemoteKafka).getPartitionLatestOffsetAndRetry(any(), anyInt()); - doReturn(150L).when(aggKafkaConsumerService).getLatestOffsetFor(anyString(), any(), any()); + doReturn(5L).when(mockTopicManager).getLatestOffsetCached(any(), anyInt()); + doReturn(150L).when(mockTopicManagerRemoteKafka).getLatestOffsetCached(any(), anyInt()); + doReturn(150L).when(aggKafkaConsumerService).getLatestOffsetBasedOnMetrics(anyString(), any(), any()); doReturn(0).when(mockPcsMultipleSourceKafkaServers).getPartition(); doReturn(0).when(mockPcsMultipleSourceKafkaServers).getUserPartition(); doReturn(5L).when(mockPcsMultipleSourceKafkaServers).getLatestProcessedLocalVersionTopicOffset(); @@ -3519,8 +3513,7 @@ public void testProcessTopicSwitch(NodeType nodeType) { storeIngestionTaskUnderTest.getStatusReportAdapter().initializePartitionReportStatus(PARTITION_FOO); storeIngestionTaskUnderTest.processTopicSwitch(controlMessage, PARTITION_FOO, 10, mockPcs); - verify(mockTopicManagerRemoteKafka, nodeType == DA_VINCI ? never() : times(1)) - .getPartitionOffsetByTime(any(), anyLong()); + verify(mockTopicManagerRemoteKafka, nodeType == DA_VINCI ? never() : times(1)).getOffsetByTime(any(), anyLong()); } @Test(dataProvider = "aaConfigProvider") diff --git a/clients/venice-admin-tool/src/main/java/com/linkedin/venice/AdminTool.java b/clients/venice-admin-tool/src/main/java/com/linkedin/venice/AdminTool.java index 84fec6e7fe..327d324b63 100644 --- a/clients/venice-admin-tool/src/main/java/com/linkedin/venice/AdminTool.java +++ b/clients/venice-admin-tool/src/main/java/com/linkedin/venice/AdminTool.java @@ -76,8 +76,6 @@ import com.linkedin.venice.helix.HelixAdapterSerializer; import com.linkedin.venice.helix.HelixSchemaAccessor; import com.linkedin.venice.helix.ZkClientFactory; -import com.linkedin.venice.kafka.TopicManager; -import com.linkedin.venice.kafka.TopicManagerRepository; import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; import com.linkedin.venice.meta.BackupStrategy; import com.linkedin.venice.meta.BufferReplayPolicy; @@ -96,6 +94,9 @@ import com.linkedin.venice.pubsub.api.PubSubConsumerAdapter; import com.linkedin.venice.pubsub.api.PubSubMessageDeserializer; import com.linkedin.venice.pubsub.api.exceptions.PubSubOpTimeoutException; +import com.linkedin.venice.pubsub.manager.TopicManager; +import com.linkedin.venice.pubsub.manager.TopicManagerContext; +import com.linkedin.venice.pubsub.manager.TopicManagerRepository; import com.linkedin.venice.pushmonitor.ExecutionStatus; import com.linkedin.venice.schema.SchemaEntry; import com.linkedin.venice.schema.avro.SchemaCompatibility; @@ -1504,17 +1505,20 @@ private static void deleteKafkaTopic(CommandLine cmd) throws Exception { kafkaTimeOut = Integer.parseInt(getRequiredArgument(cmd, Arg.KAFKA_OPERATION_TIMEOUT)) * Time.MS_PER_SECOND; } - try (TopicManagerRepository topicManagerRepository = TopicManagerRepository.builder() - .setPubSubProperties(k -> veniceProperties) - .setKafkaOperationTimeoutMs(kafkaTimeOut) - .setTopicDeletionStatusPollIntervalMs(topicDeletionStatusPollingInterval) - .setTopicMinLogCompactionLagMs(0L) - .setLocalKafkaBootstrapServers(kafkaBootstrapServer) - .setPubSubConsumerAdapterFactory(PUB_SUB_CLIENTS_FACTORY.getConsumerAdapterFactory()) - .setPubSubAdminAdapterFactory(PUB_SUB_CLIENTS_FACTORY.getAdminAdapterFactory()) - .setPubSubTopicRepository(pubSubTopicRepository) - .build()) { - TopicManager topicManager = topicManagerRepository.getTopicManager(); + TopicManagerContext topicManagerContext = + new TopicManagerContext.Builder().setPubSubPropertiesSupplier(k -> veniceProperties) + .setPubSubOperationTimeoutMs(kafkaTimeOut) + .setTopicDeletionStatusPollIntervalMs(topicDeletionStatusPollingInterval) + .setTopicMinLogCompactionLagMs(0L) + .setPubSubConsumerAdapterFactory(PUB_SUB_CLIENTS_FACTORY.getConsumerAdapterFactory()) + .setPubSubAdminAdapterFactory(PUB_SUB_CLIENTS_FACTORY.getAdminAdapterFactory()) + .setPubSubTopicRepository(pubSubTopicRepository) + .setTopicMetadataFetcherConsumerPoolSize(1) + .setTopicMetadataFetcherThreadPoolSize(1) + .build(); + + try (TopicManager topicManager = + new TopicManagerRepository(topicManagerContext, kafkaBootstrapServer).getLocalTopicManager()) { String topicName = getRequiredArgument(cmd, Arg.KAFKA_TOPIC_NAME); try { topicManager.ensureTopicIsDeletedAndBlock(PUB_SUB_TOPIC_REPOSITORY.getTopic(topicName)); diff --git a/clients/venice-admin-tool/src/main/java/com/linkedin/venice/KafkaTopicDumper.java b/clients/venice-admin-tool/src/main/java/com/linkedin/venice/KafkaTopicDumper.java index 1df0b18d23..9428d4d1ae 100644 --- a/clients/venice-admin-tool/src/main/java/com/linkedin/venice/KafkaTopicDumper.java +++ b/clients/venice-admin-tool/src/main/java/com/linkedin/venice/KafkaTopicDumper.java @@ -1,7 +1,7 @@ package com.linkedin.venice; import static com.linkedin.venice.chunking.ChunkKeyValueTransformer.KeyType.WITH_VALUE_CHUNK; -import static com.linkedin.venice.kafka.partitionoffset.PartitionOffsetFetcherImpl.DEFAULT_KAFKA_OFFSET_API_TIMEOUT; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_OFFSET_API_TIMEOUT_DURATION_DEFAULT_VALUE; import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; import com.linkedin.venice.chunking.ChunkKeyValueTransformer; @@ -206,7 +206,8 @@ public KafkaTopicDumper( PubSubTopicPartition partition = new PubSubTopicPartitionImpl(pubSubTopicRepository.getTopic(topicName), partitionNumber); - Long partitionBeginningOffset = consumer.beginningOffset(partition, DEFAULT_KAFKA_OFFSET_API_TIMEOUT); + Long partitionBeginningOffset = + consumer.beginningOffset(partition, PUBSUB_OFFSET_API_TIMEOUT_DURATION_DEFAULT_VALUE); long computedStartingOffset = Math.max(partitionBeginningOffset, startingOffset); LOGGER.info("Starting from offset: {}", computedStartingOffset); consumer.subscribe(partition, computedStartingOffset - 1); diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputFormat.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputFormat.java index 254e5781ee..7c214b7151 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputFormat.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputFormat.java @@ -8,12 +8,13 @@ import com.linkedin.venice.hadoop.input.kafka.avro.KafkaInputMapperValue; import com.linkedin.venice.hadoop.mapreduce.datawriter.task.ReporterBackedMapReduceDataWriterTaskTracker; import com.linkedin.venice.hadoop.task.datawriter.DataWriterTaskTracker; -import com.linkedin.venice.kafka.TopicManager; -import com.linkedin.venice.kafka.TopicManagerRepository; import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.adapter.kafka.admin.ApacheKafkaAdminAdapterFactory; import com.linkedin.venice.pubsub.adapter.kafka.consumer.ApacheKafkaConsumerAdapterFactory; import com.linkedin.venice.pubsub.api.PubSubConsumerAdapter; +import com.linkedin.venice.pubsub.manager.TopicManager; +import com.linkedin.venice.pubsub.manager.TopicManagerContext; +import com.linkedin.venice.pubsub.manager.TopicManagerRepository; import com.linkedin.venice.utils.VeniceProperties; import java.io.IOException; import java.util.HashMap; @@ -46,22 +47,22 @@ public class KafkaInputFormat implements InputFormat getLatestOffsets(JobConf config) { VeniceProperties consumerProperties = KafkaInputUtils.getConsumerProperties(config); - try (TopicManagerRepository topicManagerRepository = TopicManagerRepository.builder() - .setPubSubProperties(k -> consumerProperties) - .setLocalKafkaBootstrapServers(config.get(KAFKA_INPUT_BROKER_URL)) - .setPubSubTopicRepository(pubSubTopicRepository) - .setPubSubAdminAdapterFactory(new ApacheKafkaAdminAdapterFactory()) - .setPubSubConsumerAdapterFactory(new ApacheKafkaConsumerAdapterFactory()) - .build()) { - try (TopicManager topicManager = topicManagerRepository.getTopicManager()) { - String topic = config.get(KAFKA_INPUT_TOPIC); - Map latestOffsets = topicManager.getTopicLatestOffsets(pubSubTopicRepository.getTopic(topic)); - Map partitionOffsetMap = new HashMap<>(latestOffsets.size()); - latestOffsets.forEach( - (partitionId, latestOffset) -> partitionOffsetMap - .put(new TopicPartition(topic, partitionId), latestOffset)); - return partitionOffsetMap; - } + TopicManagerContext topicManagerContext = + new TopicManagerContext.Builder().setPubSubPropertiesSupplier(k -> consumerProperties) + .setPubSubTopicRepository(pubSubTopicRepository) + .setPubSubAdminAdapterFactory(new ApacheKafkaAdminAdapterFactory()) + .setPubSubConsumerAdapterFactory(new ApacheKafkaConsumerAdapterFactory()) + .setTopicMetadataFetcherThreadPoolSize(1) + .setTopicMetadataFetcherConsumerPoolSize(1) + .build(); + try (TopicManager topicManager = + new TopicManagerRepository(topicManagerContext, config.get(KAFKA_INPUT_BROKER_URL)).getLocalTopicManager()) { + String topic = config.get(KAFKA_INPUT_TOPIC); + Map latestOffsets = topicManager.getTopicLatestOffsets(pubSubTopicRepository.getTopic(topic)); + Map partitionOffsetMap = new HashMap<>(latestOffsets.size()); + latestOffsets.forEach( + (partitionId, latestOffset) -> partitionOffsetMap.put(new TopicPartition(topic, partitionId), latestOffset)); + return partitionOffsetMap; } } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/ConfigConstants.java b/internal/venice-common/src/main/java/com/linkedin/venice/ConfigConstants.java index 535f0b30ee..5fa3f6c1c8 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/ConfigConstants.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/ConfigConstants.java @@ -1,6 +1,5 @@ package com.linkedin.venice; -import com.linkedin.venice.utils.Time; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -11,11 +10,6 @@ public class ConfigConstants { * Start of controller config default value */ - /** - * Default value of sleep interval for polling topic deletion status from ZK. - */ - public static final int DEFAULT_TOPIC_DELETION_STATUS_POLL_INTERVAL_MS = 2 * Time.MS_PER_SECOND; - public static final int UNSPECIFIED_REPLICATION_METADATA_VERSION = -1; /** diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/ConfigKeys.java b/internal/venice-common/src/main/java/com/linkedin/venice/ConfigKeys.java index 9a3d62d844..992b9bbfd1 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/ConfigKeys.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/ConfigKeys.java @@ -86,6 +86,18 @@ private ConfigKeys() { // Kafka security protocol public static final String KAFKA_SECURITY_PROTOCOL = "security.protocol"; + /** + * Number of PubSub consumer clients to be used per topic manager for fetching metadata. + */ + public static final String PUBSUB_TOPIC_MANAGER_METADATA_FETCHER_CONSUMER_POOL_SIZE = + "pubsub.topic.manager.metadata.fetcher.consumer.pool.size"; + + /** + * The size of the per topic manager thread pool for fetching metadata asynchronously. + */ + public static final String PUBSUB_TOPIC_MANAGER_METADATA_FETCHER_THREAD_POOL_SIZE = + "pubsub.topic.manager.metadata.fetcher.thread.pool.size"; + // Cluster specific configs for controller public static final String CONTROLLER_NAME = "controller.name"; diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/TopicManagerRepository.java b/internal/venice-common/src/main/java/com/linkedin/venice/kafka/TopicManagerRepository.java deleted file mode 100644 index 1bd0c6d03d..0000000000 --- a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/TopicManagerRepository.java +++ /dev/null @@ -1,184 +0,0 @@ -package com.linkedin.venice.kafka; - -import static com.linkedin.venice.ConfigConstants.DEFAULT_TOPIC_DELETION_STATUS_POLL_INTERVAL_MS; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_MIN_LOG_COMPACTION_LAG_MS; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_OPERATION_TIMEOUT_MS; - -import com.linkedin.venice.exceptions.VeniceException; -import com.linkedin.venice.pubsub.PubSubAdminAdapterFactory; -import com.linkedin.venice.pubsub.PubSubConsumerAdapterFactory; -import com.linkedin.venice.pubsub.PubSubTopicRepository; -import com.linkedin.venice.pubsub.api.PubSubAdminAdapter; -import com.linkedin.venice.pubsub.api.PubSubConsumerAdapter; -import com.linkedin.venice.utils.VeniceProperties; -import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap; -import com.linkedin.venice.utils.lazy.Lazy; -import io.tehuti.metrics.MetricsRepository; -import java.io.Closeable; -import java.util.Map; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Function; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - - -public class TopicManagerRepository implements Closeable { - private static final Logger LOGGER = LogManager.getLogger(TopicManagerRepository.class); - private final Map topicManagersMap = new VeniceConcurrentHashMap<>(); - private final Function topicManagerCreator; - private final Lazy localTopicManager; - - public TopicManagerRepository(Builder builder) { - this.topicManagerCreator = (kafkaServerAddress) -> new TopicManager(builder, kafkaServerAddress); - this.localTopicManager = Lazy.of( - () -> topicManagersMap.computeIfAbsent( - builder.localKafkaBootstrapServers, - k -> topicManagerCreator.apply(builder.localKafkaBootstrapServers))); - } - - /** - * By default, return TopicManager for local Kafka cluster. - */ - public TopicManager getTopicManager() { - return localTopicManager.get(); - } - - public TopicManager getTopicManager(String kafkaBootstrapServers) { - return topicManagersMap - .computeIfAbsent(kafkaBootstrapServers, k -> topicManagerCreator.apply(kafkaBootstrapServers)); - } - - @Override - public void close() { - AtomicReference lastException = new AtomicReference<>(); - topicManagersMap.entrySet().stream().forEach(entry -> { - try { - LOGGER.info("Closing TopicManager for Kafka cluster [" + entry.getKey() + "]"); - entry.getValue().close(); - LOGGER.info("Closed TopicManager for Kafka cluster [" + entry.getKey() + "]"); - } catch (Exception e) { - LOGGER.error("Error when closing TopicManager for Kafka cluster [" + entry.getKey() + "]"); - lastException.set(e); - } - }); - if (lastException.get() != null) { - throw new VeniceException(lastException.get()); - } - LOGGER.info("All TopicManager closed."); - } - - /** - * @return a new builder for the {@link TopicManagerRepository} - */ - public static Builder builder() { - return new Builder(); - } - - public static class Builder { - private volatile boolean built = false; - private String localKafkaBootstrapServers; - private long kafkaOperationTimeoutMs = DEFAULT_KAFKA_OPERATION_TIMEOUT_MS; - private long topicDeletionStatusPollIntervalMs = DEFAULT_TOPIC_DELETION_STATUS_POLL_INTERVAL_MS; - private long topicMinLogCompactionLagMs = DEFAULT_KAFKA_MIN_LOG_COMPACTION_LAG_MS; - private PubSubAdminAdapterFactory pubSubAdminAdapterFactory; - private PubSubConsumerAdapterFactory pubSubConsumerAdapterFactory; - private PubSubTopicRepository pubSubTopicRepository; - private MetricsRepository metricsRepository; - private SSLPropertiesSupplier pubSubProperties; - - private interface Setter { - void apply(); - } - - private Builder set(Setter setter) { - if (!built) { - setter.apply(); - } - return this; - } - - public TopicManagerRepository build() { - // flip the build flag to prevent modification. - this.built = true; - return new TopicManagerRepository(this); - } - - public String getLocalKafkaBootstrapServers() { - return localKafkaBootstrapServers; - } - - public long getKafkaOperationTimeoutMs() { - return kafkaOperationTimeoutMs; - } - - public long getTopicDeletionStatusPollIntervalMs() { - return topicDeletionStatusPollIntervalMs; - } - - public long getTopicMinLogCompactionLagMs() { - return topicMinLogCompactionLagMs; - } - - public MetricsRepository getMetricsRepository() { - return metricsRepository; - } - - public PubSubTopicRepository getPubSubTopicRepository() { - return pubSubTopicRepository; - } - - public PubSubAdminAdapterFactory getPubSubAdminAdapterFactory() { - return pubSubAdminAdapterFactory; - } - - public PubSubConsumerAdapterFactory getPubSubConsumerAdapterFactory() { - return pubSubConsumerAdapterFactory; - } - - public SSLPropertiesSupplier getPubSubProperties() { - return pubSubProperties; - } - - public Builder setLocalKafkaBootstrapServers(String localKafkaBootstrapServers) { - return set(() -> this.localKafkaBootstrapServers = localKafkaBootstrapServers); - } - - public Builder setKafkaOperationTimeoutMs(long kafkaOperationTimeoutMs) { - return set(() -> this.kafkaOperationTimeoutMs = kafkaOperationTimeoutMs); - } - - public Builder setTopicDeletionStatusPollIntervalMs(long topicDeletionStatusPollIntervalMs) { - return set(() -> this.topicDeletionStatusPollIntervalMs = topicDeletionStatusPollIntervalMs); - } - - public Builder setTopicMinLogCompactionLagMs(long topicMinLogCompactionLagMs) { - return set(() -> this.topicMinLogCompactionLagMs = topicMinLogCompactionLagMs); - } - - public Builder setMetricsRepository(MetricsRepository metricsRepository) { - return set(() -> this.metricsRepository = metricsRepository); - } - - public Builder setPubSubTopicRepository(PubSubTopicRepository pubSubTopicRepository) { - return set(() -> this.pubSubTopicRepository = pubSubTopicRepository); - } - - public Builder setPubSubAdminAdapterFactory( - PubSubAdminAdapterFactory pubSubAdminAdapterFactory) { - return set(() -> this.pubSubAdminAdapterFactory = pubSubAdminAdapterFactory); - } - - public Builder setPubSubConsumerAdapterFactory( - PubSubConsumerAdapterFactory pubSubConsumerAdapterFactory) { - return set(() -> this.pubSubConsumerAdapterFactory = pubSubConsumerAdapterFactory); - } - - public Builder setPubSubProperties(SSLPropertiesSupplier pubSubProperties) { - return set(() -> this.pubSubProperties = pubSubProperties); - } - } - - public interface SSLPropertiesSupplier { - VeniceProperties get(String pubSubBootstrapServers); - } -} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/InstrumentedPartitionOffsetFetcher.java b/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/InstrumentedPartitionOffsetFetcher.java deleted file mode 100644 index 084d9ceed9..0000000000 --- a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/InstrumentedPartitionOffsetFetcher.java +++ /dev/null @@ -1,101 +0,0 @@ -package com.linkedin.venice.kafka.partitionoffset; - -import com.linkedin.venice.pubsub.PubSubTopicPartitionInfo; -import com.linkedin.venice.pubsub.api.PubSubTopic; -import com.linkedin.venice.pubsub.api.PubSubTopicPartition; -import com.linkedin.venice.pubsub.api.exceptions.PubSubTopicDoesNotExistException; -import com.linkedin.venice.utils.Time; -import com.linkedin.venice.utils.Utils; -import it.unimi.dsi.fastutil.ints.Int2LongMap; -import java.util.List; -import javax.annotation.Nonnull; -import org.apache.commons.lang.Validate; - - -public class InstrumentedPartitionOffsetFetcher implements PartitionOffsetFetcher { - private final PartitionOffsetFetcher partitionOffsetFetcher; - private final PartitionOffsetFetcherStats stats; - private final Time time; - - public InstrumentedPartitionOffsetFetcher( - @Nonnull PartitionOffsetFetcher partitionOffsetFetcher, - @Nonnull PartitionOffsetFetcherStats stats, - @Nonnull Time time) { - Validate.notNull(partitionOffsetFetcher); - Validate.notNull(stats); - Validate.notNull(time); - this.partitionOffsetFetcher = partitionOffsetFetcher; - this.stats = stats; - this.time = time; - } - - @Override - public Int2LongMap getTopicLatestOffsets(PubSubTopic topic) { - final long startTimeMs = time.getMilliseconds(); - Int2LongMap res = partitionOffsetFetcher.getTopicLatestOffsets(topic); - stats.recordLatency( - PartitionOffsetFetcherStats.OCCURRENCE_LATENCY_SENSOR_TYPE.GET_TOPIC_LATEST_OFFSETS, - Utils.calculateDurationMs(time, startTimeMs)); - return res; - } - - @Override - public long getPartitionLatestOffsetAndRetry(PubSubTopicPartition topicPartition, int retries) { - final long startTimeMs = time.getMilliseconds(); - try { - long value = partitionOffsetFetcher.getPartitionLatestOffsetAndRetry(topicPartition, retries); - stats.recordLatency( - PartitionOffsetFetcherStats.OCCURRENCE_LATENCY_SENSOR_TYPE.GET_PARTITION_LATEST_OFFSET_WITH_RETRY, - Utils.calculateDurationMs(time, startTimeMs)); - return value; - } catch (PubSubTopicDoesNotExistException e) { - stats.recordGetLatestOffsetError(); - throw e; - } - } - - @Override - public long getPartitionOffsetByTime(PubSubTopicPartition topicPartition, long timestamp) { - final long startTimeMs = time.getMilliseconds(); - long res = partitionOffsetFetcher.getPartitionOffsetByTime(topicPartition, timestamp); - stats.recordLatency( - PartitionOffsetFetcherStats.OCCURRENCE_LATENCY_SENSOR_TYPE.GET_PARTITION_OFFSET_BY_TIME, - Utils.calculateDurationMs(time, startTimeMs)); - return res; - } - - @Override - public long getProducerTimestampOfLastDataRecord(PubSubTopicPartition topicPartition, int retries) { - final long startTimeMs = time.getMilliseconds(); - long res = partitionOffsetFetcher.getProducerTimestampOfLastDataRecord(topicPartition, retries); - stats.recordLatency( - PartitionOffsetFetcherStats.OCCURRENCE_LATENCY_SENSOR_TYPE.GET_LATEST_PRODUCER_TIMESTAMP_ON_DATA_RECORD_WITH_RETRY, - Utils.calculateDurationMs(time, startTimeMs)); - return res; - } - - @Override - public List partitionsFor(PubSubTopic topic) { - final long startTimeMs = time.getMilliseconds(); - List res = partitionOffsetFetcher.partitionsFor(topic); - stats.recordLatency( - PartitionOffsetFetcherStats.OCCURRENCE_LATENCY_SENSOR_TYPE.PARTITIONS_FOR, - Utils.calculateDurationMs(time, startTimeMs)); - return res; - } - - @Override - public long getOffsetByTimeIfOutOfRange(PubSubTopicPartition topicPartition, long timestamp) { - final long startTimeMs = time.getMilliseconds(); - long res = partitionOffsetFetcher.getOffsetByTimeIfOutOfRange(topicPartition, timestamp); - stats.recordLatency( - PartitionOffsetFetcherStats.OCCURRENCE_LATENCY_SENSOR_TYPE.GET_PARTITION_OFFSET_BY_TIME_IF_OUT_OF_RANGE, - Utils.calculateDurationMs(time, startTimeMs)); - return res; - } - - @Override - public void close() { - partitionOffsetFetcher.close(); - } -} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcher.java b/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcher.java deleted file mode 100644 index ef9914f6a0..0000000000 --- a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcher.java +++ /dev/null @@ -1,36 +0,0 @@ -package com.linkedin.venice.kafka.partitionoffset; - -import com.linkedin.venice.annotation.Threadsafe; -import com.linkedin.venice.pubsub.PubSubTopicPartitionInfo; -import com.linkedin.venice.pubsub.api.PubSubTopic; -import com.linkedin.venice.pubsub.api.PubSubTopicPartition; -import it.unimi.dsi.fastutil.ints.Int2LongMap; -import java.io.Closeable; -import java.util.List; - - -@Threadsafe -public interface PartitionOffsetFetcher extends Closeable { - Int2LongMap getTopicLatestOffsets(PubSubTopic topic); - - long getPartitionLatestOffsetAndRetry(PubSubTopicPartition pubSubTopicPartition, int retries); - - long getPartitionOffsetByTime(PubSubTopicPartition pubSubTopicPartition, long timestamp); - - /** - * Get the producer timestamp of the last data message (non-control message) in the given topic partition. In other - * words, if the last message in a topic partition is a control message, this method should keep looking at its previous - * message(s) until it finds one that is not a control message and gets its producer timestamp. - * @param pubSubTopicPartition - * @param retries - * @return producer timestamp - */ - long getProducerTimestampOfLastDataRecord(PubSubTopicPartition pubSubTopicPartition, int retries); - - List partitionsFor(PubSubTopic topic); - - long getOffsetByTimeIfOutOfRange(PubSubTopicPartition pubSubTopicPartition, long timestamp); - - @Override - void close(); -} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherFactory.java b/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherFactory.java deleted file mode 100644 index 9c40667341..0000000000 --- a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherFactory.java +++ /dev/null @@ -1,46 +0,0 @@ -package com.linkedin.venice.kafka.partitionoffset; - -import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; -import com.linkedin.venice.pubsub.PubSubConsumerAdapterFactory; -import com.linkedin.venice.pubsub.api.PubSubAdminAdapter; -import com.linkedin.venice.pubsub.api.PubSubMessageDeserializer; -import com.linkedin.venice.serialization.avro.KafkaValueSerializer; -import com.linkedin.venice.utils.SystemTime; -import com.linkedin.venice.utils.VeniceProperties; -import com.linkedin.venice.utils.lazy.Lazy; -import com.linkedin.venice.utils.pools.LandFillObjectPool; -import io.tehuti.metrics.MetricsRepository; -import java.util.Optional; - - -public class PartitionOffsetFetcherFactory { - public static PartitionOffsetFetcher createDefaultPartitionOffsetFetcher( - PubSubConsumerAdapterFactory pubSubConsumerAdapterFactory, - VeniceProperties veniceProperties, - String pubSubBootstrapServers, - Lazy kafkaAdminWrapper, - long kafkaOperationTimeoutMs, - Optional optionalMetricsRepository) { - PubSubMessageDeserializer pubSubMessageDeserializer = new PubSubMessageDeserializer( - new KafkaValueSerializer(), - new LandFillObjectPool<>(KafkaMessageEnvelope::new), - new LandFillObjectPool<>(KafkaMessageEnvelope::new)); - PartitionOffsetFetcher partitionOffsetFetcher = new PartitionOffsetFetcherImpl( - kafkaAdminWrapper, - Lazy.of( - () -> pubSubConsumerAdapterFactory - .create(veniceProperties, false, pubSubMessageDeserializer, pubSubBootstrapServers)), - kafkaOperationTimeoutMs, - pubSubBootstrapServers); - if (optionalMetricsRepository.isPresent()) { - return new InstrumentedPartitionOffsetFetcher( - partitionOffsetFetcher, - new PartitionOffsetFetcherStats( - optionalMetricsRepository.get(), - "PartitionOffsetFetcherStats_" + pubSubBootstrapServers), - new SystemTime()); - } else { - return partitionOffsetFetcher; - } - } -} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherImpl.java b/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherImpl.java deleted file mode 100644 index d288b7e3be..0000000000 --- a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherImpl.java +++ /dev/null @@ -1,514 +0,0 @@ -package com.linkedin.venice.kafka.partitionoffset; - -import static com.linkedin.venice.offsets.OffsetRecord.LOWEST_OFFSET; - -import com.linkedin.venice.exceptions.VeniceException; -import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; -import com.linkedin.venice.message.KafkaKey; -import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl; -import com.linkedin.venice.pubsub.PubSubTopicPartitionInfo; -import com.linkedin.venice.pubsub.api.PubSubAdminAdapter; -import com.linkedin.venice.pubsub.api.PubSubConsumerAdapter; -import com.linkedin.venice.pubsub.api.PubSubMessage; -import com.linkedin.venice.pubsub.api.PubSubTopic; -import com.linkedin.venice.pubsub.api.PubSubTopicPartition; -import com.linkedin.venice.pubsub.api.exceptions.PubSubClientRetriableException; -import com.linkedin.venice.pubsub.api.exceptions.PubSubOpTimeoutException; -import com.linkedin.venice.pubsub.api.exceptions.PubSubTopicDoesNotExistException; -import com.linkedin.venice.utils.RetryUtils; -import com.linkedin.venice.utils.Utils; -import com.linkedin.venice.utils.lazy.Lazy; -import com.linkedin.venice.utils.locks.AutoCloseableLock; -import it.unimi.dsi.fastutil.ints.Int2LongMap; -import it.unimi.dsi.fastutil.ints.Int2LongMaps; -import it.unimi.dsi.fastutil.ints.Int2LongOpenHashMap; -import java.time.Duration; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; -import java.util.stream.Collectors; -import javax.annotation.Nonnull; -import org.apache.commons.io.IOUtils; -import org.apache.commons.lang.Validate; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - - -public class PartitionOffsetFetcherImpl implements PartitionOffsetFetcher { - private static final List> PUBSUB_RETRIABLE_FAILURES = - Collections.singletonList(PubSubClientRetriableException.class); - public static final Duration DEFAULT_KAFKA_OFFSET_API_TIMEOUT = Duration.ofMinutes(1); - public static final long NO_PRODUCER_TIME_IN_EMPTY_TOPIC_PARTITION = -1; - private static final int KAFKA_POLLING_RETRY_MAX_ATTEMPT = 3; - - private final Logger logger; - private final Lock adminConsumerLock; - private final Lazy kafkaAdminWrapper; - private final Lazy pubSubConsumer; - private final Duration kafkaOperationTimeout; - - public PartitionOffsetFetcherImpl( - @Nonnull Lazy kafkaAdminWrapper, - @Nonnull Lazy pubSubConsumer, - long kafkaOperationTimeoutMs, - String kafkaBootstrapServers) { - Validate.notNull(kafkaAdminWrapper); - this.kafkaAdminWrapper = kafkaAdminWrapper; - this.pubSubConsumer = pubSubConsumer; - this.adminConsumerLock = new ReentrantLock(); - this.kafkaOperationTimeout = Duration.ofMillis(kafkaOperationTimeoutMs); - this.logger = - LogManager.getLogger(PartitionOffsetFetcherImpl.class.getSimpleName() + " [" + kafkaBootstrapServers + "]"); - } - - @Override - public Int2LongMap getTopicLatestOffsets(PubSubTopic topic) { - try (AutoCloseableLock ignore = AutoCloseableLock.of(adminConsumerLock)) { - List partitionInfoList = pubSubConsumer.get().partitionsFor(topic); - if (partitionInfoList == null || partitionInfoList.isEmpty()) { - logger.warn("Unexpected! Topic: {} has a null partition set, returning empty map for latest offsets", topic); - return Int2LongMaps.EMPTY_MAP; - } - List topicPartitions = partitionInfoList.stream() - .map(partitionInfo -> new PubSubTopicPartitionImpl(topic, partitionInfo.partition())) - .collect(Collectors.toList()); - - Map offsetsByTopicPartitions = - pubSubConsumer.get().endOffsets(topicPartitions, DEFAULT_KAFKA_OFFSET_API_TIMEOUT); - Int2LongMap offsetsByTopicPartitionIds = new Int2LongOpenHashMap(offsetsByTopicPartitions.size()); - for (Map.Entry offsetByTopicPartition: offsetsByTopicPartitions.entrySet()) { - offsetsByTopicPartitionIds - .put(offsetByTopicPartition.getKey().getPartitionNumber(), offsetByTopicPartition.getValue().longValue()); - } - return offsetsByTopicPartitionIds; - } - } - - private long getLatestOffset(PubSubTopicPartition pubSubTopicPartition) throws PubSubTopicDoesNotExistException { - if (pubSubTopicPartition.getPartitionNumber() < 0) { - throw new IllegalArgumentException( - "Cannot retrieve latest offsets for invalid partition " + pubSubTopicPartition.getPartitionNumber()); - } - try (AutoCloseableLock ignore = AutoCloseableLock.of(adminConsumerLock)) { - if (!kafkaAdminWrapper.get().containsTopicWithPartitionCheckExpectationAndRetry(pubSubTopicPartition, 3, true)) { - throw new PubSubTopicDoesNotExistException( - "Either topic: " + pubSubTopicPartition.getPubSubTopic() + " does not exist or partition: " - + pubSubTopicPartition.getPartitionNumber() + " is invalid"); - } - - Map offsetMap = pubSubConsumer.get() - .endOffsets(Collections.singletonList(pubSubTopicPartition), DEFAULT_KAFKA_OFFSET_API_TIMEOUT); - Long offset = offsetMap.get(pubSubTopicPartition); - if (offset != null) { - return offset; - } else { - throw new VeniceException( - "offset result returned from endOffsets does not contain entry: " + pubSubTopicPartition); - } - } - } - - @Override - public long getPartitionLatestOffsetAndRetry(PubSubTopicPartition pubSubTopicPartition, int retries) { - if (retries < 1) { - throw new IllegalArgumentException("Invalid retries. Got: " + retries); - } - int attempt = 0; - PubSubOpTimeoutException lastException = new PubSubOpTimeoutException("This exception should not be thrown"); - while (attempt < retries) { - try { - return getLatestOffset(pubSubTopicPartition); - } catch (PubSubOpTimeoutException e) { // topic and partition is listed in the exception object - logger.warn("Failed to get offset. Retries remaining: {}", retries - attempt, e); - lastException = e; - attempt++; - } - } - throw lastException; - } - - @Override - public long getPartitionOffsetByTime(PubSubTopicPartition pubSubTopicPartition, long timestamp) { - try (AutoCloseableLock ignore = AutoCloseableLock.of(adminConsumerLock)) { - Long result = offsetsForTimesWithRetry(pubSubTopicPartition, timestamp); - if (result == null) { - result = getOffsetByTimeIfOutOfRange(pubSubTopicPartition, timestamp); - } else if (result == -1L) { - // The given timestamp exceed the timestamp of the last message. So return the last offset. - logger.warn("Offsets result is empty. Will complement with the last offsets."); - result = endOffsetsWithRetry(pubSubTopicPartition) + 1; - } - return result; - } - } - - private Long offsetsForTimesWithRetry(PubSubTopicPartition pubSubTopicPartition, long timestamp) { - try (AutoCloseableLock ignore = AutoCloseableLock.of(adminConsumerLock)) { - Long topicPartitionOffset = RetryUtils.executeWithMaxAttemptAndExponentialBackoff( - () -> pubSubConsumer.get().offsetForTime(pubSubTopicPartition, timestamp, kafkaOperationTimeout), - 25, - Duration.ofMillis(100), - Duration.ofSeconds(5), - Duration.ofMinutes(1), - PUBSUB_RETRIABLE_FAILURES); - return topicPartitionOffset; - } - } - - private Long endOffsetsWithRetry(PubSubTopicPartition partition) { - try (AutoCloseableLock ignore = AutoCloseableLock.of(adminConsumerLock)) { - Long topicPartitionOffset = RetryUtils.executeWithMaxAttemptAndExponentialBackoff( - () -> pubSubConsumer.get().endOffset(partition), - 25, - Duration.ofMillis(100), - Duration.ofSeconds(5), - Duration.ofMinutes(1), - PUBSUB_RETRIABLE_FAILURES); - return topicPartitionOffset; - } - } - - @Override - public long getProducerTimestampOfLastDataRecord(PubSubTopicPartition pubSubTopicPartition, int retries) { - if (retries < 1) { - throw new IllegalArgumentException("Invalid retries. Got: " + retries); - } - int attempt = 0; - long timestamp; - PubSubOpTimeoutException lastException = new PubSubOpTimeoutException("This exception should not be thrown"); - while (attempt < retries) { - try { - timestamp = getProducerTimestampOfLastDataRecord(pubSubTopicPartition); - return timestamp; - } catch (PubSubOpTimeoutException e) {// topic and partition is listed in the exception object - logger.warn( - "Failed to get producer timestamp on the latest data record. Retries remaining: {}", - retries - attempt, - e); - lastException = e; - attempt++; - } - } - throw lastException; - } - - /** - * If the topic is empty or all the messages are truncated (startOffset==endOffset), return -1; - * otherwise, return the producer timestamp of the last message in the selected partition of a topic - */ - private long getProducerTimestampOfLastDataRecord(PubSubTopicPartition pubSubTopicPartition) - throws PubSubTopicDoesNotExistException { - List> lastConsumedRecords = - consumeLatestRecords(pubSubTopicPartition, 1); - if (lastConsumedRecords.isEmpty()) { - return NO_PRODUCER_TIME_IN_EMPTY_TOPIC_PARTITION; - } - - // Check the latest record as the first attempt - PubSubMessage lastRecord = lastConsumedRecords.iterator().next(); - if (!lastRecord.getKey().isControlMessage()) { - return lastRecord.getValue().producerMetadata.messageTimestamp; - } - - // Second attempt and read 60 records this time. There could be several control messages at the end of a RT - // partition - // if multiple Samza jobs write to this topic partition. 60 should be sufficient. - final int lastRecordsCount = 60; - logger.info( - "The last record in topic partition {} is a control message. Hence, try to find the " - + "last data record among the last {} records from the end of that partition", - pubSubTopicPartition, - lastRecordsCount); - - lastConsumedRecords = consumeLatestRecords(pubSubTopicPartition, lastRecordsCount); - if (lastConsumedRecords.isEmpty()) { - // Topic partition becomes empty and it can happen if the topic gets truncated after the first attempt and before - // this second attempt. This case should happen very rarely. - logger.warn( - "Second attempt to find producer timestamp from topic partition {} by consuming the last" - + " {} record(s) consumed no record. Assume the topic partition is empty.", - pubSubTopicPartition, - lastRecordsCount); - return NO_PRODUCER_TIME_IN_EMPTY_TOPIC_PARTITION; - } - Iterator> consumerRecordsIterator = - lastConsumedRecords.iterator(); - - Long latestDataRecordProducerTimestamp = null; - Long startOffset = null; - int recordsCount = 0; - - while (consumerRecordsIterator.hasNext()) { - PubSubMessage record = consumerRecordsIterator.next(); - startOffset = (startOffset == null ? record.getOffset() : startOffset); - recordsCount++; - - if (record.getKey().isControlMessage()) { - continue; - } - latestDataRecordProducerTimestamp = record.getValue().producerMetadata.messageTimestamp; - } - if (latestDataRecordProducerTimestamp == null) { - // It is quite impossible to have no data record in this many records from the end of that partition - throw new VeniceException( - String.format( - "Failed to find latest data record producer timestamp in topic partition %s " - + "since no data record is found in the last %d records starting from offset %d", - pubSubTopicPartition, - recordsCount, - startOffset)); - } - return latestDataRecordProducerTimestamp; - } - - /** - * This method retrieves last {@code lastRecordsCount} records from a topic partition and there are 4 steps below. - * 1. Find the current end offset N - * 2. Seek back {@code lastRecordsCount} records from the end offset N - * 3. Keep consuming records until the last consumed offset is greater than or equal to N - * 4. Return all consumed records - * - * There are 2 things to note: - * 1. When this method returns, these returned records are not necessarily the "last" records because after step 2, - * there could be more records produced to this topic partition and this method only consume records until the end - * offset retrieved at the above step 2. - * - * 2. This method might return more than {@code lastRecordsCount} records since the consumer poll method gets a batch - * of consumer records each time and the batch size is arbitrary. - */ - private List> consumeLatestRecords( - final PubSubTopicPartition pubSubTopicPartition, - final int lastRecordsCount) { - - if (pubSubTopicPartition.getPartitionNumber() < 0) { - throw new IllegalArgumentException( - "Cannot retrieve latest producer timestamp for invalid topic partition " + pubSubTopicPartition); - } - if (lastRecordsCount < 1) { - throw new IllegalArgumentException( - "Last record count must be greater than or equal to 1. Got: " + lastRecordsCount); - } - - try (AutoCloseableLock ignore = AutoCloseableLock.of(adminConsumerLock)) { - if (!kafkaAdminWrapper.get() - .containsTopicWithExpectationAndRetry(pubSubTopicPartition.getPubSubTopic(), 3, true)) { - throw new PubSubTopicDoesNotExistException(pubSubTopicPartition.getPubSubTopic()); - } - try { - Map offsetByTopicPartition = pubSubConsumer.get() - .endOffsets(Collections.singletonList(pubSubTopicPartition), DEFAULT_KAFKA_OFFSET_API_TIMEOUT); - if (offsetByTopicPartition == null || !offsetByTopicPartition.containsKey(pubSubTopicPartition)) { - throw new VeniceException( - "Got no results of finding end offsets for topic partition: " + pubSubTopicPartition); - } - final long latestOffset = offsetByTopicPartition.get(pubSubTopicPartition); - - if (latestOffset <= 0) { - // Empty topic - return Collections.emptyList(); - } else { - Long earliestOffset = - pubSubConsumer.get().beginningOffset(pubSubTopicPartition, DEFAULT_KAFKA_OFFSET_API_TIMEOUT); - if (earliestOffset == null) { - throw new VeniceException( - "Got no results of finding the earliest offset for topic partition: " + pubSubTopicPartition); - } - if (earliestOffset == latestOffset) { - // Empty topic - return Collections.emptyList(); - } else { - // poll the last message and retrieve the producer timestamp - final long startConsumeOffset = Math.max(latestOffset - lastRecordsCount, earliestOffset); - pubSubConsumer.get().subscribe(pubSubTopicPartition, startConsumeOffset - 1); - List> allConsumedRecords = - new ArrayList<>(lastRecordsCount); - - // Keep consuming records from that topic partition until the last consumed record's offset is greater or - // equal - // to the partition end offset retrieved before. - do { - List> oneBatchConsumedRecords = - Collections.emptyList(); - int currAttempt = 0; - - while (currAttempt++ < KAFKA_POLLING_RETRY_MAX_ATTEMPT && oneBatchConsumedRecords.isEmpty()) { - logger.info( - "Trying to get records from topic partition {} from offset {} to its log end " - + "offset. Attempt# {} / {}", - pubSubTopicPartition, - startConsumeOffset, - currAttempt, - KAFKA_POLLING_RETRY_MAX_ATTEMPT); - - oneBatchConsumedRecords = - pubSubConsumer.get().poll(kafkaOperationTimeout.toMillis()).get(pubSubTopicPartition); - } - if (oneBatchConsumedRecords.isEmpty()) { - /** - * Failed the job if we cannot get the last offset of the topic. - */ - String errorMsg = "Failed to get the last record from topic-partition: " + pubSubTopicPartition - + " after " + KAFKA_POLLING_RETRY_MAX_ATTEMPT + " attempts"; - logger.error(errorMsg); - throw new VeniceException(errorMsg); - } - logger.info( - "Consumed {} record(s) from topic partition {}", - oneBatchConsumedRecords.size(), - pubSubTopicPartition); - - allConsumedRecords.addAll(oneBatchConsumedRecords); - } while (allConsumedRecords.get(allConsumedRecords.size() - 1).getOffset() + 1 < latestOffset); - - return allConsumedRecords; - } - } - } finally { - pubSubConsumer.get().unSubscribe(pubSubTopicPartition); - } - } - } - - @Override - public List partitionsFor(PubSubTopic topic) { - try (AutoCloseableLock ignore = AutoCloseableLock.of(adminConsumerLock)) { - return pubSubConsumer.get().partitionsFor(topic); - } - } - - @Override - public long getOffsetByTimeIfOutOfRange(PubSubTopicPartition pubSubTopicPartition, long timestamp) - throws PubSubTopicDoesNotExistException { - try (AutoCloseableLock ignore = AutoCloseableLock.of(adminConsumerLock)) { - long latestOffset = getLatestOffset(pubSubTopicPartition); - if (latestOffset <= 0) { - long nextOffset = LOWEST_OFFSET + 1; - logger.info("End offset for topic {} is {}; return offset {}", pubSubTopicPartition, latestOffset, nextOffset); - return nextOffset; - } - - long earliestOffset = getEarliestOffset(pubSubTopicPartition); - if (earliestOffset == latestOffset) { - /** - * This topic/partition is empty or retention delete the entire partition - */ - logger.info( - "Both beginning offset and end offset is {} for topic {}; it's empty; return offset {}", - latestOffset, - pubSubTopicPartition, - latestOffset); - return latestOffset; - } - - try { - pubSubConsumer.get().subscribe(pubSubTopicPartition, latestOffset - 2); - Map>> records = new HashMap<>(); - /** - * We should retry to get the last record from that topic/partition, never return 0L here because 0L offset - * will result in replaying all the messages in real-time buffer. This function is mainly used during buffer - * replay for hybrid stores. - */ - int attempts = 0; - while (attempts++ < KAFKA_POLLING_RETRY_MAX_ATTEMPT && records.isEmpty()) { - logger.info( - "Trying to get the last record from topic: {} at offset: {}. Attempt#{}/{}", - pubSubTopicPartition, - latestOffset - 1, - attempts, - KAFKA_POLLING_RETRY_MAX_ATTEMPT); - records = pubSubConsumer.get().poll(kafkaOperationTimeout.toMillis()); - } - if (records.isEmpty()) { - /** - * Failed the job if we cannot get the last offset of the topic. - */ - String errorMsg = "Failed to get the last record from topic: " + pubSubTopicPartition + " after " - + KAFKA_POLLING_RETRY_MAX_ATTEMPT + " attempts"; - logger.error(errorMsg); - throw new VeniceException(errorMsg); - } - - // Get the latest record from the poll result - - PubSubMessage record = Utils.iterateOnMapOfLists(records).next(); - - if (timestamp <= record.getPubSubMessageTime()) { - /** - * There could be a race condition in this function: - * 1. In function: {@link #getPartitionsOffsetsByTime}, {@link Consumer#offsetsForTimes} is invoked. - * 2. The asked timestamp is out of range. - * 3. Some messages get produced to the topic. - * 4. {@link #getOffsetByTimeIfOutOfRange} gets invoked, and it realizes that the latest message's timestamp - * is higher than the seeking timestamp. - * - * In this case, we should call {@link #getPartitionOffsetByTime} again, since the seeking timestamp will be in the range - * instead of returning the earliest offset. - */ - Long result = pubSubConsumer.get().offsetForTime(pubSubTopicPartition, timestamp); - if (result != null && result != -1L) { - logger.info( - "Successfully return offset: {} for topic: {} for timestamp: {}", - result, - pubSubTopicPartition, - timestamp); - return result; - } - } - - /** - * 1. If the required timestamp is bigger than the timestamp of last record, return the offset after the last record. - * 2. Otherwise, return earlier offset to consume from the beginning. - */ - long resultOffset = (timestamp > record.getPubSubMessageTime()) ? latestOffset : earliestOffset; - logger.info( - "Successfully return offset: {} for topic: {} for timestamp: {}", - resultOffset, - pubSubTopicPartition, - timestamp); - return resultOffset; - } finally { - pubSubConsumer.get().unSubscribe(pubSubTopicPartition); - } - } - } - - /** - * @return the beginning offset of a topic/partition. Synchronized because it calls #getConsumer() - */ - private long getEarliestOffset(PubSubTopicPartition pubSubTopicPartition) throws PubSubTopicDoesNotExistException { - try (AutoCloseableLock ignore = AutoCloseableLock.of(adminConsumerLock)) { - if (!kafkaAdminWrapper.get() - .containsTopicWithExpectationAndRetry(pubSubTopicPartition.getPubSubTopic(), 3, true)) { - throw new PubSubTopicDoesNotExistException( - "Topic " + pubSubTopicPartition.getPubSubTopic() + " does not exist!"); - } - if (pubSubTopicPartition.getPartitionNumber() < 0) { - throw new IllegalArgumentException( - "Cannot retrieve latest offsets for invalid partition " + pubSubTopicPartition.getPartitionNumber()); - } - Long offset = pubSubConsumer.get().beginningOffset(pubSubTopicPartition, DEFAULT_KAFKA_OFFSET_API_TIMEOUT); - if (offset == null) { - throw new VeniceException( - "offset result returned from beginningOffsets does not contain entry: " + pubSubTopicPartition); - } - return offset; - } - } - - @Override - public void close() { - if (kafkaAdminWrapper.isPresent()) { - IOUtils.closeQuietly(kafkaAdminWrapper.get(), logger::error); - } - if (pubSubConsumer.isPresent()) { - try (AutoCloseableLock ignore = AutoCloseableLock.of(adminConsumerLock)) { - IOUtils.closeQuietly(pubSubConsumer.get(), logger::error); - } - } - } -} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherStats.java b/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherStats.java deleted file mode 100644 index 851638b4e8..0000000000 --- a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherStats.java +++ /dev/null @@ -1,55 +0,0 @@ -package com.linkedin.venice.kafka.partitionoffset; - -import com.linkedin.venice.stats.AbstractVeniceStats; -import com.linkedin.venice.stats.TehutiUtils; -import io.tehuti.metrics.MetricsRepository; -import io.tehuti.metrics.Sensor; -import io.tehuti.metrics.stats.Avg; -import io.tehuti.metrics.stats.Max; -import io.tehuti.metrics.stats.Min; -import io.tehuti.metrics.stats.OccurrenceRate; -import java.util.Collections; -import java.util.EnumMap; -import java.util.Map; - - -public class PartitionOffsetFetcherStats extends AbstractVeniceStats { - public enum OCCURRENCE_LATENCY_SENSOR_TYPE { - GET_TOPIC_LATEST_OFFSETS, GET_PARTITION_LATEST_OFFSET_WITH_RETRY, GET_PARTITIONS_OFFSETS_BY_TIME, - GET_PARTITION_OFFSET_BY_TIME, GET_LATEST_PRODUCER_TIMESTAMP_ON_DATA_RECORD_WITH_RETRY, PARTITIONS_FOR, - GET_PARTITION_OFFSET_BY_TIME_IF_OUT_OF_RANGE - } - - private final Map sensorsByTypes; - private Sensor getPartitionLatestOffsetError; - - public PartitionOffsetFetcherStats(MetricsRepository metricsRepository, String name) { - super(metricsRepository, name); - Map tmpRateSensorsByTypes = - new EnumMap<>(OCCURRENCE_LATENCY_SENSOR_TYPE.class); - for (OCCURRENCE_LATENCY_SENSOR_TYPE sensorType: OCCURRENCE_LATENCY_SENSOR_TYPE.values()) { - final String sensorName = sensorType.name().toLowerCase(); - tmpRateSensorsByTypes.put( - sensorType, - registerSensorIfAbsent( - sensorName, - new OccurrenceRate(), - new Max(), - new Min(), - new Avg(), - TehutiUtils.getPercentileStat(getName() + AbstractVeniceStats.DELIMITER + sensorName))); - } - - this.sensorsByTypes = Collections.unmodifiableMap(tmpRateSensorsByTypes); - this.getPartitionLatestOffsetError = - registerSensorIfAbsent("get_partition_latest_offset_with_retry_error", new OccurrenceRate()); - } - - public void recordLatency(OCCURRENCE_LATENCY_SENSOR_TYPE sensor_type, long requestLatencyMs) { - sensorsByTypes.get(sensor_type).record(requestLatencyMs); - } - - public void recordGetLatestOffsetError() { - this.getPartitionLatestOffsetError.record(); - } -} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/meta/ZKStore.java b/internal/venice-common/src/main/java/com/linkedin/venice/meta/ZKStore.java index 9e64d4829d..edd7f3ee46 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/meta/ZKStore.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/meta/ZKStore.java @@ -5,11 +5,11 @@ import com.linkedin.venice.compression.CompressionStrategy; import com.linkedin.venice.exceptions.StoreDisabledException; import com.linkedin.venice.exceptions.VeniceException; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.systemstore.schemas.StoreProperties; import com.linkedin.venice.systemstore.schemas.StoreVersion; import com.linkedin.venice.utils.AvroCompatibilityUtils; import com.linkedin.venice.utils.AvroRecordUtils; +import com.linkedin.venice.utils.StoreUtils; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -742,7 +742,7 @@ public void setBackupVersionRetentionMs(long backupVersionRetentionMs) { public long getRetentionTime() { HybridStoreConfig config = this.getHybridStoreConfig(); if (config != null) { - return TopicManager.getExpectedRetentionTimeInMs(this, config); + return StoreUtils.getExpectedRetentionTimeInMs(this, config); } else { return DEFAULT_RT_RETENTION_TIME; } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/PubSubConstants.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/PubSubConstants.java index f857e74731..e2621d82c0 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/PubSubConstants.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/PubSubConstants.java @@ -1,5 +1,14 @@ package com.linkedin.venice.pubsub; +import com.linkedin.venice.pubsub.api.exceptions.PubSubClientRetriableException; +import com.linkedin.venice.pubsub.api.exceptions.PubSubOpTimeoutException; +import com.linkedin.venice.utils.Time; +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + + /** * Constants used by pub-sub components. */ @@ -21,8 +30,11 @@ public class PubSubConstants { public static final String PUBSUB_CONSUMER_POLL_RETRY_BACKOFF_MS = "pubsub.consumer.poll.retry.backoff.ms"; public static final int PUBSUB_CONSUMER_POLL_RETRY_BACKOFF_MS_DEFAULT_VALUE = 0; + public static final String PUBSUB_CONSUMER_POSITION_RESET_STRATEGY = "pubsub.consumer.position.reset.strategy"; + public static final String PUBSUB_CONSUMER_POSITION_RESET_STRATEGY_DEFAULT_VALUE = "earliest"; + public static final long PUBSUB_ADMIN_GET_TOPIC_CONFIG_RETRY_IN_SECONDS_DEFAULT_VALUE = 300; - public static final long UNKNOWN_TOPIC_RETENTION = Long.MIN_VALUE; + public static final long PUBSUB_TOPIC_UNKNOWN_RETENTION = Long.MIN_VALUE; public static final String PUBSUB_CONSUMER_TOPIC_QUERY_RETRY_TIMES = "pubsub.consumer.topic.query.retry.times"; public static final int PUBSUB_CONSUMER_TOPIC_QUERY_RETRY_TIMES_DEFAULT_VALUE = 5; @@ -37,4 +49,42 @@ public class PubSubConstants { public static final String PUBSUB_CONSUMER_CHECK_TOPIC_EXISTENCE = "pubsub.consumer.check.topic.existence"; public static final boolean PUBSUB_CONSUMER_CHECK_TOPIC_EXISTENCE_DEFAULT_VALUE = false; + + /** + * Default setting is that no log compaction should happen for hybrid store version topics + * if the messages are produced within 24 hours; otherwise servers could encounter MISSING + * data DIV errors for reprocessing jobs which could potentially generate lots of + * duplicate keys. + */ + public static final long DEFAULT_KAFKA_MIN_LOG_COMPACTION_LAG_MS = 24 * Time.MS_PER_HOUR; + + public static final List> CREATE_TOPIC_RETRIABLE_EXCEPTIONS = + Collections.unmodifiableList(Arrays.asList(PubSubOpTimeoutException.class, PubSubClientRetriableException.class)); + + /** + * Default value of sleep interval for polling topic deletion status from ZK. + */ + public static final int PUBSUB_TOPIC_DELETION_STATUS_POLL_INTERVAL_MS_DEFAULT_VALUE = 2 * Time.MS_PER_SECOND; + + public static final Duration PUBSUB_OFFSET_API_TIMEOUT_DURATION_DEFAULT_VALUE = Duration.ofMinutes(1); + + public static final long PUBSUB_NO_PRODUCER_TIME_IN_EMPTY_TOPIC_PARTITION = -1; + + public static final int PUBSUB_CONSUMER_POLLING_FOR_METADATA_RETRY_MAX_ATTEMPT = 3; + + public static final int PUBSUB_TOPIC_MANAGER_METADATA_FETCHER_CONSUMER_POOL_SIZE_DEFAULT_VALUE = 2; + + public static final int PUBSUB_FAST_OPERATION_TIMEOUT_MS = Time.MS_PER_SECOND; + + public static final int PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE = 30 * Time.MS_PER_SECOND; + + public static final int PUBSUB_TOPIC_DELETE_RETRY_TIMES = 3; + + public static final long ETERNAL_TOPIC_RETENTION_POLICY_MS = Long.MAX_VALUE; + + public static final long DEFAULT_TOPIC_RETENTION_POLICY_MS = 5 * Time.MS_PER_DAY; + + public static final long BUFFER_REPLAY_MINIMAL_SAFETY_MARGIN = 2 * Time.MS_PER_DAY; + + public static final int DEFAULT_KAFKA_REPLICATION_FACTOR = 3; } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/PubSubTopicPartitionInfo.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/PubSubTopicPartitionInfo.java index fb26021e50..adaf858f9e 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/PubSubTopicPartitionInfo.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/PubSubTopicPartitionInfo.java @@ -1,37 +1,40 @@ package com.linkedin.venice.pubsub; import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.api.PubSubTopicPartition; public class PubSubTopicPartitionInfo { - private final PubSubTopic pubSubTopic; - private final int partition; + private final PubSubTopicPartition topicPartition; private final Boolean hasInSyncReplica; public PubSubTopicPartitionInfo(PubSubTopic pubSubTopic, int partition, Boolean hasInSyncReplica) { - this.pubSubTopic = pubSubTopic; - this.partition = partition; + this.topicPartition = new PubSubTopicPartitionImpl(pubSubTopic, partition); this.hasInSyncReplica = hasInSyncReplica; } public PubSubTopic topic() { - return pubSubTopic; + return topicPartition.getPubSubTopic(); } /** * The partition id */ public int partition() { - return partition; + return topicPartition.getPartitionNumber(); } public Boolean hasInSyncReplicas() { return hasInSyncReplica; } + public PubSubTopicPartition getTopicPartition() { + return topicPartition; + } + @Override public String toString() { return String - .format("Partition(topic = %s, partition=%s, hasInSyncReplica = %s)", pubSubTopic, partition, hasInSyncReplica); + .format("Partition(topic = %s, partition=%s, hasInSyncReplica = %s)", topic(), partition(), hasInSyncReplica); } } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/admin/ApacheKafkaAdminAdapter.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/admin/ApacheKafkaAdminAdapter.java index 9c831854e6..313f05dc57 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/admin/ApacheKafkaAdminAdapter.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/admin/ApacheKafkaAdminAdapter.java @@ -447,7 +447,7 @@ private Properties unmarshallProperties(PubSubTopicConfiguration pubSubTopicConf * Retrieves the retention settings for all Kafka topics. * * @return A map of Kafka topics and their corresponding retention settings in milliseconds. - * If a topic does not have a retention setting, it will be mapped to {@link PubSubConstants#UNKNOWN_TOPIC_RETENTION}. + * If a topic does not have a retention setting, it will be mapped to {@link PubSubConstants#PUBSUB_TOPIC_UNKNOWN_RETENTION}. * @throws PubSubClientRetriableException If a retriable error occurs while attempting to retrieve retention settings. * @throws PubSubClientException If an error occurs while attempting to retrieve retention settings or if the current thread is interrupted while attempting to retrieve retention settings. */ @@ -458,7 +458,7 @@ public Map getAllTopicRetentions() { // Option A: perform a string-to-long conversion if it's present... .map(configEntry -> Long.parseLong(configEntry.value())) // Option B: ... or default to a sentinel value if it's missing - .orElse(PubSubConstants.UNKNOWN_TOPIC_RETENTION), + .orElse(PubSubConstants.PUBSUB_TOPIC_UNKNOWN_RETENTION), "retention"); } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/consumer/ApacheKafkaConsumerAdapter.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/consumer/ApacheKafkaConsumerAdapter.java index 41023428e5..2d050ba622 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/consumer/ApacheKafkaConsumerAdapter.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/consumer/ApacheKafkaConsumerAdapter.java @@ -116,16 +116,25 @@ public void subscribe(PubSubTopicPartition pubSubTopicPartition, long lastReadOf topicPartitionList.add(topicPartition); kafkaConsumer.assign(topicPartitionList); // add the topic-partition to the subscription // Use the last read offset to seek to the next offset to read. - long consumptionStartOffset = lastReadOffset == OffsetRecord.LOWEST_OFFSET ? 0 : lastReadOffset + 1; - if (lastReadOffset == OffsetRecord.LOWEST_OFFSET) { + long consumptionStartOffset = lastReadOffset <= OffsetRecord.LOWEST_OFFSET ? 0 : lastReadOffset + 1; + if (lastReadOffset <= OffsetRecord.LOWEST_OFFSET) { + if (lastReadOffset < OffsetRecord.LOWEST_OFFSET) { + LOGGER.warn( + "Last read offset: {} for topic-partition: {} is less than the lowest offset: {}, seeking to beginning." + + " This may indicate an off-by-one error.", + lastReadOffset, + pubSubTopicPartition, + OffsetRecord.LOWEST_OFFSET); + } kafkaConsumer.seekToBeginning(Collections.singletonList(topicPartition)); } else { kafkaConsumer.seek(topicPartition, consumptionStartOffset); } assignments.put(topicPartition, pubSubTopicPartition); LOGGER.info( - "Subscribed to topic-partition: {} with consumptionStartOffset: {}", + "Subscribed to topic-partition: {} at offset: {} and last read offset was: {}", pubSubTopicPartition, + consumptionStartOffset, lastReadOffset); } @@ -230,19 +239,18 @@ public Map records = ConsumerRecords.empty(); Map>> polledPubSubMessages = - new HashMap<>(); + Collections.emptyMap(); while (attemptCount <= config.getConsumerPollRetryTimes() && !Thread.currentThread().isInterrupted()) { try { records = kafkaConsumer.poll(Duration.ofMillis(timeoutMs)); + polledPubSubMessages = new HashMap<>(records.partitions().size()); for (TopicPartition topicPartition: records.partitions()) { PubSubTopicPartition pubSubTopicPartition = assignments.get(topicPartition); List> topicPartitionConsumerRecords = records.records(topicPartition); List> topicPartitionPubSubMessages = new ArrayList<>(topicPartitionConsumerRecords.size()); for (ConsumerRecord consumerRecord: topicPartitionConsumerRecords) { - PubSubMessage pubSubMessage = - deserialize(consumerRecord, pubSubTopicPartition); - topicPartitionPubSubMessages.add(pubSubMessage); + topicPartitionPubSubMessages.add(deserialize(consumerRecord, pubSubTopicPartition)); } polledPubSubMessages.put(pubSubTopicPartition, topicPartitionPubSubMessages); } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/consumer/ApacheKafkaConsumerConfig.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/consumer/ApacheKafkaConsumerConfig.java index 0d45bb267c..eb851e533a 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/consumer/ApacheKafkaConsumerConfig.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/adapter/kafka/consumer/ApacheKafkaConsumerConfig.java @@ -1,5 +1,6 @@ package com.linkedin.venice.pubsub.adapter.kafka.consumer; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_CONSUMER_POSITION_RESET_STRATEGY; import static com.linkedin.venice.pubsub.adapter.kafka.producer.ApacheKafkaProducerConfig.KAFKA_CONFIG_PREFIX; import com.linkedin.venice.pubsub.PubSubConstants; @@ -58,6 +59,13 @@ public class ApacheKafkaConsumerConfig { consumerProperties.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, DEFAULT_RECEIVE_BUFFER_SIZE); } + if (!consumerProperties.containsKey(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG) + && veniceProperties.containsKey(PUBSUB_CONSUMER_POSITION_RESET_STRATEGY)) { + consumerProperties.put( + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, + veniceProperties.getString(PUBSUB_CONSUMER_POSITION_RESET_STRATEGY)); + } + // Do not change the default value of the following two configs unless you know what you are doing. consumerProperties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class); consumerProperties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class); diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubAdminAdapter.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubAdminAdapter.java index 404a6d44f0..455c8f3b6f 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubAdminAdapter.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubAdminAdapter.java @@ -227,7 +227,7 @@ default List> getRetriableExceptions() { * Retrieves the retention settings for all PubSub topics. * * @return A map of pub-sub topics and their corresponding retention settings in milliseconds. - * If a topic does not have a retention setting, it will be mapped to {@link PubSubConstants#UNKNOWN_TOPIC_RETENTION}. + * If a topic does not have a retention setting, it will be mapped to {@link PubSubConstants#PUBSUB_TOPIC_UNKNOWN_RETENTION}. * @throws PubSubClientRetriableException If a retriable error occurs while attempting to retrieve retention settings. * @throws PubSubClientException If an error occurs while attempting to retrieve retention settings or if the current thread is interrupted while attempting to retrieve retention settings. */ diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubConsumerAdapter.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubConsumerAdapter.java index 1a5e08ebb5..d6c1ba5908 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubConsumerAdapter.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubConsumerAdapter.java @@ -120,9 +120,6 @@ public interface PubSubConsumerAdapter extends AutoCloseable, Closeable { * Retrieves the consuming offset lag for a PubSub topic partition. The offset lag represents the difference * between the last consumed message offset and the latest available message offset for the partition. * - * This is a best-effort approach that typically uses metrics-based lag computation to reduce the number - * of calls to the PubSub system. - * * @param pubSubTopicPartition The PubSub topic partition for which to fetch the offset lag. * @return The offset lag, which is zero or a positive value if a valid lag was collected by the consumer, * or -1 if the lag cannot be determined or is not applicable. @@ -132,8 +129,7 @@ default long getOffsetLag(PubSubTopicPartition pubSubTopicPartition) { } /** - * Retrieves the latest available offset for a PubSub topic partition. This is a best-effort approach - * that typically uses metrics-based end offset to reduce the number of calls to the PubSub system. + * Retrieves the latest available offset for a PubSub topic partition. * * @param pubSubTopicPartition The PubSub topic partition for which to fetch the latest offset. * @return The latest offset, which is zero or a positive value if an offset was collected by the consumer, diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubInstrumentedAdminAdapter.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubInstrumentedAdminAdapter.java deleted file mode 100644 index 4ebb5a5ce1..0000000000 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/api/PubSubInstrumentedAdminAdapter.java +++ /dev/null @@ -1,166 +0,0 @@ -package com.linkedin.venice.pubsub.api; - -import static com.linkedin.venice.stats.PubSubAdminWrapperStats.OCCURRENCE_LATENCY_SENSOR_TYPE.CLOSE; -import static com.linkedin.venice.stats.PubSubAdminWrapperStats.OCCURRENCE_LATENCY_SENSOR_TYPE.CONTAINS_TOPIC; -import static com.linkedin.venice.stats.PubSubAdminWrapperStats.OCCURRENCE_LATENCY_SENSOR_TYPE.CONTAINS_TOPIC_WITH_RETRY; -import static com.linkedin.venice.stats.PubSubAdminWrapperStats.OCCURRENCE_LATENCY_SENSOR_TYPE.CREATE_TOPIC; -import static com.linkedin.venice.stats.PubSubAdminWrapperStats.OCCURRENCE_LATENCY_SENSOR_TYPE.DELETE_TOPIC; -import static com.linkedin.venice.stats.PubSubAdminWrapperStats.OCCURRENCE_LATENCY_SENSOR_TYPE.GET_ALL_TOPIC_RETENTIONS; -import static com.linkedin.venice.stats.PubSubAdminWrapperStats.OCCURRENCE_LATENCY_SENSOR_TYPE.GET_SOME_TOPIC_CONFIGS; -import static com.linkedin.venice.stats.PubSubAdminWrapperStats.OCCURRENCE_LATENCY_SENSOR_TYPE.GET_TOPIC_CONFIG; -import static com.linkedin.venice.stats.PubSubAdminWrapperStats.OCCURRENCE_LATENCY_SENSOR_TYPE.GET_TOPIC_CONFIG_WITH_RETRY; -import static com.linkedin.venice.stats.PubSubAdminWrapperStats.OCCURRENCE_LATENCY_SENSOR_TYPE.LIST_ALL_TOPICS; -import static com.linkedin.venice.stats.PubSubAdminWrapperStats.OCCURRENCE_LATENCY_SENSOR_TYPE.SET_TOPIC_CONFIG; - -import com.linkedin.venice.pubsub.PubSubTopicConfiguration; -import com.linkedin.venice.pubsub.api.exceptions.PubSubTopicDoesNotExistException; -import com.linkedin.venice.stats.PubSubAdminWrapperStats; -import com.linkedin.venice.utils.SystemTime; -import com.linkedin.venice.utils.Time; -import com.linkedin.venice.utils.Utils; -import io.tehuti.metrics.MetricsRepository; -import java.io.IOException; -import java.time.Duration; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.function.Supplier; -import javax.annotation.Nonnull; -import org.apache.commons.lang.Validate; - - -/** - * This class delegates another {@link PubSubAdminAdapter} instance and keeps track of the invocation rate of methods - * on the delegated instance - */ -public class PubSubInstrumentedAdminAdapter implements PubSubAdminAdapter { - private final PubSubAdminAdapter pubSubAdminAdapter; - private final PubSubAdminWrapperStats pubSubAdminWrapperStats; - private final Time time; - - public PubSubInstrumentedAdminAdapter( - PubSubAdminAdapter pubSubAdminAdapter, - MetricsRepository metricsRepository, - String statsName) { - this(pubSubAdminAdapter, metricsRepository, statsName, new SystemTime()); - } - - public PubSubInstrumentedAdminAdapter( - @Nonnull PubSubAdminAdapter pubSubAdminAdapter, - @Nonnull MetricsRepository metricsRepository, - @Nonnull String statsName, - @Nonnull Time time) { - Validate.notNull(pubSubAdminAdapter); - Validate.notNull(metricsRepository); - Validate.notEmpty(statsName); - Validate.notNull(time); - this.pubSubAdminAdapter = pubSubAdminAdapter; - this.time = time; - this.pubSubAdminWrapperStats = PubSubAdminWrapperStats.getInstance(metricsRepository, statsName); - } - - @Override - public void createTopic( - PubSubTopic topicName, - int numPartitions, - int replication, - PubSubTopicConfiguration pubSubTopicConfiguration) { - instrument(CREATE_TOPIC, () -> { - pubSubAdminAdapter.createTopic(topicName, numPartitions, replication, pubSubTopicConfiguration); - return null; - }); - } - - @Override - public void deleteTopic(PubSubTopic topicName, Duration timeout) { - instrument(DELETE_TOPIC, () -> { - pubSubAdminAdapter.deleteTopic(topicName, timeout); - return null; - }); - } - - @Override - public Set listAllTopics() { - return instrument(LIST_ALL_TOPICS, () -> pubSubAdminAdapter.listAllTopics()); - } - - @Override - public void setTopicConfig(PubSubTopic topicName, PubSubTopicConfiguration pubSubTopicConfiguration) { - instrument(SET_TOPIC_CONFIG, () -> { - pubSubAdminAdapter.setTopicConfig(topicName, pubSubTopicConfiguration); - return null; - }); - } - - @Override - public Map getAllTopicRetentions() { - return instrument(GET_ALL_TOPIC_RETENTIONS, () -> pubSubAdminAdapter.getAllTopicRetentions()); - } - - @Override - public PubSubTopicConfiguration getTopicConfig(PubSubTopic topicName) throws PubSubTopicDoesNotExistException { - return instrument(GET_TOPIC_CONFIG, () -> pubSubAdminAdapter.getTopicConfig(topicName)); - } - - @Override - public PubSubTopicConfiguration getTopicConfigWithRetry(PubSubTopic pubSubTopic) { - return instrument(GET_TOPIC_CONFIG_WITH_RETRY, () -> pubSubAdminAdapter.getTopicConfigWithRetry(pubSubTopic)); - } - - @Override - public boolean containsTopic(PubSubTopic topic) { - return instrument(CONTAINS_TOPIC, () -> pubSubAdminAdapter.containsTopic(topic)); - } - - @Override - public boolean containsTopicWithPartitionCheck(PubSubTopicPartition pubSubTopicPartition) { - return instrument(CONTAINS_TOPIC, () -> pubSubAdminAdapter.containsTopicWithPartitionCheck(pubSubTopicPartition)); - } - - @Override - public boolean containsTopicWithExpectationAndRetry(PubSubTopic topic, int maxRetries, final boolean expectedResult) { - return instrument( - CONTAINS_TOPIC_WITH_RETRY, - () -> pubSubAdminAdapter.containsTopicWithExpectationAndRetry(topic, maxRetries, expectedResult)); - } - - @Override - public List> getRetriableExceptions() { - return pubSubAdminAdapter.getRetriableExceptions(); - } - - @Override - public long getTopicConfigMaxRetryInMs() { - return pubSubAdminAdapter.getTopicConfigMaxRetryInMs(); - } - - @Override - public Map getSomeTopicConfigs(Set topicNames) { - return instrument(GET_SOME_TOPIC_CONFIGS, () -> pubSubAdminAdapter.getSomeTopicConfigs(topicNames)); - } - - @Override - public void close() throws IOException { - instrument(CLOSE, () -> { - Utils.closeQuietlyWithErrorLogged(pubSubAdminAdapter); - return null; - }); - } - - @Override - public String getClassName() { - return String.format( - "%s delegated by %s", - pubSubAdminAdapter.getClassName(), - PubSubInstrumentedAdminAdapter.class.getName()); - } - - private T instrument( - PubSubAdminWrapperStats.OCCURRENCE_LATENCY_SENSOR_TYPE type, - Supplier functionToInstrument) { - final long startTimeMs = time.getMilliseconds(); - final T res = functionToInstrument.get(); - pubSubAdminWrapperStats.recordLatency(type, Utils.calculateDurationMs(time, startTimeMs)); - return res; - } -} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/TopicManager.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java similarity index 57% rename from internal/venice-common/src/main/java/com/linkedin/venice/kafka/TopicManager.java rename to internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java index 2b2bc57359..21b4bd5bdd 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/TopicManager.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java @@ -1,20 +1,31 @@ -package com.linkedin.venice.kafka; +package com.linkedin.venice.pubsub.manager; + +import static com.linkedin.venice.pubsub.PubSubConstants.CREATE_TOPIC_RETRIABLE_EXCEPTIONS; +import static com.linkedin.venice.pubsub.PubSubConstants.DEFAULT_TOPIC_RETENTION_POLICY_MS; +import static com.linkedin.venice.pubsub.PubSubConstants.ETERNAL_TOPIC_RETENTION_POLICY_MS; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_FAST_OPERATION_TIMEOUT_MS; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_TOPIC_DELETE_RETRY_TIMES; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_TOPIC_UNKNOWN_RETENTION; +import static com.linkedin.venice.pubsub.manager.TopicManagerStats.SENSOR_TYPE.CONTAINS_TOPIC_WITH_RETRY; +import static com.linkedin.venice.pubsub.manager.TopicManagerStats.SENSOR_TYPE.CREATE_TOPIC; +import static com.linkedin.venice.pubsub.manager.TopicManagerStats.SENSOR_TYPE.DELETE_TOPIC; +import static com.linkedin.venice.pubsub.manager.TopicManagerStats.SENSOR_TYPE.GET_ALL_TOPIC_RETENTIONS; +import static com.linkedin.venice.pubsub.manager.TopicManagerStats.SENSOR_TYPE.GET_SOME_TOPIC_CONFIGS; +import static com.linkedin.venice.pubsub.manager.TopicManagerStats.SENSOR_TYPE.GET_TOPIC_CONFIG; +import static com.linkedin.venice.pubsub.manager.TopicManagerStats.SENSOR_TYPE.GET_TOPIC_CONFIG_WITH_RETRY; +import static com.linkedin.venice.pubsub.manager.TopicManagerStats.SENSOR_TYPE.LIST_ALL_TOPICS; +import static com.linkedin.venice.pubsub.manager.TopicManagerStats.SENSOR_TYPE.SET_TOPIC_CONFIG; import com.github.benmanes.caffeine.cache.Cache; import com.github.benmanes.caffeine.cache.Caffeine; import com.linkedin.venice.exceptions.VeniceException; -import com.linkedin.venice.kafka.partitionoffset.PartitionOffsetFetcher; -import com.linkedin.venice.kafka.partitionoffset.PartitionOffsetFetcherFactory; -import com.linkedin.venice.meta.HybridStoreConfig; -import com.linkedin.venice.meta.Store; -import com.linkedin.venice.pubsub.PubSubAdminAdapterFactory; import com.linkedin.venice.pubsub.PubSubConstants; import com.linkedin.venice.pubsub.PubSubTopicConfiguration; +import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl; import com.linkedin.venice.pubsub.PubSubTopicPartitionInfo; import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubAdminAdapter; import com.linkedin.venice.pubsub.api.PubSubConsumerAdapter; -import com.linkedin.venice.pubsub.api.PubSubInstrumentedAdminAdapter; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; import com.linkedin.venice.pubsub.api.exceptions.PubSubClientException; @@ -24,151 +35,69 @@ import com.linkedin.venice.pubsub.api.exceptions.PubSubTopicExistsException; import com.linkedin.venice.utils.ExceptionUtils; import com.linkedin.venice.utils.RetryUtils; -import com.linkedin.venice.utils.Time; import com.linkedin.venice.utils.Utils; -import com.linkedin.venice.utils.lazy.Lazy; -import io.tehuti.metrics.MetricsRepository; import it.unimi.dsi.fastutil.ints.Int2LongMap; import java.io.Closeable; import java.time.Duration; import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Properties; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; /** - * Topic Manager is shared by multiple cluster's controllers running in one physical Venice controller instance. + * Topic manager is responsible for creating, deleting, and updating topics. It also provides APIs to query topic metadata. * - * This class contains one global {@link PubSubConsumerAdapter}, which is not thread-safe, so when you add new functions, - * which is using this global consumer, please add 'synchronized' keyword, otherwise this {@link TopicManager} - * won't be thread-safe, and Kafka consumer will report the following error when multiple threads are trying to - * use the same consumer: PubSubConsumerAdapter is not safe for multi-threaded access. + * It is essentially a wrapper over {@link PubSubAdminAdapter} and {@link PubSubConsumerAdapter} with additional + * features such as caching, metrics, and retry. + * + * TODO: We still have retries in the {@link PubSubAdminAdapter}, we will eventually move them here. */ public class TopicManager implements Closeable { - private static final int FAST_KAFKA_OPERATION_TIMEOUT_MS = Time.MS_PER_SECOND; - protected static final long ETERNAL_TOPIC_RETENTION_POLICY_MS = Long.MAX_VALUE; - - public static final long DEFAULT_TOPIC_RETENTION_POLICY_MS = 5 * Time.MS_PER_DAY; - public static final long BUFFER_REPLAY_MINIMAL_SAFETY_MARGIN = 2 * Time.MS_PER_DAY; - - public static final int DEFAULT_KAFKA_OPERATION_TIMEOUT_MS = 30 * Time.MS_PER_SECOND; - public static final int MAX_TOPIC_DELETE_RETRIES = 3; - public static final int DEFAULT_KAFKA_REPLICATION_FACTOR = 3; - - /** - * Default setting is that no log compaction should happen for hybrid store version topics - * if the messages are produced within 24 hours; otherwise servers could encounter MISSING - * data DIV errors for reprocessing jobs which could potentially generate lots of - * duplicate keys. - */ - public static final long DEFAULT_KAFKA_MIN_LOG_COMPACTION_LAG_MS = 24 * Time.MS_PER_HOUR; - - private static final List> CREATE_TOPIC_RETRIABLE_EXCEPTIONS = - Collections.unmodifiableList(Arrays.asList(PubSubOpTimeoutException.class, PubSubClientRetriableException.class)); - - // Immutable state private final Logger logger; - private final String pubSubBootstrapServers; - private final long kafkaOperationTimeoutMs; - private final long topicMinLogCompactionLagMs; - private final PubSubAdminAdapterFactory pubSubAdminAdapterFactory; - // TODO: Use single PubSubAdminAdapter for both read and write operations - private final Lazy pubSubWriteOnlyAdminAdapter; - private final Lazy pubSubReadOnlyAdminAdapter; - private final PartitionOffsetFetcher partitionOffsetFetcher; - - // It's expensive to grab the topic config over and over again, and it changes infrequently. So we temporarily cache - // queried configs. + private final String pubSubClusterAddress; + private final TopicManagerContext topicManagerContext; + private final PubSubAdminAdapter pubSubAdminAdapter; + private final PubSubTopicRepository pubSubTopicRepository; + private final TopicManagerStats stats; + private final TopicMetadataFetcher topicMetadataFetcher; + private AtomicBoolean isClosed = new AtomicBoolean(false); + + // TODO: Consider moving this cache to TopicMetadataFetcher + // It's expensive to grab the topic config over and over again, and it changes infrequently. + // So we temporarily cache queried configs. Cache topicConfigCache = Caffeine.newBuilder().expireAfterWrite(5, TimeUnit.MINUTES).build(); - public TopicManager(TopicManagerRepository.Builder builder, String pubSubBootstrapServers) { - String pubSubServersForLogger = Utils.getSanitizedStringForLogger(pubSubBootstrapServers); - this.logger = LogManager.getLogger(this.getClass().getSimpleName() + " [" + pubSubServersForLogger + "]"); - this.kafkaOperationTimeoutMs = builder.getKafkaOperationTimeoutMs(); - this.topicMinLogCompactionLagMs = builder.getTopicMinLogCompactionLagMs(); - this.pubSubAdminAdapterFactory = builder.getPubSubAdminAdapterFactory(); - this.pubSubBootstrapServers = pubSubBootstrapServers; - - TopicManagerRepository.SSLPropertiesSupplier pubSubProperties = builder.getPubSubProperties(); - PubSubTopicRepository pubSubTopicRepository = builder.getPubSubTopicRepository(); - - Optional optionalMetricsRepository = Optional.ofNullable(builder.getMetricsRepository()); - - this.pubSubReadOnlyAdminAdapter = Lazy.of(() -> { - PubSubAdminAdapter pubSubReadOnlyAdmin = - pubSubAdminAdapterFactory.create(pubSubProperties.get(pubSubBootstrapServers), pubSubTopicRepository); - pubSubReadOnlyAdmin = createInstrumentedPubSubAdmin( - optionalMetricsRepository, - "ReadOnlyKafkaAdminStats", - pubSubReadOnlyAdmin, - pubSubBootstrapServers); - logger.info( - "{} is using read-only pubsub admin client of class: {}", - this.getClass().getSimpleName(), - pubSubReadOnlyAdmin.getClassName()); - return pubSubReadOnlyAdmin; - }); - - this.pubSubWriteOnlyAdminAdapter = Lazy.of(() -> { - PubSubAdminAdapter pubSubWriteOnlyAdmin = - pubSubAdminAdapterFactory.create(pubSubProperties.get(pubSubBootstrapServers), pubSubTopicRepository); - pubSubWriteOnlyAdmin = createInstrumentedPubSubAdmin( - optionalMetricsRepository, - "WriteOnlyKafkaAdminStats", - pubSubWriteOnlyAdmin, - pubSubBootstrapServers); - logger.info( - "{} is using write-only pubsub admin client of class: {}", - this.getClass().getSimpleName(), - pubSubWriteOnlyAdmin.getClassName()); - return pubSubWriteOnlyAdmin; - }); - - this.partitionOffsetFetcher = PartitionOffsetFetcherFactory.createDefaultPartitionOffsetFetcher( - builder.getPubSubConsumerAdapterFactory(), - pubSubProperties.get(pubSubBootstrapServers), - pubSubBootstrapServers, - pubSubReadOnlyAdminAdapter, - kafkaOperationTimeoutMs, - optionalMetricsRepository); - } - - private PubSubAdminAdapter createInstrumentedPubSubAdmin( - Optional optionalMetricsRepository, - String statsNamePrefix, - PubSubAdminAdapter pubSubAdmin, - String pubSubBootstrapServers) { - if (optionalMetricsRepository.isPresent()) { - // Use pub sub bootstrap server to identify which pub sub admin client stats it is - final String pubSubAdminStatsName = - String.format("%s_%s_%s", statsNamePrefix, pubSubAdmin.getClassName(), pubSubBootstrapServers); - PubSubAdminAdapter instrumentedPubSubAdminAdapter = - new PubSubInstrumentedAdminAdapter(pubSubAdmin, optionalMetricsRepository.get(), pubSubAdminStatsName); - logger.info( - "Created instrumented pubsub admin client for pubsub cluster with bootstrap " - + "servers: {} and with stat name prefix: {}", - pubSubBootstrapServers, - statsNamePrefix); - return instrumentedPubSubAdminAdapter; - } else { - logger.info( - "Created non-instrumented pubsub admin client for pubsub cluster with bootstrap servers: {}", - pubSubBootstrapServers); - return pubSubAdmin; - } + public TopicManager(String pubSubClusterAddress, TopicManagerContext context) { + this.logger = LogManager.getLogger( + TopicManager.class.getSimpleName() + " [" + Utils.getSanitizedStringForLogger(pubSubClusterAddress) + "]"); + this.pubSubClusterAddress = Objects.requireNonNull(pubSubClusterAddress, "pubSubClusterAddress cannot be null"); + this.topicManagerContext = context; + this.stats = new TopicManagerStats(context.getMetricsRepository(), pubSubClusterAddress); + this.pubSubTopicRepository = context.getPubSubTopicRepository(); + this.pubSubAdminAdapter = context.getPubSubAdminAdapterFactory() + .create(context.getPubSubProperties(pubSubClusterAddress), pubSubTopicRepository); + this.topicMetadataFetcher = new TopicMetadataFetcher(pubSubClusterAddress, context, stats, pubSubAdminAdapter); + this.logger.info( + "Created a topic manager for the pubsub cluster address: {} with context: {}", + pubSubClusterAddress, + context); } /** * Create a topic, and block until the topic is created, with a default timeout of - * {@value #DEFAULT_KAFKA_OPERATION_TIMEOUT_MS}, after which this function will throw a VeniceException. + * {@value PubSubConstants#PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE}, after which this function will throw a VeniceException. * * @see {@link #createTopic(PubSubTopic, int, int, boolean, boolean, Optional)} */ @@ -188,17 +117,17 @@ public void createTopic( /** * Create a topic, and block until the topic is created, with a default timeout of - * {@value #DEFAULT_KAFKA_OPERATION_TIMEOUT_MS}, after which this function will throw a VeniceException. + * {@value PubSubConstants#PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE}, after which this function will throw a VeniceException. * * @param topicName Name for the new topic * @param numPartitions number of partitions * @param replication replication factor * @param eternal if true, the topic will have "infinite" (~250 mil years) retention - * if false, its retention will be set to {@link #DEFAULT_TOPIC_RETENTION_POLICY_MS} by default + * if false, its retention will be set to {@link PubSubConstants#DEFAULT_TOPIC_RETENTION_POLICY_MS} by default * @param logCompaction whether to enable log compaction on the topic * @param minIsr if present, will apply the specified min.isr to this topic, - * if absent, Kafka cluster defaults will be used - * @param useFastKafkaOperationTimeout if false, normal kafka operation timeout will be used, + * if absent, PubSub cluster defaults will be used + * @param useFastPubSubOperationTimeout if false, normal PubSub operation timeout will be used, * if true, a much shorter timeout will be used to make topic creation non-blocking. */ public void createTopic( @@ -208,7 +137,7 @@ public void createTopic( boolean eternal, boolean logCompaction, Optional minIsr, - boolean useFastKafkaOperationTimeout) { + boolean useFastPubSubOperationTimeout) { long retentionTimeMs; if (eternal) { retentionTimeMs = ETERNAL_TOPIC_RETENTION_POLICY_MS; @@ -222,12 +151,12 @@ public void createTopic( retentionTimeMs, logCompaction, minIsr, - useFastKafkaOperationTimeout); + useFastPubSubOperationTimeout); } /** * Create a topic, and block until the topic is created, with a default timeout of - * {@value #DEFAULT_KAFKA_OPERATION_TIMEOUT_MS}, after which this function will throw a VeniceException. + * {@value PubSubConstants#PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE}, after which this function will throw a VeniceException. * * @param topicName Name for the new topic * @param numPartitions number of partitions @@ -235,8 +164,8 @@ public void createTopic( * @param retentionTimeMs Retention time, in ms, for the topic * @param logCompaction whether to enable log compaction on the topic * @param minIsr if present, will apply the specified min.isr to this topic, - * if absent, Kafka cluster defaults will be used - * @param useFastKafkaOperationTimeout if false, normal kafka operation timeout will be used, + * if absent, PubSub cluster defaults will be used + * @param useFastPubSubOperationTimeout if false, normal PubSub operation timeout will be used, * if true, a much shorter timeout will be used to make topic creation non-blocking. */ public void createTopic( @@ -246,16 +175,17 @@ public void createTopic( long retentionTimeMs, boolean logCompaction, Optional minIsr, - boolean useFastKafkaOperationTimeout) { + boolean useFastPubSubOperationTimeout) { long startTime = System.currentTimeMillis(); - long deadlineMs = - startTime + (useFastKafkaOperationTimeout ? FAST_KAFKA_OPERATION_TIMEOUT_MS : kafkaOperationTimeoutMs); + long deadlineMs = startTime + (useFastPubSubOperationTimeout + ? PUBSUB_FAST_OPERATION_TIMEOUT_MS + : topicManagerContext.getPubSubOperationTimeoutMs()); PubSubTopicConfiguration pubSubTopicConfiguration = new PubSubTopicConfiguration( Optional.of(retentionTimeMs), logCompaction, minIsr, - topicMinLogCompactionLagMs, + topicManagerContext.getTopicMinLogCompactionLagMs(), Optional.empty()); logger.info( "Creating topic: {} partitions: {} replication: {}, configuration: {}", @@ -266,12 +196,14 @@ public void createTopic( try { RetryUtils.executeWithMaxAttemptAndExponentialBackoff( - () -> pubSubWriteOnlyAdminAdapter.get() - .createTopic(topicName, numPartitions, replication, pubSubTopicConfiguration), + () -> createTopic(topicName, numPartitions, replication, pubSubTopicConfiguration), 10, Duration.ofMillis(200), Duration.ofSeconds(1), - Duration.ofMillis(useFastKafkaOperationTimeout ? FAST_KAFKA_OPERATION_TIMEOUT_MS : kafkaOperationTimeoutMs), + Duration.ofMillis( + useFastPubSubOperationTimeout + ? PUBSUB_FAST_OPERATION_TIMEOUT_MS + : topicManagerContext.getPubSubOperationTimeoutMs()), CREATE_TOPIC_RETRIABLE_EXCEPTIONS); } catch (Exception e) { if (ExceptionUtils.recursiveClassEquals(e, PubSubTopicExistsException.class)) { @@ -288,7 +220,7 @@ public void createTopic( } } waitUntilTopicCreated(topicName, numPartitions, deadlineMs); - boolean eternal = retentionTimeMs == ETERNAL_TOPIC_RETENTION_POLICY_MS; + boolean eternal = retentionTimeMs == PubSubConstants.ETERNAL_TOPIC_RETENTION_POLICY_MS; logger.info("Successfully created {}topic: {}", eternal ? "eternal " : "", topicName); } @@ -331,19 +263,19 @@ public boolean updateTopicRetention( Optional retentionTimeMs = pubSubTopicConfiguration.retentionInMs(); if (!retentionTimeMs.isPresent() || expectedRetentionInMs != retentionTimeMs.get()) { pubSubTopicConfiguration.setRetentionInMs(Optional.of(expectedRetentionInMs)); - pubSubWriteOnlyAdminAdapter.get().setTopicConfig(topicName, pubSubTopicConfiguration); + setTopicConfig(topicName, pubSubTopicConfiguration); logger.info( "Updated topic: {} with retention.ms: {} in cluster [{}]", topicName, expectedRetentionInMs, - this.pubSubBootstrapServers); + this.pubSubClusterAddress); return true; } // Retention time has already been updated for this topic before return false; } - public synchronized void updateTopicCompactionPolicy(PubSubTopic topic, boolean expectedLogCompacted) { + public void updateTopicCompactionPolicy(PubSubTopic topic, boolean expectedLogCompacted) { updateTopicCompactionPolicy(topic, expectedLogCompacted, -1, Optional.empty()); } @@ -355,7 +287,7 @@ public synchronized void updateTopicCompactionPolicy(PubSubTopic topic, boolean * override the default config * @throws PubSubTopicDoesNotExistException, if the topic doesn't exist */ - public synchronized void updateTopicCompactionPolicy( + public void updateTopicCompactionPolicy( PubSubTopic topic, boolean expectedLogCompacted, long minLogCompactionLagMs, @@ -367,7 +299,7 @@ public synchronized void updateTopicCompactionPolicy( if (minLogCompactionLagMs > 0) { expectedMinLogCompactionLagMs = minLogCompactionLagMs; } else { - expectedMinLogCompactionLagMs = topicMinLogCompactionLagMs; + expectedMinLogCompactionLagMs = topicManagerContext.getTopicMinLogCompactionLagMs(); } expectedMaxLogCompactionLagMs = maxLogCompactionLagMs; @@ -390,7 +322,7 @@ public synchronized void updateTopicCompactionPolicy( pubSubTopicConfiguration.setLogCompacted(expectedLogCompacted); pubSubTopicConfiguration.setMinLogCompactionLagMs(expectedMinLogCompactionLagMs); pubSubTopicConfiguration.setMaxLogCompactionLagMs(expectedMaxLogCompactionLagMs); - pubSubWriteOnlyAdminAdapter.get().setTopicConfig(topic, pubSubTopicConfiguration); + setTopicConfig(topic, pubSubTopicConfiguration); logger.info( "Kafka compaction policy for topic: {} has been updated from {} to {}, min compaction lag updated from" + " {} to {}, max compaction lag updated from {} to {}", @@ -426,7 +358,7 @@ public boolean updateTopicMinInSyncReplica(PubSubTopic topicName, int minISR) // config doesn't exist config is different if (!currentMinISR.isPresent() || !currentMinISR.get().equals(minISR)) { pubSubTopicConfiguration.setMinInSyncReplicas(Optional.of(minISR)); - pubSubWriteOnlyAdminAdapter.get().setTopicConfig(topicName, pubSubTopicConfiguration); + setTopicConfig(topicName, pubSubTopicConfiguration); logger.info("Updated topic: {} with min.insync.replicas: {}", topicName, minISR); return true; } @@ -434,8 +366,15 @@ public boolean updateTopicMinInSyncReplica(PubSubTopic topicName, int minISR) return false; } + /** + * Get retention time for all topics in the pubsub cluster. + * @return a map of topic name to retention time in MS. + */ public Map getAllTopicRetentions() { - return pubSubReadOnlyAdminAdapter.get().getAllTopicRetentions(); + long startTime = System.currentTimeMillis(); + Map topicRetentions = pubSubAdminAdapter.getAllTopicRetentions(); + stats.recordLatency(GET_ALL_TOPIC_RETENTIONS, startTime); + return topicRetentions; } /** @@ -446,11 +385,11 @@ public long getTopicRetention(PubSubTopic topicName) throws PubSubTopicDoesNotEx return getTopicRetention(pubSubTopicConfiguration); } - public long getTopicRetention(PubSubTopicConfiguration pubSubTopicConfiguration) { + public static long getTopicRetention(PubSubTopicConfiguration pubSubTopicConfiguration) { if (pubSubTopicConfiguration.retentionInMs().isPresent()) { return pubSubTopicConfiguration.retentionInMs().get(); } - return PubSubConstants.UNKNOWN_TOPIC_RETENTION; + return PUBSUB_TOPIC_UNKNOWN_RETENTION; } /** @@ -469,23 +408,41 @@ public boolean isTopicTruncated(PubSubTopic topicName, long truncatedTopicMaxRet } public boolean isRetentionBelowTruncatedThreshold(long retention, long truncatedTopicMaxRetentionMs) { - return retention != PubSubConstants.UNKNOWN_TOPIC_RETENTION && retention <= truncatedTopicMaxRetentionMs; + return retention != PUBSUB_TOPIC_UNKNOWN_RETENTION && retention <= truncatedTopicMaxRetentionMs; + } + + private void createTopic( + PubSubTopic pubSubTopic, + int numPartitions, + int replicationFactor, + PubSubTopicConfiguration topicConfiguration) { + long startTime = System.currentTimeMillis(); + pubSubAdminAdapter.createTopic(pubSubTopic, numPartitions, replicationFactor, topicConfiguration); + stats.recordLatency(CREATE_TOPIC, startTime); + } + + private void setTopicConfig(PubSubTopic pubSubTopic, PubSubTopicConfiguration pubSubTopicConfiguration) { + long startTime = System.currentTimeMillis(); + pubSubAdminAdapter.setTopicConfig(pubSubTopic, pubSubTopicConfiguration); + stats.recordLatency(SET_TOPIC_CONFIG, startTime); } /** * This operation is a little heavy, since it will pull the configs for all the topics. */ - public PubSubTopicConfiguration getTopicConfig(PubSubTopic topicName) throws PubSubTopicDoesNotExistException { - final PubSubTopicConfiguration pubSubTopicConfiguration = - pubSubReadOnlyAdminAdapter.get().getTopicConfig(topicName); - topicConfigCache.put(topicName, pubSubTopicConfiguration); + public PubSubTopicConfiguration getTopicConfig(PubSubTopic pubSubTopic) { + long startTime = System.currentTimeMillis(); + PubSubTopicConfiguration pubSubTopicConfiguration = pubSubAdminAdapter.getTopicConfig(pubSubTopic); + topicConfigCache.put(pubSubTopic, pubSubTopicConfiguration); + stats.recordLatency(GET_TOPIC_CONFIG, startTime); return pubSubTopicConfiguration; } public PubSubTopicConfiguration getTopicConfigWithRetry(PubSubTopic topicName) { - final PubSubTopicConfiguration pubSubTopicConfiguration = - pubSubReadOnlyAdminAdapter.get().getTopicConfigWithRetry(topicName); + long startTime = System.currentTimeMillis(); + PubSubTopicConfiguration pubSubTopicConfiguration = pubSubAdminAdapter.getTopicConfigWithRetry(topicName); topicConfigCache.put(topicName, pubSubTopicConfiguration); + stats.recordLatency(GET_TOPIC_CONFIG_WITH_RETRY, startTime); return pubSubTopicConfiguration; } @@ -493,7 +450,7 @@ public PubSubTopicConfiguration getTopicConfigWithRetry(PubSubTopic topicName) { * Still heavy, but can be called repeatedly to amortize that cost. */ public PubSubTopicConfiguration getCachedTopicConfig(PubSubTopic topicName) { - // query the cache first, if it doesn't have it, query it from kafka and store it. + // query the cache first, if it doesn't have it, query it from PubSub and store it. PubSubTopicConfiguration pubSubTopicConfiguration = topicConfigCache.getIfPresent(topicName); if (pubSubTopicConfiguration == null) { pubSubTopicConfiguration = getTopicConfigWithRetry(topicName); @@ -502,11 +459,12 @@ public PubSubTopicConfiguration getCachedTopicConfig(PubSubTopic topicName) { } public Map getSomeTopicConfigs(Set topicNames) { - final Map topicConfigs = - pubSubReadOnlyAdminAdapter.get().getSomeTopicConfigs(topicNames); + long startTime = System.currentTimeMillis(); + Map topicConfigs = pubSubAdminAdapter.getSomeTopicConfigs(topicNames); for (Map.Entry topicConfig: topicConfigs.entrySet()) { topicConfigCache.put(topicConfig.getKey(), topicConfig.getValue()); } + stats.recordLatency(GET_SOME_TOPIC_CONFIGS, startTime); return topicConfigs; } @@ -522,10 +480,15 @@ public void ensureTopicIsDeletedAndBlock(PubSubTopic pubSubTopic) { logger.info("Deleting topic: {}", pubSubTopic); try { - pubSubWriteOnlyAdminAdapter.get().deleteTopic(pubSubTopic, Duration.ofMillis(kafkaOperationTimeoutMs)); + long startTime = System.currentTimeMillis(); + pubSubAdminAdapter.deleteTopic(pubSubTopic, Duration.ofMillis(topicManagerContext.getPubSubOperationTimeoutMs())); + stats.recordLatency(DELETE_TOPIC, startTime); logger.info("Topic: {} has been deleted", pubSubTopic); } catch (PubSubOpTimeoutException e) { - logger.warn("Failed to delete topic: {} after {} ms", pubSubTopic, kafkaOperationTimeoutMs); + logger.warn( + "Failed to delete topic: {} after {} ms", + pubSubTopic, + topicManagerContext.getPubSubOperationTimeoutMs()); } catch (PubSubTopicDoesNotExistException e) { // No-op. Topic is deleted already, consider this as a successful deletion. } catch (PubSubClientRetriableException | PubSubClientException e) { @@ -534,43 +497,46 @@ public void ensureTopicIsDeletedAndBlock(PubSubTopic pubSubTopic) { } // let's make sure the topic is deleted - if (pubSubWriteOnlyAdminAdapter.get().containsTopic(pubSubTopic)) { + if (containsTopic(pubSubTopic)) { throw new PubSubTopicExistsException("Topic: " + pubSubTopic.getName() + " still exists after deletion"); } } + /** + * Delete a topic with retry and block until it is deleted or operation times out. + * @param pubSubTopic + */ public void ensureTopicIsDeletedAndBlockWithRetry(PubSubTopic pubSubTopic) { int attempts = 0; - while (attempts++ < MAX_TOPIC_DELETE_RETRIES) { + while (attempts++ < PUBSUB_TOPIC_DELETE_RETRY_TIMES) { try { - logger.debug("Deleting topic: {} with retry attempt {} / {}", pubSubTopic, attempts, MAX_TOPIC_DELETE_RETRIES); - ensureTopicIsDeletedAndBlock(pubSubTopic); - return; - } catch (PubSubClientRetriableException e) { - String errorMessage = e instanceof PubSubOpTimeoutException ? "timed out" : "errored out"; - logger.warn( - "Topic deletion for topic: {} {}! Retry attempt {} / {}", + logger.debug( + "Deleting topic: {} with retry attempt {} / {}", pubSubTopic, - errorMessage, attempts, - MAX_TOPIC_DELETE_RETRIES); - if (attempts == MAX_TOPIC_DELETE_RETRIES) { - logger.error("Topic deletion for topic {} {}! Giving up!!", pubSubTopic, errorMessage, e); + PUBSUB_TOPIC_DELETE_RETRY_TIMES); + ensureTopicIsDeletedAndBlock(pubSubTopic); + return; + } catch (PubSubTopicExistsException | PubSubClientRetriableException e) { + if (attempts == PUBSUB_TOPIC_DELETE_RETRY_TIMES) { + logger.error( + "Topic deletion for topic {} {}! Giving up after {} retries", + pubSubTopic, + e instanceof PubSubOpTimeoutException ? "timed out" : "errored out", + attempts, + e); throw e; } } } } + // TODO: Evaluate if we need synchronized here public synchronized Set listTopics() { - return pubSubReadOnlyAdminAdapter.get().listAllTopics(); - } - - /** - * A quick check to see whether the topic exists. - */ - public boolean containsTopic(PubSubTopic topic) { - return pubSubReadOnlyAdminAdapter.get().containsTopic(topic); + long startTime = System.currentTimeMillis(); + Set topics = pubSubAdminAdapter.listAllTopics(); + stats.recordLatency(LIST_ALL_TOPICS, startTime); + return topics; } /** @@ -581,7 +547,10 @@ public boolean containsTopicWithExpectationAndRetry( PubSubTopic topic, int maxAttempts, final boolean expectedResult) { - return pubSubReadOnlyAdminAdapter.get().containsTopicWithExpectationAndRetry(topic, maxAttempts, expectedResult); + long startTime = System.currentTimeMillis(); + boolean containsTopic = pubSubAdminAdapter.containsTopicWithExpectationAndRetry(topic, maxAttempts, expectedResult); + stats.recordLatency(CONTAINS_TOPIC_WITH_RETRY, startTime); + return containsTopic; } public boolean containsTopicWithExpectationAndRetry( @@ -591,14 +560,16 @@ public boolean containsTopicWithExpectationAndRetry( Duration initialBackoff, Duration maxBackoff, Duration maxDuration) { - return pubSubReadOnlyAdminAdapter.get() - .containsTopicWithExpectationAndRetry( - topic, - maxAttempts, - expectedResult, - initialBackoff, - maxBackoff, - maxDuration); + long startTime = System.currentTimeMillis(); + boolean containsTopic = pubSubAdminAdapter.containsTopicWithExpectationAndRetry( + topic, + maxAttempts, + expectedResult, + initialBackoff, + maxBackoff, + maxDuration); + stats.recordLatency(CONTAINS_TOPIC_WITH_RETRY, startTime); + return containsTopic; } /** @@ -614,13 +585,11 @@ public boolean containsTopicAndAllPartitionsAreOnline(PubSubTopic topic) { * @return true if the topic exists and all its partitions have at least one in-sync replica * false if the topic does not exist at all or if it exists but isn't completely available */ - public synchronized boolean containsTopicAndAllPartitionsAreOnline( - PubSubTopic topic, - Integer expectedPartitionCount) { + public boolean containsTopicAndAllPartitionsAreOnline(PubSubTopic topic, Integer expectedPartitionCount) { if (!containsTopic(topic)) { return false; } - List partitionInfoList = partitionOffsetFetcher.partitionsFor(topic); + List partitionInfoList = topicMetadataFetcher.getTopicPartitionInfo(topic); if (partitionInfoList == null) { logger.warn("getConsumer().partitionsFor() returned null for topic: {}", topic); return false; @@ -651,72 +620,121 @@ public synchronized boolean containsTopicAndAllPartitionsAreOnline( return allPartitionsHaveAnInSyncReplica; } + // For testing only + public void setTopicConfigCache(Cache topicConfigCache) { + this.topicConfigCache = topicConfigCache; + } + /** - * Generate a map from partition number to the last offset available for that partition - * @param topic - * @return a Map of partition to the latest offset, or an empty map if there's any problem + * Get information about all partitions for a given topic. + * @param pubSubTopic the topic to get partition info for + * @return a list of {@link PubSubTopicPartitionInfo} for the given topic + */ + public List getTopicPartitionInfo(PubSubTopic pubSubTopic) { + return topicMetadataFetcher.getTopicPartitionInfo(pubSubTopic); + } + + /** + * Get the latest offsets for all partitions of a given topic. + * @param pubSubTopic the topic to get latest offsets for + * @return a Map of partition to the latest offset, or an empty map if there's any problem getting the offsets */ - public Int2LongMap getTopicLatestOffsets(PubSubTopic topic) { - return partitionOffsetFetcher.getTopicLatestOffsets(topic); + public Int2LongMap getTopicLatestOffsets(PubSubTopic pubSubTopic) { + return topicMetadataFetcher.getTopicLatestOffsets(pubSubTopic); + } + + /** + * Get partition count for a given topic. + * @param pubSubTopic the topic to get partition count for + * @return the number of partitions for the given topic + * @throws PubSubTopicDoesNotExistException if the topic does not exist + */ + public int getPartitionCount(PubSubTopic pubSubTopic) { + List partitionInfoList = getTopicPartitionInfo(pubSubTopic); + if (partitionInfoList == null) { + throw new PubSubTopicDoesNotExistException("Topic: " + pubSubTopic + " does not exist"); + } + return partitionInfoList.size(); + } + + public boolean containsTopic(PubSubTopic pubSubTopic) { + return topicMetadataFetcher.containsTopic(pubSubTopic); + } + + public boolean containsTopicCached(PubSubTopic pubSubTopic) { + return topicMetadataFetcher.containsTopicCached(pubSubTopic); + } + + public long getLatestOffsetWithRetries(PubSubTopicPartition pubSubTopicPartition, int retries) { + return topicMetadataFetcher.getLatestOffsetWithRetries(pubSubTopicPartition, retries); + } + + public long getLatestOffsetCached(PubSubTopic pubSubTopic, int partitionId) { + return topicMetadataFetcher.getLatestOffsetCached(new PubSubTopicPartitionImpl(pubSubTopic, partitionId)); } - public long getPartitionLatestOffsetAndRetry(PubSubTopicPartition pubSubTopicPartition, int retries) { - return partitionOffsetFetcher.getPartitionLatestOffsetAndRetry(pubSubTopicPartition, retries); + public long getProducerTimestampOfLastDataMessageWithRetries(PubSubTopicPartition pubSubTopicPartition, int retries) { + return topicMetadataFetcher.getProducerTimestampOfLastDataMessageWithRetries(pubSubTopicPartition, retries); } - public long getProducerTimestampOfLastDataRecord(PubSubTopicPartition pubSubTopicPartition, int retries) { - return partitionOffsetFetcher.getProducerTimestampOfLastDataRecord(pubSubTopicPartition, retries); + public long getProducerTimestampOfLastDataMessageCached(PubSubTopicPartition pubSubTopicPartition) { + return topicMetadataFetcher.getProducerTimestampOfLastDataMessageCached(pubSubTopicPartition); } /** * Get offsets for only one partition with a specific timestamp. */ - public long getPartitionOffsetByTime(PubSubTopicPartition pubSubTopicPartition, long timestamp) { - return partitionOffsetFetcher.getPartitionOffsetByTime(pubSubTopicPartition, timestamp); + public long getOffsetByTime(PubSubTopicPartition pubSubTopicPartition, long timestamp) { + return topicMetadataFetcher.getOffsetForTimeWithRetries(pubSubTopicPartition, timestamp, 25); } /** - * Get a list of {@link PubSubTopicPartitionInfo} objects for the specified topic. - * @param topic - * @return + * Invalidate the cache for the given topic and all its partitions. + * @param pubSubTopic the topic to invalidate */ - public List partitionsFor(PubSubTopic topic) { - return partitionOffsetFetcher.partitionsFor(topic); + public CompletableFuture invalidateCache(PubSubTopic pubSubTopic) { + return topicMetadataFetcher.invalidateKeyAsync(pubSubTopic); } - public String getPubSubBootstrapServers() { - return this.pubSubBootstrapServers; + /** + * Prefetch and cache the latest offset for the given topic-partition. + * @param pubSubTopicPartition the topic-partition to prefetch and cache the latest offset for + */ + public void prefetchAndCacheLatestOffset(PubSubTopicPartition pubSubTopicPartition) { + topicMetadataFetcher.populateCacheWithLatestOffset(pubSubTopicPartition); + } + + public String getPubSubClusterAddress() { + return this.pubSubClusterAddress; } @Override - public synchronized void close() { - Utils.closeQuietlyWithErrorLogged(partitionOffsetFetcher); - pubSubReadOnlyAdminAdapter.ifPresent(Utils::closeQuietlyWithErrorLogged); - pubSubWriteOnlyAdminAdapter.ifPresent(Utils::closeQuietlyWithErrorLogged); + public void close() { + if (isClosed.get()) { + logger.warn("{} is already closed", this); + } + try { + closeAsync().get(600, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + logger.error("Interrupted while closing: {}", this, e); + } catch (ExecutionException | TimeoutException e) { + logger.error("Failed to close: {}", this, e); + } } - // For testing only - public void setTopicConfigCache(Cache topicConfigCache) { - this.topicConfigCache = topicConfigCache; + CompletableFuture closeAsync() { + return CompletableFuture.runAsync(() -> { + if (isClosed.compareAndSet(false, true)) { + logger.info("Closing {}", this); + Utils.closeQuietlyWithErrorLogged(topicMetadataFetcher); + Utils.closeQuietlyWithErrorLogged(pubSubAdminAdapter); + } + }); } - /** - * The default retention time for the RT topic is defined in {@link TopicManager#DEFAULT_TOPIC_RETENTION_POLICY_MS}, - * but if the rewind time is larger than this, then the RT topic's retention time needs to be set even higher, - * in order to guarantee that buffer replays do not lose data. In order to achieve this, the retention time is - * set to the max of either: - * - * 1. {@link TopicManager#DEFAULT_TOPIC_RETENTION_POLICY_MS}; or - * 2. {@link HybridStoreConfig#getRewindTimeInSeconds()} + {@link Store#getBootstrapToOnlineTimeoutInHours()} + {@value #BUFFER_REPLAY_MINIMAL_SAFETY_MARGIN}; - * - * This is a convenience function, and thus must be ignored by the JSON machinery. - * - * @return the retention time for the RT topic, in milliseconds. - */ - public static long getExpectedRetentionTimeInMs(Store store, HybridStoreConfig hybridConfig) { - long rewindTimeInMs = hybridConfig.getRewindTimeInSeconds() * Time.MS_PER_SECOND; - long bootstrapToOnlineTimeInMs = (long) store.getBootstrapToOnlineTimeoutInHours() * Time.MS_PER_HOUR; - long minimumRetentionInMs = rewindTimeInMs + bootstrapToOnlineTimeInMs + BUFFER_REPLAY_MINIMAL_SAFETY_MARGIN; - return Math.max(minimumRetentionInMs, TopicManager.DEFAULT_TOPIC_RETENTION_POLICY_MS); + @Override + public String toString() { + return "TopicManager{pubSubClusterAddress=" + pubSubClusterAddress + "}"; } } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerContext.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerContext.java new file mode 100644 index 0000000000..c12e813a84 --- /dev/null +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerContext.java @@ -0,0 +1,222 @@ +package com.linkedin.venice.pubsub.manager; + +import static com.linkedin.venice.pubsub.PubSubConstants.DEFAULT_KAFKA_MIN_LOG_COMPACTION_LAG_MS; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_TOPIC_DELETION_STATUS_POLL_INTERVAL_MS_DEFAULT_VALUE; + +import com.linkedin.venice.pubsub.PubSubAdminAdapterFactory; +import com.linkedin.venice.pubsub.PubSubConsumerAdapterFactory; +import com.linkedin.venice.pubsub.PubSubTopicRepository; +import com.linkedin.venice.pubsub.api.PubSubAdminAdapter; +import com.linkedin.venice.pubsub.api.PubSubConsumerAdapter; +import com.linkedin.venice.utils.VeniceProperties; +import io.tehuti.metrics.MetricsRepository; + + +/** + * A context object that contains all the dependencies needed by {@link TopicManager}. + */ +public class TopicManagerContext { + private final PubSubAdminAdapterFactory pubSubAdminAdapterFactory; + private final PubSubConsumerAdapterFactory pubSubConsumerAdapterFactory; + private final PubSubTopicRepository pubSubTopicRepository; + private final MetricsRepository metricsRepository; + private final PubSubPropertiesSupplier pubSubPropertiesSupplier; + private final long pubSubOperationTimeoutMs; + private final long topicDeletionStatusPollIntervalMs; + private final long topicMinLogCompactionLagMs; + private final long topicOffsetCheckIntervalMs; + private final int topicMetadataFetcherConsumerPoolSize; + private final int topicMetadataFetcherThreadPoolSize; + + private TopicManagerContext(Builder builder) { + this.pubSubOperationTimeoutMs = builder.pubSubOperationTimeoutMs; + this.topicDeletionStatusPollIntervalMs = builder.topicDeletionStatusPollIntervalMs; + this.topicMinLogCompactionLagMs = builder.topicMinLogCompactionLagMs; + this.pubSubAdminAdapterFactory = builder.pubSubAdminAdapterFactory; + this.pubSubConsumerAdapterFactory = builder.pubSubConsumerAdapterFactory; + this.pubSubTopicRepository = builder.pubSubTopicRepository; + this.metricsRepository = builder.metricsRepository; + this.pubSubPropertiesSupplier = builder.pubSubPropertiesSupplier; + this.topicOffsetCheckIntervalMs = builder.topicOffsetCheckIntervalMs; + this.topicMetadataFetcherConsumerPoolSize = builder.topicMetadataFetcherConsumerPoolSize; + this.topicMetadataFetcherThreadPoolSize = builder.topicMetadataFetcherThreadPoolSize; + } + + public long getPubSubOperationTimeoutMs() { + return pubSubOperationTimeoutMs; + } + + public long getTopicDeletionStatusPollIntervalMs() { + return topicDeletionStatusPollIntervalMs; + } + + public long getTopicMinLogCompactionLagMs() { + return topicMinLogCompactionLagMs; + } + + public PubSubAdminAdapterFactory getPubSubAdminAdapterFactory() { + return pubSubAdminAdapterFactory; + } + + public PubSubConsumerAdapterFactory getPubSubConsumerAdapterFactory() { + return pubSubConsumerAdapterFactory; + } + + public PubSubTopicRepository getPubSubTopicRepository() { + return pubSubTopicRepository; + } + + public MetricsRepository getMetricsRepository() { + return metricsRepository; + } + + public PubSubPropertiesSupplier getPubSubPropertiesSupplier() { + return pubSubPropertiesSupplier; + } + + public VeniceProperties getPubSubProperties(String pubSubBootstrapServers) { + return pubSubPropertiesSupplier.get(pubSubBootstrapServers); + } + + public long getTopicOffsetCheckIntervalMs() { + return topicOffsetCheckIntervalMs; + } + + public int getTopicMetadataFetcherConsumerPoolSize() { + return topicMetadataFetcherConsumerPoolSize; + } + + public int getTopicMetadataFetcherThreadPoolSize() { + return topicMetadataFetcherThreadPoolSize; + } + + public interface PubSubPropertiesSupplier { + VeniceProperties get(String pubSubBootstrapServers); + } + + @Override + public String toString() { + return "TopicManagerContext{pubSubOperationTimeoutMs=" + pubSubOperationTimeoutMs + + ", topicDeletionStatusPollIntervalMs=" + topicDeletionStatusPollIntervalMs + ", topicMinLogCompactionLagMs=" + + topicMinLogCompactionLagMs + ", topicOffsetCheckIntervalMs=" + topicOffsetCheckIntervalMs + + ", topicMetadataFetcherConsumerPoolSize=" + topicMetadataFetcherConsumerPoolSize + + ", topicMetadataFetcherThreadPoolSize=" + topicMetadataFetcherThreadPoolSize + ", pubSubAdminAdapterFactory=" + + pubSubAdminAdapterFactory.getClass().getSimpleName() + ", pubSubConsumerAdapterFactory=" + + pubSubConsumerAdapterFactory.getClass().getSimpleName() + '}'; + } + + public static class Builder { + private PubSubAdminAdapterFactory pubSubAdminAdapterFactory; + private PubSubConsumerAdapterFactory pubSubConsumerAdapterFactory; + private PubSubTopicRepository pubSubTopicRepository; + private MetricsRepository metricsRepository; + private PubSubPropertiesSupplier pubSubPropertiesSupplier; + private long pubSubOperationTimeoutMs = PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE; + private long topicDeletionStatusPollIntervalMs = PUBSUB_TOPIC_DELETION_STATUS_POLL_INTERVAL_MS_DEFAULT_VALUE; + private long topicMinLogCompactionLagMs = DEFAULT_KAFKA_MIN_LOG_COMPACTION_LAG_MS; + private long topicOffsetCheckIntervalMs = 60_000L; // 1 minute + private int topicMetadataFetcherConsumerPoolSize = 1; + private int topicMetadataFetcherThreadPoolSize = 2; + + public Builder setPubSubOperationTimeoutMs(long pubSubOperationTimeoutMs) { + this.pubSubOperationTimeoutMs = pubSubOperationTimeoutMs; + return this; + } + + public Builder setTopicDeletionStatusPollIntervalMs(long topicDeletionStatusPollIntervalMs) { + this.topicDeletionStatusPollIntervalMs = topicDeletionStatusPollIntervalMs; + return this; + } + + public Builder setTopicMinLogCompactionLagMs(long topicMinLogCompactionLagMs) { + this.topicMinLogCompactionLagMs = topicMinLogCompactionLagMs; + return this; + } + + public Builder setPubSubAdminAdapterFactory( + PubSubAdminAdapterFactory pubSubAdminAdapterFactory) { + this.pubSubAdminAdapterFactory = pubSubAdminAdapterFactory; + return this; + } + + public Builder setPubSubConsumerAdapterFactory( + PubSubConsumerAdapterFactory pubSubConsumerAdapterFactory) { + this.pubSubConsumerAdapterFactory = pubSubConsumerAdapterFactory; + return this; + } + + public Builder setPubSubTopicRepository(PubSubTopicRepository pubSubTopicRepository) { + this.pubSubTopicRepository = pubSubTopicRepository; + return this; + } + + public Builder setMetricsRepository(MetricsRepository metricsRepository) { + this.metricsRepository = metricsRepository; + return this; + } + + public Builder setPubSubPropertiesSupplier(PubSubPropertiesSupplier pubSubPropertiesSupplier) { + this.pubSubPropertiesSupplier = pubSubPropertiesSupplier; + return this; + } + + public Builder setTopicOffsetCheckIntervalMs(long topicOffsetCheckIntervalMs) { + this.topicOffsetCheckIntervalMs = topicOffsetCheckIntervalMs; + return this; + } + + public Builder setTopicMetadataFetcherConsumerPoolSize(int topicMetadataFetcherConsumerPoolSize) { + this.topicMetadataFetcherConsumerPoolSize = topicMetadataFetcherConsumerPoolSize; + return this; + } + + public Builder setTopicMetadataFetcherThreadPoolSize(int topicMetadataFetcherThreadPoolSize) { + this.topicMetadataFetcherThreadPoolSize = topicMetadataFetcherThreadPoolSize; + return this; + } + + public void verify() { + if (pubSubAdminAdapterFactory == null) { + throw new IllegalArgumentException("pubSubAdminAdapterFactory cannot be null"); + } + + if (pubSubConsumerAdapterFactory == null) { + throw new IllegalArgumentException("pubSubConsumerAdapterFactory cannot be null"); + } + + if (pubSubTopicRepository == null) { + throw new IllegalArgumentException("pubSubTopicRepository cannot be null"); + } + + if (pubSubPropertiesSupplier == null) { + throw new IllegalArgumentException("pubSubPropertiesSupplier cannot be null"); + } + + if (pubSubOperationTimeoutMs <= 0) { + throw new IllegalArgumentException("pubSubOperationTimeoutMs must be positive"); + } + + if (topicDeletionStatusPollIntervalMs < 0) { + throw new IllegalArgumentException("topicDeletionStatusPollIntervalMs must be positive"); + } + + if (topicOffsetCheckIntervalMs < 0) { + throw new IllegalArgumentException("topicOffsetCheckIntervalMs must be positive"); + } + + if (topicMetadataFetcherConsumerPoolSize <= 0) { + throw new IllegalArgumentException("topicMetadataFetcherConsumerPoolSize must be positive"); + } + + if (topicMetadataFetcherThreadPoolSize <= 0) { + throw new IllegalArgumentException("topicMetadataFetcherThreadPoolSize must be positive"); + } + } + + public TopicManagerContext build() { + verify(); + return new TopicManagerContext(this); + } + } +} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerRepository.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerRepository.java new file mode 100644 index 0000000000..94d0be66e5 --- /dev/null +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerRepository.java @@ -0,0 +1,74 @@ +package com.linkedin.venice.pubsub.manager; + +import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap; +import java.io.Closeable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + + +/** + * A repository of {@link TopicManager} instances, each associated with a specific PubSub region and cluster. + * This repository maintains one {@link TopicManager} for each unique PubSub bootstrap server address. + * While not mandatory, it is expected that each Venice component will have one and only one instance of this class. + */ +public class TopicManagerRepository implements Closeable { + private static final Logger LOGGER = LogManager.getLogger(TopicManagerRepository.class); + private final Map topicManagers = new VeniceConcurrentHashMap<>(); + private final TopicManagerContext topicManagerContext; + private final String localPubSubAddress; + + public TopicManagerRepository(TopicManagerContext topicManagerContext, String localPubSubAddress) { + this.topicManagerContext = Objects.requireNonNull(topicManagerContext, "topicManagerContext cannot be null"); + this.localPubSubAddress = Objects.requireNonNull(localPubSubAddress, "localPubSubAddress cannot be null"); + } + + // added in order to help with testing; visibility is package-private for testing purposes + TopicManager createTopicManager(String pubSubAddress) { + return new TopicManager(pubSubAddress, topicManagerContext); + } + + /** + * By default, return TopicManager for local PubSub cluster. + */ + public TopicManager getLocalTopicManager() { + return getTopicManager(localPubSubAddress); + } + + public TopicManager getTopicManager(String pubSubAddress) { + return topicManagers.computeIfAbsent(pubSubAddress, this::createTopicManager); + } + + public Collection getAllTopicManagers() { + return topicManagers.values(); + } + + @Override + public void close() { + long startTime = System.currentTimeMillis(); + List> closeFutures = new ArrayList<>(topicManagers.size()); + for (TopicManager topicManager: topicManagers.values()) { + closeFutures.add(topicManager.closeAsync()); + } + try { + CompletableFuture.allOf(closeFutures.toArray(new CompletableFuture[0])).get(2, TimeUnit.MINUTES); + LOGGER.info( + "All TopicManagers in the TopicManagerRepository have been closed in {} ms", + System.currentTimeMillis() - startTime); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOGGER.error("Interrupted while waiting for TopicManagers to close", e); + } catch (ExecutionException | TimeoutException e) { + // log and ignore exception + LOGGER.error("Error when closing TopicManagerRepository", e); + } + } +} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerStats.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerStats.java new file mode 100644 index 0000000000..ebfb797711 --- /dev/null +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerStats.java @@ -0,0 +1,58 @@ +package com.linkedin.venice.pubsub.manager; + +import com.linkedin.venice.stats.AbstractVeniceStats; +import com.linkedin.venice.stats.TehutiUtils; +import io.tehuti.metrics.MetricsRepository; +import io.tehuti.metrics.Sensor; +import io.tehuti.metrics.stats.Avg; +import io.tehuti.metrics.stats.Max; +import io.tehuti.metrics.stats.Min; +import io.tehuti.metrics.stats.OccurrenceRate; +import java.util.EnumMap; + + +/** + * Stats for topic manager operations + */ +class TopicManagerStats extends AbstractVeniceStats { + private static final String TOPIC_MANAGER_STATS_PREFIX = "TopicManagerStats_"; + private EnumMap sensorsByTypes = null; + + enum SENSOR_TYPE { + CREATE_TOPIC, DELETE_TOPIC, LIST_ALL_TOPICS, SET_TOPIC_CONFIG, GET_ALL_TOPIC_RETENTIONS, GET_TOPIC_CONFIG, + GET_TOPIC_CONFIG_WITH_RETRY, CONTAINS_TOPIC, GET_SOME_TOPIC_CONFIGS, CONTAINS_TOPIC_WITH_RETRY, + GET_TOPIC_LATEST_OFFSETS, GET_PARTITION_LATEST_OFFSETS, PARTITIONS_FOR, GET_OFFSET_FOR_TIME, + GET_PRODUCER_TIMESTAMP_OF_LAST_DATA_MESSAGE + } + + TopicManagerStats(MetricsRepository metricsRepository, String pubSubAddress) { + super(metricsRepository, TOPIC_MANAGER_STATS_PREFIX + TehutiUtils.fixMalformedMetricName(pubSubAddress)); + if (metricsRepository == null) { + return; + } + sensorsByTypes = new EnumMap<>(SENSOR_TYPE.class); + for (SENSOR_TYPE sensorType: SENSOR_TYPE.values()) { + final String sensorName = sensorType.name().toLowerCase(); + sensorsByTypes.put( + sensorType, + registerSensorIfAbsent( + sensorName, + new OccurrenceRate(), + new Max(), + new Min(), + new Avg(), + TehutiUtils.getPercentileStat(getName() + AbstractVeniceStats.DELIMITER + sensorName))); + } + } + + EnumMap getSensorsByTypes() { + return sensorsByTypes; + } + + void recordLatency(SENSOR_TYPE sensorType, long startTimeInMs) { + if (sensorsByTypes == null || sensorType == null) { + return; + } + sensorsByTypes.get(sensorType).record(System.currentTimeMillis() - startTimeInMs); + } +} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java new file mode 100644 index 0000000000..e1ed57e77f --- /dev/null +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java @@ -0,0 +1,737 @@ +package com.linkedin.venice.pubsub.manager; + +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_OFFSET_API_TIMEOUT_DURATION_DEFAULT_VALUE; +import static com.linkedin.venice.pubsub.manager.TopicManagerStats.SENSOR_TYPE.CONTAINS_TOPIC; +import static com.linkedin.venice.pubsub.manager.TopicManagerStats.SENSOR_TYPE.GET_OFFSET_FOR_TIME; +import static com.linkedin.venice.pubsub.manager.TopicManagerStats.SENSOR_TYPE.GET_PARTITION_LATEST_OFFSETS; +import static com.linkedin.venice.pubsub.manager.TopicManagerStats.SENSOR_TYPE.GET_PRODUCER_TIMESTAMP_OF_LAST_DATA_MESSAGE; +import static com.linkedin.venice.pubsub.manager.TopicManagerStats.SENSOR_TYPE.GET_TOPIC_LATEST_OFFSETS; +import static com.linkedin.venice.pubsub.manager.TopicManagerStats.SENSOR_TYPE.PARTITIONS_FOR; +import static java.util.concurrent.TimeUnit.MILLISECONDS; + +import com.linkedin.venice.annotation.Threadsafe; +import com.linkedin.venice.exceptions.VeniceException; +import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; +import com.linkedin.venice.message.KafkaKey; +import com.linkedin.venice.pubsub.PubSubConstants; +import com.linkedin.venice.pubsub.PubSubTopicPartitionInfo; +import com.linkedin.venice.pubsub.api.PubSubAdminAdapter; +import com.linkedin.venice.pubsub.api.PubSubConsumerAdapter; +import com.linkedin.venice.pubsub.api.PubSubMessage; +import com.linkedin.venice.pubsub.api.PubSubMessageDeserializer; +import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.api.PubSubTopicPartition; +import com.linkedin.venice.pubsub.api.exceptions.PubSubOpTimeoutException; +import com.linkedin.venice.pubsub.api.exceptions.PubSubTopicDoesNotExistException; +import com.linkedin.venice.stats.StatsErrorCode; +import com.linkedin.venice.utils.DaemonThreadFactory; +import com.linkedin.venice.utils.RetryUtils; +import com.linkedin.venice.utils.Utils; +import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap; +import it.unimi.dsi.fastutil.ints.Int2LongMap; +import it.unimi.dsi.fastutil.ints.Int2LongMaps; +import it.unimi.dsi.fastutil.ints.Int2LongOpenHashMap; +import java.io.Closeable; +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Supplier; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + + +@Threadsafe +class TopicMetadataFetcher implements Closeable { + private static final Logger LOGGER = LogManager.getLogger(TopicMetadataFetcher.class); + private static final int DEFAULT_MAX_RETRIES_FOR_POPULATING_TMD_CACHE_ENTRY = 5; + private static final Duration INITIAL_RETRY_DELAY = Duration.ofMillis(100); + private static final List> PUBSUB_RETRIABLE_FAILURES = + Arrays.asList(PubSubTopicDoesNotExistException.class, PubSubOpTimeoutException.class); + + /** + * Blocking queue is used to ensure single-threaded access to a consumer as PubSubConsumerAdapter + * implementations are not guaranteed to be thread-safe. + */ + private final BlockingQueue pubSubConsumerPool; + private final List closeables; + private final ThreadPoolExecutor threadPoolExecutor; + private final PubSubAdminAdapter pubSubAdminAdapter; + private final TopicManagerStats stats; + private final String pubSubClusterAddress; + + /** + * The following caches store metadata related to topics, including details such as the latest offset, + * earliest offset, and topic existence. Cached values are set to expire after a specified time-to-live + * duration. When a value is not present in the cache, it is fetched synchronously. If a cached value is + * expired, it is retrieved and updated asynchronously. + * To avoid overwhelming metadata fetcher, only one asynchronous request is issued at a time for a given + * key within the specific cache. + */ + private final Map> topicExistenceCache = new VeniceConcurrentHashMap<>(); + private final Map> latestOffsetCache = new VeniceConcurrentHashMap<>(); + private final Map> lastProducerTimestampCache = + new VeniceConcurrentHashMap<>(); + private final long cachedEntryTtlInNs; + + TopicMetadataFetcher( + String pubSubClusterAddress, + TopicManagerContext topicManagerContext, + TopicManagerStats stats, + PubSubAdminAdapter pubSubAdminAdapter) { + this.pubSubClusterAddress = pubSubClusterAddress; + this.stats = stats; + this.pubSubAdminAdapter = pubSubAdminAdapter; + this.pubSubConsumerPool = new LinkedBlockingQueue<>(topicManagerContext.getTopicMetadataFetcherConsumerPoolSize()); + this.closeables = new ArrayList<>(topicManagerContext.getTopicMetadataFetcherConsumerPoolSize()); + this.cachedEntryTtlInNs = MILLISECONDS.toNanos(topicManagerContext.getTopicOffsetCheckIntervalMs()); + PubSubMessageDeserializer pubSubMessageDeserializer = PubSubMessageDeserializer.getInstance(); + for (int i = 0; i < topicManagerContext.getTopicMetadataFetcherConsumerPoolSize(); i++) { + PubSubConsumerAdapter pubSubConsumerAdapter = topicManagerContext.getPubSubConsumerAdapterFactory() + .create( + topicManagerContext.getPubSubProperties(pubSubClusterAddress), + false, + pubSubMessageDeserializer, + pubSubClusterAddress); + + closeables.add(pubSubConsumerAdapter); + if (!pubSubConsumerPool.offer(pubSubConsumerAdapter)) { + throw new VeniceException("Failed to initialize consumer pool for topic metadata fetcher"); + } + } + + /** + * A thread-pool used to execute all async methods in this class + */ + threadPoolExecutor = new ThreadPoolExecutor( + topicManagerContext.getTopicMetadataFetcherThreadPoolSize(), + topicManagerContext.getTopicMetadataFetcherThreadPoolSize(), + 15L, + TimeUnit.MINUTES, + new LinkedBlockingQueue<>(), + new DaemonThreadFactory("TopicMetadataFetcherThreadPool")); + threadPoolExecutor.allowCoreThreadTimeOut(true); + + LOGGER.info( + "Initialized TopicMetadataFetcher for pubSubClusterAddress: {} with consumer pool size: {} and thread pool size: {}", + pubSubClusterAddress, + topicManagerContext.getTopicMetadataFetcherConsumerPoolSize(), + topicManagerContext.getTopicMetadataFetcherThreadPoolSize()); + } + + // Constructor for unit testing only + TopicMetadataFetcher( + String pubSubClusterAddress, + TopicManagerStats stats, + PubSubAdminAdapter pubSubAdminAdapter, + BlockingQueue pubSubConsumerPool, + ThreadPoolExecutor threadPoolExecutor, + long cachedEntryTtlInNs) { + this.pubSubClusterAddress = pubSubClusterAddress; + this.stats = stats; + this.pubSubAdminAdapter = pubSubAdminAdapter; + this.pubSubConsumerPool = pubSubConsumerPool; + this.threadPoolExecutor = threadPoolExecutor; + this.cachedEntryTtlInNs = cachedEntryTtlInNs; + this.closeables = new ArrayList<>(pubSubConsumerPool); + } + + // acquire the consumer from the pool + private PubSubConsumerAdapter acquireConsumer() { + try { + return pubSubConsumerPool.take(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new VeniceException("Interrupted while acquiring pubSubConsumerAdapter", e); + } + } + + // release the consumer back to the pool + private void releaseConsumer(PubSubConsumerAdapter pubSubConsumerAdapter) { + try { + pubSubConsumerPool.put(pubSubConsumerAdapter); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new VeniceException("Interrupted while releasing pubSubConsumerAdapter", e); + } + } + + // package private for unit testing + void validateTopicPartition(PubSubTopicPartition pubSubTopicPartition) { + Objects.requireNonNull(pubSubTopicPartition, "pubSubTopicPartition cannot be null"); + if (pubSubTopicPartition.getPartitionNumber() < 0) { + throw new IllegalArgumentException("Invalid partition number: " + pubSubTopicPartition.getPartitionNumber()); + } + if (containsTopicCached(pubSubTopicPartition.getPubSubTopic())) { + return; + } + boolean topicExists = RetryUtils.executeWithMaxAttempt( + () -> containsTopic(pubSubTopicPartition.getPubSubTopic()), + 3, + INITIAL_RETRY_DELAY, + PUBSUB_RETRIABLE_FAILURES); + if (!topicExists) { + throw new PubSubTopicDoesNotExistException("Topic does not exist: " + pubSubTopicPartition.getPubSubTopic()); + } + } + + @Override + public void close() throws IOException { + LOGGER.info( + "Closing TopicMetadataFetcher for pubSubClusterAddress: {} with num of consumers: {}", + pubSubClusterAddress, + closeables.size()); + threadPoolExecutor.shutdown(); + try { + if (!threadPoolExecutor.awaitTermination(50, MILLISECONDS)) { + threadPoolExecutor.shutdownNow(); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + long waitUntil = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(30); + while (System.currentTimeMillis() <= waitUntil && !closeables.isEmpty()) { + LOGGER.info( + "Waiting for {} consumers to be closed for pubSubClusterAddress: {} remainingTime: {} ms", + closeables.size(), + pubSubClusterAddress, + waitUntil - System.currentTimeMillis()); + PubSubConsumerAdapter pubSubConsumerAdapter = null; + try { + pubSubConsumerAdapter = pubSubConsumerPool.poll(5, MILLISECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + if (pubSubConsumerAdapter != null) { + closeables.remove(pubSubConsumerAdapter); + Utils.closeQuietlyWithErrorLogged(pubSubConsumerAdapter); + } + } + if (closeables.isEmpty()) { + LOGGER.info("Closed all metadata fetcher consumers for pubSubClusterAddress: {}", pubSubClusterAddress); + return; + } + LOGGER.warn( + "Failed to close metadata fetcher consumers for pubSubClusterAddress: {}. Forcibly closing " + + "remaining {} consumers. This may be caused by an improper shutdown sequence.", + pubSubClusterAddress, + closeables.size()); + for (Closeable closeable: closeables) { + Utils.closeQuietlyWithErrorLogged(closeable); + } + } + + /** + * Check if a topic exists in the pubsub cluster + * @param topic topic to check + * @return true if the topic exists, false otherwise + */ + boolean containsTopic(PubSubTopic topic) { + long startTime = System.currentTimeMillis(); + boolean containsTopic = pubSubAdminAdapter.containsTopic(topic); + stats.recordLatency(CONTAINS_TOPIC, startTime); + return containsTopic; + } + + CompletableFuture containsTopicAsync(PubSubTopic topic) { + return CompletableFuture.supplyAsync(() -> containsTopic(topic), threadPoolExecutor); + } + + boolean containsTopicCached(PubSubTopic topic) { + ValueAndExpiryTime cachedValue = + topicExistenceCache.computeIfAbsent(topic, k -> new ValueAndExpiryTime<>(containsTopic(topic))); + updateCacheAsync(topic, cachedValue, topicExistenceCache, () -> containsTopicAsync(topic)); + return cachedValue.getValue(); + } + + /** + * Get the latest offsets for all partitions of a topic. This is a blocking call. + * @param topic topic to get latest offsets for + * @return a map of partition id to latest offset. If the topic does not exist, an empty map is returned. + */ + Int2LongMap getTopicLatestOffsets(PubSubTopic topic) { + PubSubConsumerAdapter pubSubConsumerAdapter = acquireConsumer(); + try { + long startTime = System.currentTimeMillis(); + List partitionInfoList = pubSubConsumerAdapter.partitionsFor(topic); + stats.recordLatency(PARTITIONS_FOR, startTime); + + if (partitionInfoList == null || partitionInfoList.isEmpty()) { + LOGGER.warn("Topic: {} may not exist or has no partitions. Returning empty map.", topic); + return Int2LongMaps.EMPTY_MAP; + } + + Collection topicPartitions = new HashSet<>(partitionInfoList.size()); + for (PubSubTopicPartitionInfo partitionInfo: partitionInfoList) { + topicPartitions.add(partitionInfo.getTopicPartition()); + } + + startTime = System.currentTimeMillis(); + Map offsetsMap = + pubSubConsumerAdapter.endOffsets(topicPartitions, PUBSUB_OFFSET_API_TIMEOUT_DURATION_DEFAULT_VALUE); + stats.recordLatency(GET_TOPIC_LATEST_OFFSETS, startTime); + Int2LongMap result = new Int2LongOpenHashMap(offsetsMap.size()); + for (Map.Entry entry: offsetsMap.entrySet()) { + result.put(entry.getKey().getPartitionNumber(), entry.getValue().longValue()); + } + return result; + } finally { + releaseConsumer(pubSubConsumerAdapter); + } + } + + /** + * Get information about all partitions of a topic. This is a blocking call. + * @param topic topic to get partition info for + * @return a list of partition info. If the topic does not exist, NULL is returned. + */ + List getTopicPartitionInfo(PubSubTopic topic) { + PubSubConsumerAdapter pubSubConsumerAdapter = acquireConsumer(); + try { + long startTime = System.currentTimeMillis(); + List res = pubSubConsumerAdapter.partitionsFor(topic); + stats.recordLatency(PARTITIONS_FOR, startTime); + return res; + } finally { + releaseConsumer(pubSubConsumerAdapter); + } + } + + /** + * Retrieves the latest offset for the specified partition of a PubSub topic. + * + * @param pubSubTopicPartition The topic and partition number to query for the latest offset. + * @return The latest offset for the specified partition. + * @throws PubSubTopicDoesNotExistException If the topic does not exist. + * @throws IllegalArgumentException If the partition number is negative. + * @throws VeniceException If the offset returned by the consumer is null. + * This could indicate a bug in the PubSubConsumerAdapter implementation. + * @throws PubSubOpTimeoutException If the consumer times out. This could indicate that the topic does not exist + * or the partition does not exist. + */ + long getLatestOffset(PubSubTopicPartition pubSubTopicPartition) { + PubSubConsumerAdapter pubSubConsumerAdapter = acquireConsumer(); + try { + long startTime = System.currentTimeMillis(); + Map offsetMap = pubSubConsumerAdapter + .endOffsets(Collections.singleton(pubSubTopicPartition), PUBSUB_OFFSET_API_TIMEOUT_DURATION_DEFAULT_VALUE); + stats.recordLatency(GET_PARTITION_LATEST_OFFSETS, startTime); + Long offset = offsetMap.get(pubSubTopicPartition); + if (offset == null) { + LOGGER.error("Received null offset for topic-partition: {}", pubSubTopicPartition); + // This should never happen; if it does, it's a bug in the PubSubConsumerAdapter implementation + throw new VeniceException("Got null as latest offset for: " + pubSubTopicPartition); + } + return offset; + } finally { + releaseConsumer(pubSubConsumerAdapter); + } + } + + long getLatestOffsetWithRetries(PubSubTopicPartition pubSubTopicPartition, int retries) { + return RetryUtils.executeWithMaxAttemptAndExponentialBackoff(() -> { + validateTopicPartition(pubSubTopicPartition); + return getLatestOffset(pubSubTopicPartition); + }, retries, INITIAL_RETRY_DELAY, Duration.ofSeconds(5), Duration.ofMinutes(5), PUBSUB_RETRIABLE_FAILURES); + } + + CompletableFuture getLatestOffsetWithRetriesAsync(PubSubTopicPartition pubSubTopicPartition, int retries) { + return CompletableFuture + .supplyAsync(() -> getLatestOffsetWithRetries(pubSubTopicPartition, retries), threadPoolExecutor); + } + + long getLatestOffsetCached(PubSubTopicPartition pubSubTopicPartition) { + ValueAndExpiryTime cachedValue; + try { + cachedValue = latestOffsetCache.computeIfAbsent(pubSubTopicPartition, k -> { + long latestOffset = + getLatestOffsetWithRetries(pubSubTopicPartition, DEFAULT_MAX_RETRIES_FOR_POPULATING_TMD_CACHE_ENTRY); + return new ValueAndExpiryTime<>(latestOffset); + }); + } catch (PubSubTopicDoesNotExistException | PubSubOpTimeoutException e) { + LOGGER.error("Failed to get end offset for topic-partition: {}", pubSubTopicPartition, e); + return StatsErrorCode.LAG_MEASUREMENT_FAILURE.code; + } + updateCacheAsync( + pubSubTopicPartition, + cachedValue, + latestOffsetCache, + () -> getLatestOffsetWithRetriesAsync( + pubSubTopicPartition, + DEFAULT_MAX_RETRIES_FOR_POPULATING_TMD_CACHE_ENTRY)); + return cachedValue.getValue(); + } + + // load the cache with the latest offset + void populateCacheWithLatestOffset(PubSubTopicPartition pubSubTopicPartition) { + CompletableFuture.runAsync(() -> { + validateTopicPartition(pubSubTopicPartition); + getLatestOffsetCached(pubSubTopicPartition); + }, threadPoolExecutor); + } + + /** + * Get the offset for a given timestamp. This is a blocking call. + * @param pubSubTopicPartition topic partition to get the offset for + * @param timestamp timestamp to get the offset for + * @return the offset for the given timestamp + */ + long getOffsetForTime(PubSubTopicPartition pubSubTopicPartition, long timestamp) { + validateTopicPartition(pubSubTopicPartition); + // We start by retrieving the latest offset. If the provided timestamp is out of range, + // we return the latest offset. This ensures that we don't miss any records produced + // after the 'offsetForTime' call when the latest offset is obtained after timestamp checking. + long latestOffset = getLatestOffset(pubSubTopicPartition); + + PubSubConsumerAdapter pubSubConsumerAdapter = acquireConsumer(); + try { + long startTime = System.currentTimeMillis(); + Long result = pubSubConsumerAdapter + .offsetForTime(pubSubTopicPartition, timestamp, PUBSUB_OFFSET_API_TIMEOUT_DURATION_DEFAULT_VALUE); + stats.recordLatency(GET_OFFSET_FOR_TIME, startTime); + if (result != null) { + return result; + } + // When the offset is null, it indicates that the provided timestamp is either out of range + // or the topic does not contain any messages. In such cases, we log a warning and return + // the offset of the last message available for the topic-partition. + LOGGER.warn( + "Received null offset for timestamp: {} on topic-partition: {}. This may occur if the timestamp is beyond " + + "the latest message timestamp or if the topic has no messages. Returning the latest offset: {}", + timestamp, + pubSubTopicPartition, + latestOffset); + return latestOffset; + } finally { + releaseConsumer(pubSubConsumerAdapter); + } + } + + /** + * Get the offset for a given timestamp with retries. This is a blocking call. + * @param pubSubTopicPartition topic partition to get the offset for + * @param timestamp timestamp to get the offset for + * @param retries number of retries + * @return the offset for the given timestamp + */ + long getOffsetForTimeWithRetries(PubSubTopicPartition pubSubTopicPartition, long timestamp, int retries) { + return RetryUtils.executeWithMaxAttemptAndExponentialBackoff( + () -> getOffsetForTime(pubSubTopicPartition, timestamp), + retries, + INITIAL_RETRY_DELAY, + Duration.ofSeconds(5), + Duration.ofMinutes(5), + PUBSUB_RETRIABLE_FAILURES); + } + + /** + * Get the producer timestamp of the last data message in a topic partition + * @param pubSubTopicPartition topic partition to get the producer timestamp for + * @return the producer timestamp of the last data message in the topic partition + * @throws VeniceException if failed to get the producer timestamp + */ + long getProducerTimestampOfLastDataMessage(PubSubTopicPartition pubSubTopicPartition) { + int fetchSize = 10; + int totalAttempts = 3; + int fetchedRecordsCount; + long startTime = System.currentTimeMillis(); + do { + List> lastConsumedRecords = + consumeLatestRecords(pubSubTopicPartition, fetchSize); + // if there are no records in this topic partition, return a special timestamp + if (lastConsumedRecords.isEmpty()) { + return PubSubConstants.PUBSUB_NO_PRODUCER_TIME_IN_EMPTY_TOPIC_PARTITION; + } + fetchedRecordsCount = lastConsumedRecords.size(); + // iterate in reverse order to find the first data message (not control message) from the end + for (int i = lastConsumedRecords.size() - 1; i >= 0; i--) { + PubSubMessage record = lastConsumedRecords.get(i); + if (!record.getKey().isControlMessage()) { + stats.recordLatency(GET_PRODUCER_TIMESTAMP_OF_LAST_DATA_MESSAGE, startTime); + // note that the timestamp is the producer timestamp and not the pubsub message (broker) timestamp + return record.getValue().getProducerMetadata().getMessageTimestamp(); + } + } + fetchSize = 50; + } while (--totalAttempts > 0); + + String errorMsg = String.format( + "No data message found in topic-partition: %s when fetching producer timestamp of the last data message. Consumed %d records from the end.", + pubSubTopicPartition, + fetchedRecordsCount); + LOGGER.warn(errorMsg); + throw new VeniceException(errorMsg); + } + + long getProducerTimestampOfLastDataMessageWithRetries(PubSubTopicPartition pubSubTopicPartition, int retries) { + return RetryUtils.executeWithMaxAttempt( + () -> getProducerTimestampOfLastDataMessage(pubSubTopicPartition), + retries, + INITIAL_RETRY_DELAY, + PUBSUB_RETRIABLE_FAILURES); + } + + CompletableFuture getProducerTimestampOfLastDataMessageWithRetriesAsync( + PubSubTopicPartition pubSubTopicPartition, + int retries) { + return CompletableFuture.supplyAsync( + () -> getProducerTimestampOfLastDataMessageWithRetries(pubSubTopicPartition, retries), + threadPoolExecutor); + } + + long getProducerTimestampOfLastDataMessageCached(PubSubTopicPartition pubSubTopicPartition) { + ValueAndExpiryTime cachedValue; + try { + cachedValue = lastProducerTimestampCache.computeIfAbsent(pubSubTopicPartition, k -> { + long producerTimestamp = getProducerTimestampOfLastDataMessageWithRetries( + pubSubTopicPartition, + DEFAULT_MAX_RETRIES_FOR_POPULATING_TMD_CACHE_ENTRY); + return new ValueAndExpiryTime<>(producerTimestamp); + }); + } catch (PubSubTopicDoesNotExistException | PubSubOpTimeoutException e) { + LOGGER.error("Failed to get producer timestamp for topic-partition: {}", pubSubTopicPartition, e); + return StatsErrorCode.LAG_MEASUREMENT_FAILURE.code; + } + + updateCacheAsync( + pubSubTopicPartition, + cachedValue, + lastProducerTimestampCache, + () -> getProducerTimestampOfLastDataMessageWithRetriesAsync( + pubSubTopicPartition, + DEFAULT_MAX_RETRIES_FOR_POPULATING_TMD_CACHE_ENTRY)); + + return cachedValue.getValue(); + } + + /** + * This method retrieves last {@code lastRecordsCount} records from a topic partition and there are 4 steps below. + * 1. Find the current end offset N + * 2. Seek back {@code lastRecordsCount} records from the end offset N + * 3. Keep consuming records until the last consumed offset is greater than or equal to N + * 4. Return all consumed records + * + * There are 2 things to note: + * 1. When this method returns, these returned records are not necessarily the "last" records because after step 2, + * there could be more records produced to this topic partition and this method only consume records until the end + * offset retrieved at the above step 2. + * + * 2. This method might return more than {@code lastRecordsCount} records since the consumer poll method gets a batch + * of consumer records each time and the batch size is arbitrary. + */ + List> consumeLatestRecords( + PubSubTopicPartition pubSubTopicPartition, + int lastRecordsCount) { + if (lastRecordsCount < 1) { + throw new IllegalArgumentException( + "Last record count must be greater than or equal to 1. Got: " + lastRecordsCount); + } + validateTopicPartition(pubSubTopicPartition); + PubSubConsumerAdapter pubSubConsumerAdapter = acquireConsumer(); + boolean subscribed = false; + try { + // find the end offset + Map offsetMap = pubSubConsumerAdapter.endOffsets( + Collections.singletonList(pubSubTopicPartition), + PUBSUB_OFFSET_API_TIMEOUT_DURATION_DEFAULT_VALUE); + if (offsetMap == null || offsetMap.isEmpty()) { + throw new VeniceException("Failed to get the end offset for topic-partition: " + pubSubTopicPartition); + } + long latestOffset = offsetMap.get(pubSubTopicPartition); + if (latestOffset <= 0) { + return Collections.emptyList(); // no records in this topic partition + } + + // find the beginning offset + long earliestOffset = + pubSubConsumerAdapter.beginningOffset(pubSubTopicPartition, PUBSUB_OFFSET_API_TIMEOUT_DURATION_DEFAULT_VALUE); + if (earliestOffset == latestOffset) { + return Collections.emptyList(); // no records in this topic partition + } + + // consume latest records + long consumePastOffset = Math.max(Math.max(latestOffset - lastRecordsCount, earliestOffset) - 1, -1); + pubSubConsumerAdapter.subscribe(pubSubTopicPartition, consumePastOffset); + subscribed = true; + LOGGER.info( + "Subscribed to topic partition: {} starting from offset: {}", + pubSubTopicPartition, + consumePastOffset + 1); + + List> allConsumedRecords = new ArrayList<>(lastRecordsCount); + + // Keep consuming records from that topic-partition until the last consumed record's + // offset is greater or equal to the partition end offset retrieved before. + do { + List> consumedBatch = Collections.emptyList(); + int pollAttempt = 1; + while (pollAttempt <= PubSubConstants.PUBSUB_CONSUMER_POLLING_FOR_METADATA_RETRY_MAX_ATTEMPT + && (consumedBatch == null || consumedBatch.isEmpty())) { + consumedBatch = + pubSubConsumerAdapter.poll(Math.max(10, PUBSUB_OFFSET_API_TIMEOUT_DURATION_DEFAULT_VALUE.toMillis())) + .get(pubSubTopicPartition); + pollAttempt++; + } + + // If batch is still empty after retries, give up. + if (consumedBatch == null || consumedBatch.isEmpty()) { + String message = String.format( + "Failed to get records from topic-partition: %s after %d attempts", + pubSubTopicPartition, + PubSubConstants.PUBSUB_CONSUMER_POLLING_FOR_METADATA_RETRY_MAX_ATTEMPT); + LOGGER.error(message); + throw new VeniceException(message); + } + allConsumedRecords.addAll(consumedBatch); + } while (allConsumedRecords.get(allConsumedRecords.size() - 1).getOffset() + 1 < latestOffset); + + return allConsumedRecords; + } finally { + if (subscribed) { + pubSubConsumerAdapter.unSubscribe(pubSubTopicPartition); + } + releaseConsumer(pubSubConsumerAdapter); + } + } + + void invalidateKey(PubSubTopicPartition pubSubTopicPartition) { + latestOffsetCache.remove(pubSubTopicPartition); + lastProducerTimestampCache.remove(pubSubTopicPartition); + lastProducerTimestampCache.remove(pubSubTopicPartition); + } + + CompletableFuture invalidateKeyAsync(PubSubTopic pubSubTopic) { + return CompletableFuture.runAsync(() -> invalidateKey(pubSubTopic)); + } + + void invalidateKey(PubSubTopic pubSubTopic) { + long startTime = System.currentTimeMillis(); + LOGGER.info("Invalidating cache for topic: {}", pubSubTopic); + topicExistenceCache.remove(pubSubTopic); + Set topicPartitions = new HashSet<>(); + + for (PubSubTopicPartition pubSubTopicPartition: latestOffsetCache.keySet()) { + if (pubSubTopicPartition.getPubSubTopic().equals(pubSubTopic)) { + topicPartitions.add(pubSubTopicPartition); + } + } + for (PubSubTopicPartition pubSubTopicPartition: lastProducerTimestampCache.keySet()) { + if (pubSubTopicPartition.getPubSubTopic().equals(pubSubTopic)) { + topicPartitions.add(pubSubTopicPartition); + } + } + + for (PubSubTopicPartition pubSubTopicPartition: topicPartitions) { + invalidateKey(pubSubTopicPartition); + } + + LOGGER.info("Invalidated cache for topic: {} in {} ms", pubSubTopic, System.currentTimeMillis() - startTime); + } + + /** + * Asynchronously updates the cache for the specified key if necessary. + * + * If there is no entry in the cache or the entry has expired and no update is in progress, + * this method will attempt to update the cache using the provided {@code completableFutureSupplier}. + * + * @param the type of the cache key + * @param the type of the cached value + * @param key the key for which the cache should be updated + * @param cachedValue the current cached value for the specified key + * @param cache the cache to be updated + * @param completableFutureSupplier a supplier providing a CompletableFuture for the asynchronous update + */ + void updateCacheAsync( + K key, + ValueAndExpiryTime cachedValue, + Map> cache, + Supplier> completableFutureSupplier) { + + if (cachedValue != null + && (cachedValue.getExpiryTimeNs() > System.nanoTime() || !cachedValue.tryAcquireUpdateLock())) { + return; + } + + completableFutureSupplier.get().whenComplete((value, throwable) -> { + if (throwable != null) { + cache.remove(key); + return; + } + putLatestValueInCache(key, value, cache); + }); + } + + // update with the latest value + private void putLatestValueInCache(K key, T latestVal, Map> cache) { + cache.compute(key, (k, v) -> { + if (v == null) { + return new ValueAndExpiryTime<>(latestVal); + } else { + v.updateValue(latestVal); + return v; + } + }); + } + + /** + * This class is used to store the value and expiry time of a cached value. + * + * Visible for unit testing. + */ + class ValueAndExpiryTime { + private volatile T value; + private volatile long expiryTimeNs; + private final AtomicBoolean isUpdateInProgress = new AtomicBoolean(false); + + ValueAndExpiryTime(T value) { + this.value = value; + this.expiryTimeNs = System.nanoTime() + cachedEntryTtlInNs; + } + + T getValue() { + return value; + } + + long getExpiryTimeNs() { + return expiryTimeNs; + } + + boolean tryAcquireUpdateLock() { + return isUpdateInProgress.compareAndSet(false, true); + } + + void updateValue(T value) { + this.value = value; + this.expiryTimeNs = System.nanoTime() + cachedEntryTtlInNs; // update the expiry time + this.isUpdateInProgress.set(false); // release the lock + } + + // test only methods + void setExpiryTimeNs(long expiryTimeNs) { + this.expiryTimeNs = expiryTimeNs; + } + + void setUpdateInProgressStatus(boolean status) { + this.isUpdateInProgress.set(status); + } + } + + @Override + public String toString() { + return "TopicMetadataFetcher{" + "pubSubClusterAddress='" + pubSubClusterAddress + '\'' + ", numOfConsumers=" + + closeables.size() + ", threadPoolExecutor=" + threadPoolExecutor + '}'; + } +} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/stats/PubSubAdminWrapperStats.java b/internal/venice-common/src/main/java/com/linkedin/venice/stats/PubSubAdminWrapperStats.java deleted file mode 100644 index ae1d0bd641..0000000000 --- a/internal/venice-common/src/main/java/com/linkedin/venice/stats/PubSubAdminWrapperStats.java +++ /dev/null @@ -1,65 +0,0 @@ -package com.linkedin.venice.stats; - -import com.linkedin.venice.utils.Pair; -import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap; -import io.tehuti.metrics.MetricsRepository; -import io.tehuti.metrics.Sensor; -import io.tehuti.metrics.stats.Avg; -import io.tehuti.metrics.stats.Max; -import io.tehuti.metrics.stats.Min; -import io.tehuti.metrics.stats.OccurrenceRate; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; - - -public class PubSubAdminWrapperStats extends AbstractVeniceStats { - private static final Map, PubSubAdminWrapperStats> PUB_SUB_ADMIN_WRAPPER_STATS_SINGLETON_MAP = - new VeniceConcurrentHashMap<>(); - - public enum OCCURRENCE_LATENCY_SENSOR_TYPE { - CREATE_TOPIC, DELETE_TOPIC, LIST_ALL_TOPICS, SET_TOPIC_CONFIG, GET_ALL_TOPIC_RETENTIONS, GET_TOPIC_CONFIG, - GET_TOPIC_CONFIG_WITH_RETRY, CONTAINS_TOPIC, GET_SOME_TOPIC_CONFIGS, @Deprecated - IS_TOPIC_DELETION_UNDER_WAY, DESCRIBE_TOPICS, CONTAINS_TOPIC_WITH_RETRY, CLOSE - } - - private final Map sensorsByTypes; - - /** - * This singleton function will guarantee for a unique pair of MetricsRepository and stat prefix, - * there should be only one instance of {@link PubSubAdminWrapperStats} created. - * This is trying to avoid the metric registration conflicts caused by multiple instances of this class. - * - * For other {@link AbstractVeniceStats} implementations, if it is not easy to pass around a singleton - * among different classes, they could choose to adopt this singleton pattern. - */ - public static PubSubAdminWrapperStats getInstance(MetricsRepository metricsRepository, String resourceName) { - return PUB_SUB_ADMIN_WRAPPER_STATS_SINGLETON_MAP.computeIfAbsent( - Pair.create(metricsRepository, TehutiUtils.fixMalformedMetricName(resourceName)), - k -> new PubSubAdminWrapperStats(k.getFirst(), k.getSecond())); - } - - private PubSubAdminWrapperStats(MetricsRepository metricsRepository, String resourceName) { - super(metricsRepository, resourceName); - Map tmpRateSensorsByTypes = - new HashMap<>(OCCURRENCE_LATENCY_SENSOR_TYPE.values().length); - for (OCCURRENCE_LATENCY_SENSOR_TYPE sensorType: OCCURRENCE_LATENCY_SENSOR_TYPE.values()) { - final String sensorName = sensorType.name().toLowerCase(); - tmpRateSensorsByTypes.put( - sensorType, - registerSensorIfAbsent( - sensorName, - new OccurrenceRate(), - new Max(), - new Min(), - new Avg(), - TehutiUtils.getPercentileStat(getName() + AbstractVeniceStats.DELIMITER + sensorName))); - } - - this.sensorsByTypes = Collections.unmodifiableMap(tmpRateSensorsByTypes); - } - - public void recordLatency(OCCURRENCE_LATENCY_SENSOR_TYPE sensor_type, long requestLatencyMs) { - sensorsByTypes.get(sensor_type).record(requestLatencyMs); - } -} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/system/store/MetaStoreWriter.java b/internal/venice-common/src/main/java/com/linkedin/venice/system/store/MetaStoreWriter.java index 6bfdacab8e..7cff374f0e 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/system/store/MetaStoreWriter.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/system/store/MetaStoreWriter.java @@ -3,7 +3,6 @@ import com.linkedin.venice.common.VeniceSystemStoreType; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.helix.HelixReadOnlyZKSharedSchemaRepository; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.meta.Instance; import com.linkedin.venice.meta.Store; import com.linkedin.venice.meta.StoreConfig; @@ -11,6 +10,7 @@ import com.linkedin.venice.meta.ZKStore; import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.pushmonitor.ExecutionStatus; import com.linkedin.venice.schema.GeneratedSchemaID; import com.linkedin.venice.schema.SchemaEntry; diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/utils/CachedCallable.java b/internal/venice-common/src/main/java/com/linkedin/venice/utils/CachedCallable.java deleted file mode 100644 index 06f698a3c1..0000000000 --- a/internal/venice-common/src/main/java/com/linkedin/venice/utils/CachedCallable.java +++ /dev/null @@ -1,43 +0,0 @@ -package com.linkedin.venice.utils; - -import java.util.concurrent.Callable; -import java.util.concurrent.atomic.AtomicReference; - - -/** - * CachedCallable is useful when referring to expensive resources. - * Return the same reference if it is fresh (within {@link #ttlMs} - * or update reference if it is stale. - */ -public class CachedCallable implements Callable { - private final Callable inner; - private final long ttlMs; - private final Time time; - private final AtomicReference valueRef; - - private volatile long lastCalledMs; - - public CachedCallable(Callable inner, long ttlMs) { - this(inner, ttlMs, SystemTime.INSTANCE); - } - - public CachedCallable(Callable inner, long ttlMs, Time time) { - this.inner = inner; - this.ttlMs = ttlMs; - this.time = time; - valueRef = new AtomicReference<>(); - } - - public T call() throws Exception { - T value = valueRef.get(); - long now = time.getMilliseconds(); - if (value == null || now - lastCalledMs > ttlMs) { - T newValue = inner.call(); - if (valueRef.compareAndSet(value, newValue)) { - lastCalledMs = now; - return newValue; - } - } - return valueRef.get(); - } -} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/utils/StoreUtils.java b/internal/venice-common/src/main/java/com/linkedin/venice/utils/StoreUtils.java new file mode 100644 index 0000000000..d6b77f7fda --- /dev/null +++ b/internal/venice-common/src/main/java/com/linkedin/venice/utils/StoreUtils.java @@ -0,0 +1,29 @@ +package com.linkedin.venice.utils; + +import com.linkedin.venice.meta.HybridStoreConfig; +import com.linkedin.venice.meta.Store; +import com.linkedin.venice.pubsub.PubSubConstants; + + +public class StoreUtils { + /** + * The default retention time for the RT topic is defined in {@link PubSubConstants#DEFAULT_TOPIC_RETENTION_POLICY_MS}, + * but if the rewind time is larger than this, then the RT topic's retention time needs to be set even higher, + * in order to guarantee that buffer replays do not lose data. In order to achieve this, the retention time is + * set to the max of either: + * + * 1. {@link PubSubConstants#DEFAULT_TOPIC_RETENTION_POLICY_MS}; or + * 2. {@link HybridStoreConfig#getRewindTimeInSeconds()} + {@link Store#getBootstrapToOnlineTimeoutInHours()} + {@value PubSubConstants#BUFFER_REPLAY_MINIMAL_SAFETY_MARGIN}; + * + * This is a convenience function, and thus must be ignored by the JSON machinery. + * + * @return the retention time for the RT topic, in milliseconds. + */ + public static long getExpectedRetentionTimeInMs(Store store, HybridStoreConfig hybridConfig) { + long rewindTimeInMs = hybridConfig.getRewindTimeInSeconds() * Time.MS_PER_SECOND; + long bootstrapToOnlineTimeInMs = (long) store.getBootstrapToOnlineTimeoutInHours() * Time.MS_PER_HOUR; + long minimumRetentionInMs = + rewindTimeInMs + bootstrapToOnlineTimeInMs + PubSubConstants.BUFFER_REPLAY_MINIMAL_SAFETY_MARGIN; + return Math.max(minimumRetentionInMs, PubSubConstants.DEFAULT_TOPIC_RETENTION_POLICY_MS); + } +} diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/meta/TestHelixReadWriteSchemaRepository.java b/internal/venice-common/src/test/java/com/linkedin/venice/meta/TestHelixReadWriteSchemaRepository.java index 0eade4d296..f66817a264 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/meta/TestHelixReadWriteSchemaRepository.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/meta/TestHelixReadWriteSchemaRepository.java @@ -6,9 +6,6 @@ import com.linkedin.venice.schema.writecompute.DerivedSchemaEntry; import com.linkedin.venice.system.store.MetaStoreWriter; import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; import java.util.Optional; import org.apache.avro.Schema; import org.mockito.Mockito; @@ -37,10 +34,8 @@ public void getDerivedSchemaCanonical() { String schemaStr1 = "{\"type\":\"record\",\"name\":\"KeyRecord\",\"fields\":[{\"name\":\"name\",\"type\":\"string\",\"doc\":\"name field\"},{\"name\":\"company1\",\"type\":\"string\"}]}"; Schema schema = Schema.parse(schemaStr); - Map> derivedSchemaEntryMap = new HashMap<>(); DerivedSchemaEntry entry = new DerivedSchemaEntry(1, 1, schema); Mockito.doReturn(true).when(storeRepository).hasStore(Mockito.anyString()); - derivedSchemaEntryMap.put(1, Arrays.asList(entry)); Mockito.doReturn(Arrays.asList(entry)).when(accessor).getAllDerivedSchemas(Mockito.eq(storeName)); GeneratedSchemaID derivedSchemaId = helixReadWriteSchemaRepository.getDerivedSchemaId(storeName, schemaStr); Assert.assertEquals(derivedSchemaId.getGeneratedSchemaVersion(), 1); diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/partitioner/TestVenicePartitioner.java b/internal/venice-common/src/test/java/com/linkedin/venice/partitioner/TestVenicePartitioner.java index d0a6a55f12..b9f6031ac0 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/partitioner/TestVenicePartitioner.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/partitioner/TestVenicePartitioner.java @@ -1,7 +1,6 @@ package com.linkedin.venice.partitioner; import java.nio.ByteBuffer; -import org.apache.kafka.common.PartitionInfo; import org.testng.Assert; import org.testng.annotations.Test; @@ -21,24 +20,23 @@ public void testConsistentPartitioning() { byte[] keyBytes = "key1".getBytes(); - PartitionInfo[] partitionArray = { new PartitionInfo("", 0, null, null, null), - new PartitionInfo("", 1, null, null, null), new PartitionInfo("", 2, null, null, null) }; + int numPartitions = 3; // Test 1 - int partition1 = vp.getPartitionId(keyBytes, partitionArray.length); - int partition2 = vp.getPartitionId(keyBytes, partitionArray.length); + int partition1 = vp.getPartitionId(keyBytes, numPartitions); + int partition2 = vp.getPartitionId(keyBytes, numPartitions); Assert.assertEquals(partition1, partition2); // Test 2 keyBytes = " ".getBytes(); - partition1 = vp.getPartitionId(keyBytes, partitionArray.length); - partition2 = vp.getPartitionId(keyBytes, partitionArray.length); + partition1 = vp.getPartitionId(keyBytes, numPartitions); + partition2 = vp.getPartitionId(keyBytes, numPartitions); Assert.assertEquals(partition1, partition2); // Test 3 keyBytes = "00000".getBytes(); - partition1 = vp.getPartitionId(keyBytes, partitionArray.length); - partition2 = vp.getPartitionId(keyBytes, partitionArray.length); + partition1 = vp.getPartitionId(keyBytes, numPartitions); + partition2 = vp.getPartitionId(keyBytes, numPartitions); Assert.assertEquals(partition1, partition2); // Test 4 diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/adapter/kafka/admin/ApacheKafkaAdminAdapterTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/adapter/kafka/admin/ApacheKafkaAdminAdapterTest.java index 5d391d9361..ecfea7c98c 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/adapter/kafka/admin/ApacheKafkaAdminAdapterTest.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/adapter/kafka/admin/ApacheKafkaAdminAdapterTest.java @@ -563,7 +563,7 @@ public void testGetAllTopicRetentions() throws ExecutionException, InterruptedEx assertEquals(result.get(pubSubTopicRepository.getTopic("t3_v2")), Long.valueOf(3)); assertEquals( result.get(pubSubTopicRepository.getTopic("t4_rt")), - Long.valueOf(PubSubConstants.UNKNOWN_TOPIC_RETENTION)); + Long.valueOf(PubSubConstants.PUBSUB_TOPIC_UNKNOWN_RETENTION)); } @Test diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicManagerContextTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicManagerContextTest.java new file mode 100644 index 0000000000..e4ea762910 --- /dev/null +++ b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicManagerContextTest.java @@ -0,0 +1,146 @@ +package com.linkedin.venice.pubsub.manager; + +import static org.mockito.Mockito.mock; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.expectThrows; + +import com.linkedin.venice.pubsub.PubSubAdminAdapterFactory; +import com.linkedin.venice.pubsub.PubSubConsumerAdapterFactory; +import com.linkedin.venice.pubsub.PubSubTopicRepository; +import com.linkedin.venice.pubsub.api.PubSubAdminAdapter; +import com.linkedin.venice.pubsub.api.PubSubConsumerAdapter; +import io.tehuti.metrics.MetricsRepository; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + + +public class TopicManagerContextTest { + private PubSubAdminAdapterFactory pubSubAdminAdapterFactory; + private PubSubConsumerAdapterFactory pubSubConsumerAdapterFactory; + private PubSubTopicRepository pubSubTopicRepository; + private MetricsRepository metricsRepository; + private TopicManagerContext.PubSubPropertiesSupplier pubSubPropertiesSupplier; + private long pubSubOperationTimeoutMs; + private long topicDeletionStatusPollIntervalMs; + private long topicMinLogCompactionLagMs; + private long topicOffsetCheckIntervalMs; + private int topicMetadataFetcherConsumerPoolSize; + private int topicMetadataFetcherThreadPoolSize; + + @BeforeMethod + public void setUp() { + pubSubAdminAdapterFactory = mock(PubSubAdminAdapterFactory.class); + pubSubConsumerAdapterFactory = mock(PubSubConsumerAdapterFactory.class); + pubSubTopicRepository = new PubSubTopicRepository(); + metricsRepository = new MetricsRepository(); + pubSubPropertiesSupplier = mock(TopicManagerContext.PubSubPropertiesSupplier.class); + pubSubOperationTimeoutMs = 11; + topicDeletionStatusPollIntervalMs = 12; + topicMinLogCompactionLagMs = 12; + topicOffsetCheckIntervalMs = 14; + topicMetadataFetcherConsumerPoolSize = 15; + topicMetadataFetcherThreadPoolSize = 16; + } + + @Test + public void testTopicManagerContext() { + TopicManagerContext topicManagerContext = + new TopicManagerContext.Builder().setPubSubAdminAdapterFactory(pubSubAdminAdapterFactory) + .setPubSubConsumerAdapterFactory(pubSubConsumerAdapterFactory) + .setPubSubTopicRepository(pubSubTopicRepository) + .setMetricsRepository(metricsRepository) + .setPubSubPropertiesSupplier(pubSubPropertiesSupplier) + .setPubSubOperationTimeoutMs(pubSubOperationTimeoutMs) + .setTopicDeletionStatusPollIntervalMs(topicDeletionStatusPollIntervalMs) + .setTopicMinLogCompactionLagMs(topicMinLogCompactionLagMs) + .setTopicOffsetCheckIntervalMs(topicOffsetCheckIntervalMs) + .setTopicMetadataFetcherConsumerPoolSize(topicMetadataFetcherConsumerPoolSize) + .setTopicMetadataFetcherThreadPoolSize(topicMetadataFetcherThreadPoolSize) + .build(); + + assertNotNull(topicManagerContext); + assertEquals(topicManagerContext.getPubSubAdminAdapterFactory(), pubSubAdminAdapterFactory); + assertEquals(topicManagerContext.getPubSubConsumerAdapterFactory(), pubSubConsumerAdapterFactory); + assertEquals(topicManagerContext.getPubSubTopicRepository(), pubSubTopicRepository); + assertEquals(topicManagerContext.getMetricsRepository(), metricsRepository); + assertEquals(topicManagerContext.getPubSubPropertiesSupplier(), pubSubPropertiesSupplier); + assertEquals(topicManagerContext.getPubSubOperationTimeoutMs(), pubSubOperationTimeoutMs); + assertEquals(topicManagerContext.getTopicDeletionStatusPollIntervalMs(), topicDeletionStatusPollIntervalMs); + assertEquals(topicManagerContext.getTopicMinLogCompactionLagMs(), topicMinLogCompactionLagMs); + assertEquals(topicManagerContext.getTopicOffsetCheckIntervalMs(), topicOffsetCheckIntervalMs); + assertEquals(topicManagerContext.getTopicMetadataFetcherConsumerPoolSize(), topicMetadataFetcherConsumerPoolSize); + assertEquals(topicManagerContext.getTopicMetadataFetcherThreadPoolSize(), topicMetadataFetcherThreadPoolSize); + } + + // test invalid arguments + @Test + public void testTopicManagerContextInvalidArguments() { + TopicManagerContext.Builder builder = + new TopicManagerContext.Builder().setPubSubAdminAdapterFactory(pubSubAdminAdapterFactory) + .setPubSubConsumerAdapterFactory(pubSubConsumerAdapterFactory) + .setPubSubTopicRepository(pubSubTopicRepository) + .setMetricsRepository(metricsRepository) + .setPubSubPropertiesSupplier(pubSubPropertiesSupplier) + .setPubSubOperationTimeoutMs(pubSubOperationTimeoutMs) + .setTopicDeletionStatusPollIntervalMs(topicDeletionStatusPollIntervalMs) + .setTopicMinLogCompactionLagMs(topicMinLogCompactionLagMs) + .setTopicOffsetCheckIntervalMs(topicOffsetCheckIntervalMs) + .setTopicMetadataFetcherConsumerPoolSize(topicMetadataFetcherConsumerPoolSize) + .setTopicMetadataFetcherThreadPoolSize(topicMetadataFetcherThreadPoolSize); + + // set admin adapter factory to null + builder.setPubSubAdminAdapterFactory(null); + Throwable ex = expectThrows(IllegalArgumentException.class, builder::build); + builder.setPubSubAdminAdapterFactory(pubSubAdminAdapterFactory); + assertEquals(ex.getMessage(), "pubSubAdminAdapterFactory cannot be null"); + + // set consumer adapter factory to null + builder.setPubSubConsumerAdapterFactory(null); + ex = expectThrows(IllegalArgumentException.class, builder::build); + builder.setPubSubConsumerAdapterFactory(pubSubConsumerAdapterFactory); + assertEquals(ex.getMessage(), "pubSubConsumerAdapterFactory cannot be null"); + + // set topic repository to null + builder.setPubSubTopicRepository(null); + ex = expectThrows(IllegalArgumentException.class, builder::build); + builder.setPubSubTopicRepository(pubSubTopicRepository); + assertEquals(ex.getMessage(), "pubSubTopicRepository cannot be null"); + + // set pub sub properties supplier to null + builder.setPubSubPropertiesSupplier(null); + ex = expectThrows(IllegalArgumentException.class, builder::build); + builder.setPubSubPropertiesSupplier(pubSubPropertiesSupplier); + assertEquals(ex.getMessage(), "pubSubPropertiesSupplier cannot be null"); + + // set pub sub operation timeout to -1 + builder.setPubSubOperationTimeoutMs(-1); + ex = expectThrows(IllegalArgumentException.class, builder::build); + builder.setPubSubOperationTimeoutMs(pubSubOperationTimeoutMs); + assertEquals(ex.getMessage(), "pubSubOperationTimeoutMs must be positive"); + + // set topic deletion status poll interval to -1 + builder.setTopicDeletionStatusPollIntervalMs(-1); + ex = expectThrows(IllegalArgumentException.class, builder::build); + builder.setTopicDeletionStatusPollIntervalMs(topicDeletionStatusPollIntervalMs); + assertEquals(ex.getMessage(), "topicDeletionStatusPollIntervalMs must be positive"); + + // set topic offset check interval to -1 + builder.setTopicOffsetCheckIntervalMs(-1); + ex = expectThrows(IllegalArgumentException.class, builder::build); + builder.setTopicOffsetCheckIntervalMs(topicOffsetCheckIntervalMs); + assertEquals(ex.getMessage(), "topicOffsetCheckIntervalMs must be positive"); + + // set topic metadata fetcher consumer pool size to 0 + builder.setTopicMetadataFetcherConsumerPoolSize(0); + ex = expectThrows(IllegalArgumentException.class, builder::build); + builder.setTopicMetadataFetcherConsumerPoolSize(topicMetadataFetcherConsumerPoolSize); + assertEquals(ex.getMessage(), "topicMetadataFetcherConsumerPoolSize must be positive"); + + // set topic metadata fetcher thread pool size to 0 + builder.setTopicMetadataFetcherThreadPoolSize(0); + ex = expectThrows(IllegalArgumentException.class, builder::build); + builder.setTopicMetadataFetcherThreadPoolSize(topicMetadataFetcherThreadPoolSize); + assertEquals(ex.getMessage(), "topicMetadataFetcherThreadPoolSize must be positive"); + } +} diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicManagerRepositoryTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicManagerRepositoryTest.java new file mode 100644 index 0000000000..f424c6ae75 --- /dev/null +++ b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicManagerRepositoryTest.java @@ -0,0 +1,63 @@ +package com.linkedin.venice.pubsub.manager; + +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; + +import java.util.concurrent.CompletableFuture; +import org.testng.annotations.Test; + + +/** + * Unit tests for {@link TopicManagerRepository}. + */ +public class TopicManagerRepositoryTest { + @Test + public void testGetTopicManagerIsCreatedIfNotExists() { + // Use spy to return dummy TopicManager form createTopicManager + TopicManagerContext topicManagerContext = mock(TopicManagerContext.class); + String localPubSubAddress = "local.example.com:9092"; + String remotePubSubAddress = "remote.example.com:9092"; + + TopicManager localTopicManager = mock(TopicManager.class); + TopicManager remoteTopicManager = mock(TopicManager.class); + + TopicManagerRepository topicManagerRepositorySpy = + spy(new TopicManagerRepository(topicManagerContext, localPubSubAddress)); + + doReturn(localTopicManager).when(topicManagerRepositorySpy).createTopicManager(localPubSubAddress); + doReturn(remoteTopicManager).when(topicManagerRepositorySpy).createTopicManager(remotePubSubAddress); + + // get all topic managers should return empty list + assertEquals(topicManagerRepositorySpy.getAllTopicManagers().size(), 0); + + // let's call getTopicManager several times for each address + for (int i = 0; i < 5; i++) { + assertEquals(topicManagerRepositorySpy.getLocalTopicManager(), localTopicManager); + assertEquals(topicManagerRepositorySpy.getTopicManager(localPubSubAddress), localTopicManager); + assertEquals(topicManagerRepositorySpy.getTopicManager(remotePubSubAddress), remoteTopicManager); + } + + // verify that createTopicManager was called only once for each address + verify(topicManagerRepositorySpy).createTopicManager(localPubSubAddress); + verify(topicManagerRepositorySpy).createTopicManager(remotePubSubAddress); + verify(topicManagerRepositorySpy, never()).createTopicManager("some.other.address:9092"); + + // getAllTopicManagers should return both topic managers + assertEquals(topicManagerRepositorySpy.getAllTopicManagers().size(), 2); + assertTrue(topicManagerRepositorySpy.getAllTopicManagers().contains(localTopicManager)); + assertTrue(topicManagerRepositorySpy.getAllTopicManagers().contains(remoteTopicManager)); + + // test close + when(localTopicManager.closeAsync()).thenReturn(CompletableFuture.completedFuture(null)); + when(remoteTopicManager.closeAsync()).thenReturn(CompletableFuture.completedFuture(null)); + topicManagerRepositorySpy.close(); + verify(localTopicManager).closeAsync(); + verify(remoteTopicManager).closeAsync(); + } +} diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicManagerStatsTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicManagerStatsTest.java new file mode 100644 index 0000000000..4d22369102 --- /dev/null +++ b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicManagerStatsTest.java @@ -0,0 +1,50 @@ +package com.linkedin.venice.pubsub.manager; + +import static com.linkedin.venice.pubsub.manager.TopicManagerStats.SENSOR_TYPE.CREATE_TOPIC; +import static com.linkedin.venice.pubsub.manager.TopicManagerStats.SENSOR_TYPE.DELETE_TOPIC; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; + +import io.tehuti.Metric; +import io.tehuti.metrics.MetricsRepository; +import io.tehuti.metrics.Sensor; +import java.util.EnumMap; +import java.util.Map; +import org.testng.annotations.Test; + + +public class TopicManagerStatsTest { + @Test + public void testRecordLatencyDoesNotThrowExceptionWhenMetricsAreDisabled() { + TopicManagerStats stats = new TopicManagerStats(null, "localhost:12345"); + assertNull(stats.getSensorsByTypes()); + stats.recordLatency(CREATE_TOPIC, 100); + stats.recordLatency(null, 100); + } + + @Test + public void testRecordLatency() { + MetricsRepository metricsRepository = new MetricsRepository(); + String pubSubClusterAddress = "venice.kafka.dc-1.linkedin.com:12345"; + TopicManagerStats stats = new TopicManagerStats(metricsRepository, pubSubClusterAddress); + EnumMap sensorsByTypes = stats.getSensorsByTypes(); + assertNotNull(sensorsByTypes); + assertEquals(sensorsByTypes.size(), TopicManagerStats.SENSOR_TYPE.values().length); + assertEquals(stats.getMetricsRepository(), metricsRepository); + assertEquals(stats.getName(), ".TopicManagerStats_venice_kafka_dc-1_linkedin_com_12345"); + + stats.recordLatency(DELETE_TOPIC, 100); + Map metrics = metricsRepository.metrics(); + assertTrue( + metrics.get(".TopicManagerStats_venice_kafka_dc-1_linkedin_com_12345--delete_topic.OccurrenceRate") + .value() > 0); + + stats.recordLatency(CREATE_TOPIC, 200); + assertTrue( + metrics.get(".TopicManagerStats_venice_kafka_dc-1_linkedin_com_12345--create_topic.OccurrenceRate") + .value() > 0); + assertTrue(metrics.get(".TopicManagerStats_venice_kafka_dc-1_linkedin_com_12345--create_topic.Avg").value() > 0); + } +} diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/kafka/TopicManagerTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicManagerTest.java similarity index 90% rename from internal/venice-common/src/test/java/com/linkedin/venice/kafka/TopicManagerTest.java rename to internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicManagerTest.java index d961257378..c2926e12ab 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/kafka/TopicManagerTest.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicManagerTest.java @@ -1,11 +1,9 @@ -package com.linkedin.venice.kafka; +package com.linkedin.venice.pubsub.manager; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_OPERATION_TIMEOUT_MS; -import static com.linkedin.venice.kafka.TopicManager.MAX_TOPIC_DELETE_RETRIES; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_TOPIC_DELETE_RETRY_TIMES; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.any; -import static org.mockito.Mockito.anyInt; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.eq; @@ -14,7 +12,6 @@ import com.github.benmanes.caffeine.cache.Cache; import com.linkedin.venice.exceptions.VeniceException; -import com.linkedin.venice.kafka.partitionoffset.PartitionOffsetFetcherImpl; import com.linkedin.venice.kafka.protocol.ControlMessage; import com.linkedin.venice.kafka.protocol.EndOfPush; import com.linkedin.venice.kafka.protocol.GUID; @@ -53,6 +50,7 @@ import com.linkedin.venice.unit.kafka.consumer.poll.RandomPollStrategy; import com.linkedin.venice.unit.kafka.producer.MockInMemoryProducerAdapter; import com.linkedin.venice.utils.AvroRecordUtils; +import com.linkedin.venice.utils.StoreUtils; import com.linkedin.venice.utils.TestUtils; import com.linkedin.venice.utils.Time; import com.linkedin.venice.utils.Utils; @@ -109,17 +107,16 @@ protected void createTopicManager() { PubSubConsumerAdapterFactory pubSubConsumerAdapterFactory = mock(PubSubConsumerAdapterFactory.class); doReturn(mockInMemoryConsumer).when(pubSubConsumerAdapterFactory).create(any(), anyBoolean(), any(), anyString()); - topicManager = TopicManagerRepository.builder() - .setPubSubProperties(k -> VeniceProperties.empty()) - .setPubSubTopicRepository(pubSubTopicRepository) - .setLocalKafkaBootstrapServers("localhost:1234") - .setPubSubConsumerAdapterFactory(pubSubConsumerAdapterFactory) - .setPubSubAdminAdapterFactory(pubSubAdminAdapterFactory) - .setKafkaOperationTimeoutMs(500L) - .setTopicDeletionStatusPollIntervalMs(100L) - .setTopicMinLogCompactionLagMs(MIN_COMPACTION_LAG) - .build() - .getTopicManager(); + TopicManagerContext topicManagerContext = + new TopicManagerContext.Builder().setPubSubPropertiesSupplier(k -> VeniceProperties.empty()) + .setPubSubTopicRepository(pubSubTopicRepository) + .setPubSubConsumerAdapterFactory(pubSubConsumerAdapterFactory) + .setPubSubAdminAdapterFactory(pubSubAdminAdapterFactory) + .setPubSubOperationTimeoutMs(500L) + .setTopicDeletionStatusPollIntervalMs(100L) + .setTopicMinLogCompactionLagMs(MIN_COMPACTION_LAG) + .build(); + topicManager = new TopicManagerRepository(topicManagerContext, "localhost:1234").getLocalTopicManager(); } protected PubSubProducerAdapter createPubSubProducerAdapter() { @@ -193,7 +190,7 @@ public void testGetProducerTimestampOfLastDataRecord() throws ExecutionException produceRandomPubSubMessage(topic, true, timestamp - 1000); produceRandomPubSubMessage(topic, true, timestamp); // This timestamp is expected to be retrieved - long retrievedTimestamp = topicManager.getProducerTimestampOfLastDataRecord(pubSubTopicPartition, 1); + long retrievedTimestamp = topicManager.getProducerTimestampOfLastDataMessageWithRetries(pubSubTopicPartition, 1); Assert.assertEquals(retrievedTimestamp, timestamp); } @@ -207,7 +204,7 @@ public void testGetProducerTimestampOfLastDataRecordWithControlMessage() produceRandomPubSubMessage(topic, true, timestamp); // This timestamp is expected to be retrieved produceRandomPubSubMessage(topic, false, timestamp + 1000); // produce a control message - long retrievedTimestamp = topicManager.getProducerTimestampOfLastDataRecord(pubSubTopicPartition, 1); + long retrievedTimestamp = topicManager.getProducerTimestampOfLastDataMessageWithRetries(pubSubTopicPartition, 1); Assert.assertEquals(retrievedTimestamp, timestamp); // Produce more data records to this topic partition @@ -219,15 +216,15 @@ public void testGetProducerTimestampOfLastDataRecordWithControlMessage() for (int i = 1; i <= 3; i++) { produceRandomPubSubMessage(topic, false, timestamp + i * 1000L); } - retrievedTimestamp = topicManager.getProducerTimestampOfLastDataRecord(pubSubTopicPartition, 1); + retrievedTimestamp = topicManager.getProducerTimestampOfLastDataMessageWithRetries(pubSubTopicPartition, 1); Assert.assertEquals(retrievedTimestamp, timestamp); } @Test public void testGetProducerTimestampOfLastDataRecordOnEmptyTopic() { final PubSubTopicPartition emptyTopicPartition = new PubSubTopicPartitionImpl(getTopic(), 0); - long retrievedTimestamp = topicManager.getProducerTimestampOfLastDataRecord(emptyTopicPartition, 1); - Assert.assertEquals(retrievedTimestamp, PartitionOffsetFetcherImpl.NO_PRODUCER_TIME_IN_EMPTY_TOPIC_PARTITION); + long retrievedTimestamp = topicManager.getProducerTimestampOfLastDataMessageWithRetries(emptyTopicPartition, 1); + Assert.assertEquals(retrievedTimestamp, PubSubConstants.PUBSUB_NO_PRODUCER_TIME_IN_EMPTY_TOPIC_PARTITION); } @Test @@ -244,7 +241,7 @@ public void testGetProducerTimestampOfLastDataRecordWithOnlyControlMessages() PubSubTopicPartition pubSubTopicPartition = new PubSubTopicPartitionImpl(topic, 0); Assert.assertThrows( VeniceException.class, - () -> topicManager.getProducerTimestampOfLastDataRecord(pubSubTopicPartition, 1)); + () -> topicManager.getProducerTimestampOfLastDataMessageWithRetries(pubSubTopicPartition, 1)); } @Test @@ -254,14 +251,14 @@ public void testCreateTopic() throws Exception { Assert.assertTrue(topicManager.containsTopicAndAllPartitionsAreOnline(topicNameWithEternalRetentionPolicy)); Assert.assertEquals( topicManager.getTopicRetention(topicNameWithEternalRetentionPolicy), - TopicManager.ETERNAL_TOPIC_RETENTION_POLICY_MS); + PubSubConstants.ETERNAL_TOPIC_RETENTION_POLICY_MS); PubSubTopic topicNameWithDefaultRetentionPolicy = getTopic(); topicManager.createTopic(topicNameWithDefaultRetentionPolicy, 1, 1, false); /* should be noop */ Assert.assertTrue(topicManager.containsTopicAndAllPartitionsAreOnline(topicNameWithDefaultRetentionPolicy)); Assert.assertEquals( topicManager.getTopicRetention(topicNameWithDefaultRetentionPolicy), - TopicManager.DEFAULT_TOPIC_RETENTION_POLICY_MS); + PubSubConstants.DEFAULT_TOPIC_RETENTION_POLICY_MS); } @Test @@ -283,13 +280,13 @@ public void testCreateTopicWhenTopicExists() throws Exception { Assert.assertTrue(topicManager.containsTopicAndAllPartitionsAreOnline(topicNameWithEternalRetentionPolicy)); Assert.assertEquals( topicManager.getTopicRetention(topicNameWithEternalRetentionPolicy), - TopicManager.ETERNAL_TOPIC_RETENTION_POLICY_MS); + PubSubConstants.ETERNAL_TOPIC_RETENTION_POLICY_MS); topicManager.createTopic(topicNameWithDefaultRetentionPolicy, 1, 1, false); /* should be noop */ Assert.assertTrue(topicManager.containsTopicAndAllPartitionsAreOnline(topicNameWithDefaultRetentionPolicy)); Assert.assertEquals( topicManager.getTopicRetention(topicNameWithDefaultRetentionPolicy), - TopicManager.DEFAULT_TOPIC_RETENTION_POLICY_MS); + PubSubConstants.DEFAULT_TOPIC_RETENTION_POLICY_MS); } @Test @@ -322,7 +319,7 @@ public void testDeleteTopicWithTimeout() throws ExecutionException { Assert.assertThrows( PubSubOpTimeoutException.class, () -> partiallyMockedTopicManager.ensureTopicIsDeletedAndBlockWithRetry(topicName)); - Mockito.verify(partiallyMockedTopicManager, times(MAX_TOPIC_DELETE_RETRIES)) + Mockito.verify(partiallyMockedTopicManager, times(PUBSUB_TOPIC_DELETE_RETRY_TIMES)) .ensureTopicIsDeletedAndBlock(topicName); } @@ -416,8 +413,8 @@ public void testGetAllTopicRetentions() { Assert.assertTrue( topicRetentions.size() > 3, "There should be at least 3 topics, " + "which were created by this test"); - Assert.assertEquals(topicRetentions.get(topic1).longValue(), TopicManager.ETERNAL_TOPIC_RETENTION_POLICY_MS); - Assert.assertEquals(topicRetentions.get(topic2).longValue(), TopicManager.DEFAULT_TOPIC_RETENTION_POLICY_MS); + Assert.assertEquals(topicRetentions.get(topic1).longValue(), PubSubConstants.ETERNAL_TOPIC_RETENTION_POLICY_MS); + Assert.assertEquals(topicRetentions.get(topic2).longValue(), PubSubConstants.DEFAULT_TOPIC_RETENTION_POLICY_MS); Assert.assertEquals(topicRetentions.get(topic3).longValue(), 5000); long deprecatedTopicRetentionMaxMs = 5000; @@ -436,7 +433,7 @@ public void testGetAllTopicRetentions() { .isRetentionBelowTruncatedThreshold(deprecatedTopicRetentionMaxMs + 1, deprecatedTopicRetentionMaxMs)); Assert.assertFalse( topicManager.isRetentionBelowTruncatedThreshold( - PubSubConstants.UNKNOWN_TOPIC_RETENTION, + PubSubConstants.PUBSUB_TOPIC_UNKNOWN_RETENTION, deprecatedTopicRetentionMaxMs)); Assert.assertTrue( topicManager @@ -487,7 +484,7 @@ public void testGetLatestOffsetForNonExistingTopic() { PubSubTopic nonExistingTopic = pubSubTopicRepository.getTopic(TestUtils.getUniqueTopicString("non-existing-topic")); Assert.assertThrows( PubSubTopicDoesNotExistException.class, - () -> topicManager.getPartitionLatestOffsetAndRetry(new PubSubTopicPartitionImpl(nonExistingTopic, 0), 10)); + () -> topicManager.getLatestOffsetWithRetries(new PubSubTopicPartitionImpl(nonExistingTopic, 0), 10)); } @Test @@ -495,7 +492,8 @@ public void testGetLatestProducerTimestampForNonExistingTopic() { PubSubTopic nonExistingTopic = pubSubTopicRepository.getTopic(TestUtils.getUniqueTopicString("non-existing-topic")); Assert.assertThrows( PubSubTopicDoesNotExistException.class, - () -> topicManager.getProducerTimestampOfLastDataRecord(new PubSubTopicPartitionImpl(nonExistingTopic, 0), 10)); + () -> topicManager + .getProducerTimestampOfLastDataMessageWithRetries(new PubSubTopicPartitionImpl(nonExistingTopic, 0), 10)); } @Test @@ -521,8 +519,7 @@ public void testTimeoutOnGettingMaxOffset() throws IOException { PubSubTopicPartition pubSubTopicPartition = new PubSubTopicPartitionImpl(topic, 0); // Mock an admin client to pass topic existence check PubSubAdminAdapter mockPubSubAdminAdapter = mock(PubSubAdminAdapter.class); - doReturn(true).when(mockPubSubAdminAdapter) - .containsTopicWithPartitionCheckExpectationAndRetry(eq(pubSubTopicPartition), anyInt(), eq(true)); + doReturn(true).when(mockPubSubAdminAdapter).containsTopic(eq(topic)); PubSubConsumerAdapter mockPubSubConsumer = mock(PubSubConsumerAdapter.class); doThrow(new PubSubOpTimeoutException("Timed out while fetching end offsets")).when(mockPubSubConsumer) .endOffsets(any(), any()); @@ -533,20 +530,23 @@ public void testTimeoutOnGettingMaxOffset() throws IOException { PubSubConsumerAdapterFactory consumerAdapterFactory = mock(PubSubConsumerAdapterFactory.class); doReturn(mockPubSubConsumer).when(consumerAdapterFactory).create(any(), anyBoolean(), any(), anyString()); doReturn(mockPubSubAdminAdapter).when(adminAdapterFactory).create(any(), eq(pubSubTopicRepository)); - try (TopicManager topicManagerForThisTest = TopicManagerRepository.builder() - .setPubSubProperties(k -> VeniceProperties.empty()) - .setPubSubTopicRepository(pubSubTopicRepository) - .setLocalKafkaBootstrapServers(localPubSubBrokerAddress) - .setPubSubAdminAdapterFactory(adminAdapterFactory) - .setPubSubConsumerAdapterFactory(consumerAdapterFactory) - .setKafkaOperationTimeoutMs(DEFAULT_KAFKA_OPERATION_TIMEOUT_MS) - .setTopicDeletionStatusPollIntervalMs(100) - .setTopicMinLogCompactionLagMs(MIN_COMPACTION_LAG) - .build() - .getTopicManager()) { + + TopicManagerContext topicManagerContext = + new TopicManagerContext.Builder().setPubSubPropertiesSupplier(k -> VeniceProperties.empty()) + .setPubSubTopicRepository(pubSubTopicRepository) + .setPubSubAdminAdapterFactory(adminAdapterFactory) + .setPubSubConsumerAdapterFactory(consumerAdapterFactory) + .setTopicDeletionStatusPollIntervalMs(100) + .setTopicMetadataFetcherConsumerPoolSize(1) + .setTopicMetadataFetcherThreadPoolSize(1) + .setTopicMinLogCompactionLagMs(MIN_COMPACTION_LAG) + .build(); + + try (TopicManager topicManagerForThisTest = + new TopicManagerRepository(topicManagerContext, localPubSubBrokerAddress).getLocalTopicManager()) { Assert.assertThrows( PubSubOpTimeoutException.class, - () -> topicManagerForThisTest.getPartitionLatestOffsetAndRetry(pubSubTopicPartition, 10)); + () -> topicManagerForThisTest.getLatestOffsetWithRetries(pubSubTopicPartition, 10)); } } @@ -617,9 +617,8 @@ public void testMinimumExpectedRetentionTime() { BufferReplayPolicy.REWIND_FROM_EOP); // Since bootstrapToOnlineTimeout + rewind time + buffer (2 days) < 5 days, retention will be set to 5 days - Assert.assertEquals( - TopicManager.getExpectedRetentionTimeInMs(store, hybridStoreConfig2DayRewind), - 5 * Time.MS_PER_DAY); + Assert + .assertEquals(StoreUtils.getExpectedRetentionTimeInMs(store, hybridStoreConfig2DayRewind), 5 * Time.MS_PER_DAY); } @Test @@ -639,9 +638,8 @@ public void testExpectedRetentionTime() { // Since bootstrapToOnlineTimeout + rewind time + buffer (2 days) > 5 days, retention will be set to the computed // value - Assert.assertEquals( - TopicManager.getExpectedRetentionTimeInMs(store, hybridStoreConfig2DayRewind), - 7 * Time.MS_PER_DAY); + Assert + .assertEquals(StoreUtils.getExpectedRetentionTimeInMs(store, hybridStoreConfig2DayRewind), 7 * Time.MS_PER_DAY); } @Test diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcherTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcherTest.java new file mode 100644 index 0000000000..8299370e99 --- /dev/null +++ b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcherTest.java @@ -0,0 +1,494 @@ +package com.linkedin.venice.pubsub.manager; + +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_CONSUMER_POLLING_FOR_METADATA_RETRY_MAX_ATTEMPT; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_NO_PRODUCER_TIME_IN_EMPTY_TOPIC_PARTITION; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertThrows; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.expectThrows; +import static org.testng.Assert.fail; + +import com.linkedin.venice.exceptions.VeniceException; +import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; +import com.linkedin.venice.kafka.protocol.ProducerMetadata; +import com.linkedin.venice.message.KafkaKey; +import com.linkedin.venice.pubsub.ImmutablePubSubMessage; +import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl; +import com.linkedin.venice.pubsub.PubSubTopicPartitionInfo; +import com.linkedin.venice.pubsub.PubSubTopicRepository; +import com.linkedin.venice.pubsub.api.PubSubAdminAdapter; +import com.linkedin.venice.pubsub.api.PubSubConsumerAdapter; +import com.linkedin.venice.pubsub.api.PubSubMessage; +import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.api.PubSubTopicPartition; +import com.linkedin.venice.pubsub.api.exceptions.PubSubClientException; +import com.linkedin.venice.pubsub.api.exceptions.PubSubOpTimeoutException; +import com.linkedin.venice.pubsub.api.exceptions.PubSubTopicDoesNotExistException; +import com.linkedin.venice.pubsub.manager.TopicMetadataFetcher.ValueAndExpiryTime; +import com.linkedin.venice.utils.ExceptionUtils; +import com.linkedin.venice.utils.Time; +import it.unimi.dsi.fastutil.ints.Int2LongMap; +import it.unimi.dsi.fastutil.ints.Int2LongMaps; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + + +public class TopicMetadataFetcherTest { + private TopicMetadataFetcher topicMetadataFetcher; + private String pubSubClusterAddress = "venicedb.pubsub.standalone:9092"; + private PubSubAdminAdapter adminMock; + private BlockingQueue pubSubConsumerPool; + private ThreadPoolExecutor threadPoolExecutor; + private long cachedEntryTtlInNs = TimeUnit.MINUTES.toNanos(5); + private PubSubTopicRepository pubSubTopicRepository = new PubSubTopicRepository(); + private String topicName = "testTopicName"; + private PubSubTopic pubSubTopic; + private PubSubConsumerAdapter consumerMock; + private TopicManagerStats stats; + + @BeforeMethod(alwaysRun = true) + public void setUp() throws InterruptedException { + consumerMock = mock(PubSubConsumerAdapter.class); + pubSubConsumerPool = new LinkedBlockingQueue<>(2); + pubSubConsumerPool.put(consumerMock); + threadPoolExecutor = (ThreadPoolExecutor) Executors.newFixedThreadPool(2); + adminMock = mock(PubSubAdminAdapter.class); + pubSubTopic = pubSubTopicRepository.getTopic(topicName); + stats = mock(TopicManagerStats.class); + topicMetadataFetcher = new TopicMetadataFetcher( + pubSubClusterAddress, + stats, + adminMock, + pubSubConsumerPool, + threadPoolExecutor, + cachedEntryTtlInNs); + assertEquals(pubSubConsumerPool.size(), 1); + } + + @Test + public void testClose() throws InterruptedException { + CountDownLatch signalReceiver = new CountDownLatch(1); + List partitions = new ArrayList<>(); + CountDownLatch holdConsumer = new CountDownLatch(1); + doAnswer(invocation -> { + try { + signalReceiver.countDown(); + return partitions; + } catch (Exception e) { + throw e; + } finally { + holdConsumer.await(); + } + }).when(consumerMock).partitionsFor(pubSubTopic); + + CompletableFuture> future = CompletableFuture + .supplyAsync(() -> topicMetadataFetcher.getTopicPartitionInfo(pubSubTopic), threadPoolExecutor); + + if (!signalReceiver.await(3, TimeUnit.MINUTES)) { + fail("Timed out waiting for signalReceiver"); + } + + try { + topicMetadataFetcher.close(); + } catch (Exception e) { + fail("TopicMetadataFetcher::close should not throw exception when closing"); + } + Throwable t = expectThrows(ExecutionException.class, future::get); + assertTrue(ExceptionUtils.recursiveClassEquals(t, InterruptedException.class)); + verify(consumerMock, times(1)).partitionsFor(pubSubTopic); + verify(consumerMock, times(1)).close(); + } + + @Test + public void testValidateTopicPartition() { + assertThrows(NullPointerException.class, () -> topicMetadataFetcher.validateTopicPartition(null)); + + final PubSubTopicPartition tp1 = new PubSubTopicPartitionImpl(pubSubTopicRepository.getTopic(topicName), -1); + assertThrows(IllegalArgumentException.class, () -> topicMetadataFetcher.validateTopicPartition(tp1)); + + final PubSubTopicPartition tp2 = new PubSubTopicPartitionImpl(pubSubTopicRepository.getTopic(topicName), 0); + TopicMetadataFetcher topicMetadataFetcherSpy = spy(topicMetadataFetcher); + doReturn(false).when(topicMetadataFetcherSpy).containsTopicCached(tp2.getPubSubTopic()); + Exception e = + expectThrows(PubSubTopicDoesNotExistException.class, () -> topicMetadataFetcherSpy.validateTopicPartition(tp2)); + assertTrue(e.getMessage().contains("does not exist")); + + doReturn(true).when(topicMetadataFetcherSpy).containsTopicCached(tp2.getPubSubTopic()); + topicMetadataFetcherSpy.validateTopicPartition(tp2); + + verify(topicMetadataFetcherSpy, times(2)).containsTopicCached(tp2.getPubSubTopic()); + } + + @Test + public void testContainsTopic() { + when(adminMock.containsTopic(pubSubTopic)).thenReturn(false); + assertFalse(topicMetadataFetcher.containsTopic(pubSubTopic)); + + when(adminMock.containsTopic(pubSubTopic)).thenReturn(true); + assertTrue(topicMetadataFetcher.containsTopic(pubSubTopic)); + + verify(adminMock, times(2)).containsTopic(pubSubTopic); + } + + @Test + public void testContainsTopicAsync() { + when(adminMock.containsTopic(pubSubTopic)).thenReturn(false); + assertFalse(topicMetadataFetcher.containsTopicAsync(pubSubTopic).join()); + + when(adminMock.containsTopic(pubSubTopic)).thenReturn(true); + assertTrue(topicMetadataFetcher.containsTopicAsync(pubSubTopic).join()); + + doThrow(new PubSubClientException("Test")).when(adminMock).containsTopic(pubSubTopic); + CompletableFuture future = topicMetadataFetcher.containsTopicAsync(pubSubTopic); + ExecutionException e = expectThrows(ExecutionException.class, future::get); + assertTrue(ExceptionUtils.recursiveClassEquals(e, PubSubClientException.class)); + + verify(adminMock, times(3)).containsTopic(pubSubTopic); + } + + @Test + public void testUpdateCacheAsyncWhenCachedValueIsNotStaleOrWhenUpdateIsInProgressShouldNotUpdateCache() { + // Set the cached value to be not stale and update not in progress + Supplier> cfSupplierMock = mock(Supplier.class); + Map> cache = new ConcurrentHashMap<>(); + ValueAndExpiryTime cachedValue = topicMetadataFetcher.new ValueAndExpiryTime<>(true); + cache.put(pubSubTopic, cachedValue); + topicMetadataFetcher.updateCacheAsync(pubSubTopic, cachedValue, cache, cfSupplierMock); + verify(cfSupplierMock, never()).get(); + + // Set the cached value to be stale and update in progress + cachedValue.setExpiryTimeNs(System.nanoTime() - 1); + cachedValue.setUpdateInProgressStatus(true); + topicMetadataFetcher.updateCacheAsync(pubSubTopic, cachedValue, cache, cfSupplierMock); + verify(cfSupplierMock, never()).get(); + } + + @Test + public void testUpdateCacheAsync() { + // WhenCachedValueIsNull --> ShouldUpdateCache + Supplier> cfSupplierMock = mock(Supplier.class); + when(cfSupplierMock.get()).thenReturn(CompletableFuture.completedFuture(true)); + Map> cache = new ConcurrentHashMap<>(); + topicMetadataFetcher.updateCacheAsync(pubSubTopic, null, cache, cfSupplierMock); + assertEquals(cache.size(), 1); + assertTrue(cache.containsKey(pubSubTopic)); + // if we can acquire the lock, it means it was released after the update + assertTrue(cache.get(pubSubTopic).tryAcquireUpdateLock()); + verify(cfSupplierMock, times(1)).get(); + + // WhenCachedValueIsStaleAndWhenAsyncUpdateSucceeds --> ShouldUpdateCache + ValueAndExpiryTime cachedValue = topicMetadataFetcher.new ValueAndExpiryTime<>(true); + cachedValue.setExpiryTimeNs(System.nanoTime() - 1); + topicMetadataFetcher.updateCacheAsync(pubSubTopic, cachedValue, cache, cfSupplierMock); + assertEquals(cache.size(), 1); + assertTrue(cache.containsKey(pubSubTopic)); + // if we can acquire the lock, it means it was released after the update + assertTrue(cache.get(pubSubTopic).tryAcquireUpdateLock()); + assertTrue(cache.get(pubSubTopic).getValue()); + verify(cfSupplierMock, times(2)).get(); + + // WhenAsyncUpdateFails --> ShouldRemoveFromCache + cache.remove(pubSubTopic); + CompletableFuture failedFuture = new CompletableFuture<>(); + failedFuture.completeExceptionally(new PubSubClientException("Test")); + when(cfSupplierMock.get()).thenReturn(failedFuture); + topicMetadataFetcher.updateCacheAsync(pubSubTopic, null, cache, cfSupplierMock); + assertEquals(cache.size(), 0); + assertFalse(cache.containsKey(pubSubTopic)); + verify(cfSupplierMock, times(3)).get(); + } + + @Test + public void testGetTopicLatestOffsets() { + // test consumer::partitionFor --> (null, empty list) + when(consumerMock.partitionsFor(pubSubTopic)).thenReturn(null).thenReturn(Collections.emptyList()); + for (int i = 0; i < 2; i++) { + verify(consumerMock, times(i)).partitionsFor(pubSubTopic); + Int2LongMap res = topicMetadataFetcher.getTopicLatestOffsets(pubSubTopic); + assertEquals(res, Int2LongMaps.EMPTY_MAP); + assertEquals(res.size(), 0); + verify(consumerMock, times(i + 1)).partitionsFor(pubSubTopic); + } + + // test consumer::partitionFor returns non-empty list + PubSubTopicPartitionInfo tp0Info = new PubSubTopicPartitionInfo(pubSubTopic, 0, true); + PubSubTopicPartitionInfo tp1Info = new PubSubTopicPartitionInfo(pubSubTopic, 1, true); + List partitionInfo = Arrays.asList(tp0Info, tp1Info); + Map offsetsMap = new ConcurrentHashMap<>(); + offsetsMap.put(tp0Info.getTopicPartition(), 111L); + offsetsMap.put(tp1Info.getTopicPartition(), 222L); + + when(consumerMock.partitionsFor(pubSubTopic)).thenReturn(partitionInfo); + when(consumerMock.endOffsets(eq(offsetsMap.keySet()), any(Duration.class))).thenReturn(offsetsMap); + + Int2LongMap res = topicMetadataFetcher.getTopicLatestOffsets(pubSubTopic); + assertEquals(res.size(), offsetsMap.size()); + assertEquals(res.get(0), 111L); + assertEquals(res.get(1), 222L); + + verify(consumerMock, times(3)).partitionsFor(pubSubTopic); + verify(consumerMock, times(1)).endOffsets(eq(offsetsMap.keySet()), any(Duration.class)); + + // check if consumer was released back to the pool + assertEquals(pubSubConsumerPool.size(), 1); + } + + @Test + public void testGetTopicPartitionInfo() { + PubSubTopicPartitionInfo tp0Info = new PubSubTopicPartitionInfo(pubSubTopic, 0, true); + PubSubTopicPartitionInfo tp1Info = new PubSubTopicPartitionInfo(pubSubTopic, 1, true); + List partitionInfo = Arrays.asList(tp0Info, tp1Info); + when(consumerMock.partitionsFor(pubSubTopic)).thenReturn(partitionInfo); + List res = topicMetadataFetcher.getTopicPartitionInfo(pubSubTopic); + assertEquals(res.size(), partitionInfo.size()); + assertEquals(res.get(0), tp0Info); + assertEquals(res.get(1), tp1Info); + assertEquals(pubSubConsumerPool.size(), 1); + } + + @Test + public void testConsumeLatestRecords() { + // test invalid input + PubSubTopicPartition invalidTp = new PubSubTopicPartitionImpl(pubSubTopic, -1); + Throwable t = + expectThrows(IllegalArgumentException.class, () -> topicMetadataFetcher.consumeLatestRecords(invalidTp, 1)); + assertTrue(t.getMessage().contains("Invalid partition number")); + + // test invalid last record count + PubSubTopicPartition topicPartition = new PubSubTopicPartitionImpl(pubSubTopic, 0); + t = expectThrows( + IllegalArgumentException.class, + () -> topicMetadataFetcher.consumeLatestRecords(topicPartition, 0)); + assertTrue(t.getMessage().contains("Last record count must be greater than or equal to 1.")); + + when(adminMock.containsTopic(pubSubTopic)).thenReturn(true); + + // test when endOffsets returns null + when(consumerMock.endOffsets(eq(Collections.singleton(topicPartition)), any(Duration.class))).thenReturn(null); + t = expectThrows(VeniceException.class, () -> topicMetadataFetcher.consumeLatestRecords(topicPartition, 1)); + assertTrue(t.getMessage().contains("Failed to get the end offset for topic-partition:")); + + // test when endOffsets returns empty map + when(consumerMock.endOffsets(eq(Collections.singleton(topicPartition)), any(Duration.class))) + .thenReturn(Collections.emptyMap()); + t = expectThrows(VeniceException.class, () -> topicMetadataFetcher.consumeLatestRecords(topicPartition, 1)); + assertTrue(t.getMessage().contains("Failed to get the end offset for topic-partition:")); + + // test when there are no records to consume as endOffset is 0 + Map endOffsetsMap = new HashMap<>(); + endOffsetsMap.put(topicPartition, 0L); + when(consumerMock.endOffsets(eq(Collections.singletonList(topicPartition)), any(Duration.class))) + .thenReturn(endOffsetsMap); + List> consumedRecords = + topicMetadataFetcher.consumeLatestRecords(topicPartition, 1); + assertEquals(consumedRecords.size(), 0); + + // test when beginningOffset (non-zero) is same as endOffset + endOffsetsMap.put(topicPartition, 1L); + when(consumerMock.endOffsets(eq(Collections.singletonList(topicPartition)), any(Duration.class))) + .thenReturn(endOffsetsMap); + when(consumerMock.beginningOffset(eq(topicPartition), any(Duration.class))).thenReturn(1L); + consumedRecords = topicMetadataFetcher.consumeLatestRecords(topicPartition, 1); + assertEquals(consumedRecords.size(), 0); + + long endOffset = 10; + long startOffset = 3; + int numRecordsToRead = 5; // read records at offsets 5, 6, 7, 8, 9 + long consumePastOffset = 4; // subscribe at offset + endOffsetsMap.put(topicPartition, endOffset); + when(consumerMock.endOffsets(eq(Collections.singletonList(topicPartition)), any(Duration.class))) + .thenReturn(endOffsetsMap); + when(consumerMock.beginningOffset(eq(topicPartition), any(Duration.class))).thenReturn(startOffset); + verify(consumerMock, never()).subscribe(topicPartition, consumePastOffset); + verify(consumerMock, never()).poll(anyLong()); + verify(consumerMock, never()).unSubscribe(eq(topicPartition)); + + // test when poll returns no records + when(consumerMock.poll(anyLong())).thenReturn(Collections.emptyMap()); + t = expectThrows( + VeniceException.class, + () -> topicMetadataFetcher.consumeLatestRecords(topicPartition, numRecordsToRead)); + assertNotNull(t.getMessage()); + assertTrue(t.getMessage().contains("Failed to get records from topic-partition:")); + verify(consumerMock, times(1)).subscribe(topicPartition, consumePastOffset); + verify(consumerMock, times(PUBSUB_CONSUMER_POLLING_FOR_METADATA_RETRY_MAX_ATTEMPT)).poll(anyLong()); + verify(consumerMock, times(1)).unSubscribe(eq(topicPartition)); + + // poll returns 1, then 2 and then 4 records (to simulate a condition where records get added after getEndOffsets + // API call) + Map>> batch1 = new HashMap<>(); + batch1.put(topicPartition, Collections.singletonList(getPubSubMessage(topicPartition, true, 5))); + Map>> batch2 = new HashMap<>(); + batch2.put( + topicPartition, + Arrays.asList(getPubSubMessage(topicPartition, true, 6), getPubSubMessage(topicPartition, false, 7))); + Map>> batch3 = new HashMap<>(); + batch3.put( + topicPartition, + Arrays.asList( + getPubSubMessage(topicPartition, false, 8), + getPubSubMessage(topicPartition, true, 9), + getPubSubMessage(topicPartition, true, 10), + getPubSubMessage(topicPartition, false, 11))); + when(consumerMock.poll(anyLong())).thenReturn(batch1).thenReturn(batch2).thenReturn(batch3); + + List> allConsumedRecords = + topicMetadataFetcher.consumeLatestRecords(topicPartition, numRecordsToRead); + assertTrue(allConsumedRecords.size() >= numRecordsToRead); + long firstOffset = allConsumedRecords.get(0).getOffset(); + assertEquals(firstOffset, 5); + long lastOffset = allConsumedRecords.get(allConsumedRecords.size() - 1).getOffset(); + assertEquals(lastOffset, 11); + verify(consumerMock, times(2)).subscribe(topicPartition, consumePastOffset); + verify(consumerMock, times(6)).poll(anyLong()); // 3 from prev test and 3 from this test + verify(consumerMock, times(2)).unSubscribe(eq(topicPartition)); + } + + private PubSubMessage getPubSubMessage(PubSubTopicPartition topicPartition, boolean isControlMessage, long offset) { + KafkaKey key = mock(KafkaKey.class); + when(key.isControlMessage()).thenReturn(isControlMessage); + KafkaMessageEnvelope val = mock(KafkaMessageEnvelope.class); + ProducerMetadata producerMetadata = new ProducerMetadata(); + producerMetadata.setMessageTimestamp(System.nanoTime()); + when(val.getProducerMetadata()).thenReturn(producerMetadata); + return new ImmutablePubSubMessage(key, val, topicPartition, offset, System.currentTimeMillis(), 512); + } + + @Test + public void testGetProducerTimestampOfLastDataMessage() { + // test when there are no records to consume + TopicMetadataFetcher metadataFetcherSpy = spy(topicMetadataFetcher); + PubSubTopicPartition topicPartition = new PubSubTopicPartitionImpl(pubSubTopic, 0); + doReturn(Collections.emptyList()).when(metadataFetcherSpy).consumeLatestRecords(eq(topicPartition), anyInt()); + long timestamp = metadataFetcherSpy.getProducerTimestampOfLastDataMessage(topicPartition); + assertEquals(timestamp, PUBSUB_NO_PRODUCER_TIME_IN_EMPTY_TOPIC_PARTITION); + verify(metadataFetcherSpy, times(1)).consumeLatestRecords(eq(topicPartition), anyInt()); + + // test when there are no data messages to consume + PubSubMessage cm = getPubSubMessage(topicPartition, true, 5); + doReturn(Collections.singletonList(cm)).when(metadataFetcherSpy).consumeLatestRecords(eq(topicPartition), anyInt()); + Throwable t = expectThrows( + VeniceException.class, + () -> metadataFetcherSpy.getProducerTimestampOfLastDataMessage(topicPartition)); + assertTrue(t.getMessage().contains("No data message found in topic-partition")); + + // test when there are data messages to consume + PubSubMessage dm0 = getPubSubMessage(topicPartition, false, 4); + doReturn(Collections.singletonList(dm0)).when(metadataFetcherSpy) + .consumeLatestRecords(eq(topicPartition), anyInt()); + timestamp = metadataFetcherSpy.getProducerTimestampOfLastDataMessage(topicPartition); + assertEquals(timestamp, dm0.getValue().getProducerMetadata().getMessageTimestamp()); + + // test: first return one control message and then one data message + doReturn(Collections.singletonList(cm)).doReturn(Collections.singletonList(dm0)) + .when(metadataFetcherSpy) + .consumeLatestRecords(eq(topicPartition), anyInt()); + timestamp = metadataFetcherSpy.getProducerTimestampOfLastDataMessage(topicPartition); + assertEquals(timestamp, dm0.getValue().getProducerMetadata().getMessageTimestamp()); + + // test: return 2 data messages + PubSubMessage dm1 = getPubSubMessage(topicPartition, false, 3); + doReturn(Arrays.asList(dm1, dm0)).when(metadataFetcherSpy).consumeLatestRecords(eq(topicPartition), anyInt()); + timestamp = metadataFetcherSpy.getProducerTimestampOfLastDataMessage(topicPartition); + assertEquals(timestamp, dm0.getValue().getProducerMetadata().getMessageTimestamp()); + } + + @Test + public void testGetLatestOffset() { + PubSubTopicPartition tp0 = new PubSubTopicPartitionImpl(pubSubTopic, 0); + Map offsetMap = new HashMap<>(); + offsetMap.put(tp0, 1001L); + when(consumerMock.endOffsets(eq(offsetMap.keySet()), any(Duration.class))).thenReturn(offsetMap); + long latestOffset = topicMetadataFetcher.getLatestOffset(tp0); + assertEquals(latestOffset, 1001L); + + // test when endOffsets returns null + when(consumerMock.endOffsets(eq(offsetMap.keySet()), any(Duration.class))).thenReturn(Collections.emptyMap()); + Throwable t = expectThrows(VeniceException.class, () -> topicMetadataFetcher.getLatestOffset(tp0)); + assertTrue(t.getMessage().contains("Got null as latest offset for")); + assertEquals(pubSubConsumerPool.size(), 1); + } + + @Test(timeOut = 60 * Time.MS_PER_SECOND) + public void testGetLatestOffsetWithRetries() throws ExecutionException, InterruptedException { + PubSubTopicPartition tp0 = new PubSubTopicPartitionImpl(pubSubTopic, 0); + when(adminMock.containsTopic(pubSubTopic)).thenReturn(true); + + TopicMetadataFetcher topicMetadataFetcherSpy = spy(topicMetadataFetcher); + doThrow(new PubSubTopicDoesNotExistException("Test1")).doThrow(new PubSubOpTimeoutException("Test2")) + .doThrow(new PubSubOpTimeoutException("Test3")) + .doReturn(99L) + .when(topicMetadataFetcherSpy) + .getLatestOffset(tp0); + assertEquals((long) topicMetadataFetcherSpy.getLatestOffsetWithRetriesAsync(tp0, 5).get(), 99L); + verify(topicMetadataFetcherSpy, times(4)).getLatestOffset(tp0); + + doThrow(new PubSubTopicDoesNotExistException("Test1")).when(topicMetadataFetcherSpy).getLatestOffset(tp0); + expectThrows( + PubSubTopicDoesNotExistException.class, + () -> topicMetadataFetcherSpy.getLatestOffsetWithRetries(tp0, 1)); + + Map offsetMap = new HashMap<>(); + offsetMap.put(tp0, 1001L); + when(consumerMock.endOffsets(eq(offsetMap.keySet()), any(Duration.class))).thenReturn(offsetMap); + long latestOffset = topicMetadataFetcher.getLatestOffsetWithRetries(tp0, 1); + assertEquals(latestOffset, 1001L); + + // test when endOffsets returns null + when(consumerMock.endOffsets(eq(offsetMap.keySet()), any(Duration.class))).thenReturn(Collections.emptyMap()); + Throwable t = expectThrows(VeniceException.class, () -> topicMetadataFetcher.getLatestOffsetWithRetries(tp0, 1)); + assertTrue(t.getMessage().contains("Got null as latest offset for")); + assertEquals(pubSubConsumerPool.size(), 1); + } + + @Test + public void testGetOffsetForTime() { + PubSubTopicPartition tp0 = new PubSubTopicPartitionImpl(pubSubTopic, 0); + when(adminMock.containsTopic(pubSubTopic)).thenReturn(true); + long latestOffset = 12321L; + TopicMetadataFetcher topicMetadataFetcherSpy = spy(topicMetadataFetcher); + doReturn(latestOffset).when(topicMetadataFetcherSpy).getLatestOffset(tp0); + + long ts = System.currentTimeMillis(); + when(consumerMock.offsetForTime(eq(tp0), eq(ts), any(Duration.class))).thenReturn(9988L); + assertEquals(topicMetadataFetcherSpy.getOffsetForTime(tp0, ts), 9988L); + assertEquals(pubSubConsumerPool.size(), 1); + + // test when offsetForTime returns null + when(consumerMock.offsetForTime(eq(tp0), eq(ts), any(Duration.class))).thenReturn(null); + assertEquals(topicMetadataFetcherSpy.getOffsetForTime(tp0, ts), latestOffset); + assertEquals(pubSubConsumerPool.size(), 1); + } +} diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/stats/TestPubSubAdminWrapperStats.java b/internal/venice-common/src/test/java/com/linkedin/venice/stats/TestPubSubAdminWrapperStats.java deleted file mode 100644 index 9727ac84f2..0000000000 --- a/internal/venice-common/src/test/java/com/linkedin/venice/stats/TestPubSubAdminWrapperStats.java +++ /dev/null @@ -1,34 +0,0 @@ -package com.linkedin.venice.stats; - -import io.tehuti.metrics.MetricsRepository; -import org.testng.Assert; -import org.testng.annotations.Test; - - -public class TestPubSubAdminWrapperStats { - @Test - public void testSingleton() { - final MetricsRepository repo1 = new MetricsRepository(); - final MetricsRepository repo2 = new MetricsRepository(); - final String prefix1 = "prefix1"; - final String prefix2 = "prefix2"; - - PubSubAdminWrapperStats statsForRepo1AndPrefix1 = PubSubAdminWrapperStats.getInstance(repo1, prefix1); - Assert.assertEquals( - PubSubAdminWrapperStats.getInstance(repo1, prefix1), - statsForRepo1AndPrefix1, - "PubSubAdminWrapperStats.getInstance should return the same instance with the same MetricsRepository and stat prefix params"); - Assert.assertNotEquals( - PubSubAdminWrapperStats.getInstance(repo1, prefix2), - statsForRepo1AndPrefix1, - "PubSubAdminWrapperStats.getInstance should return a different instance with the same MetricsRepository and different prefix params"); - Assert.assertNotEquals( - PubSubAdminWrapperStats.getInstance(repo2, prefix1), - statsForRepo1AndPrefix1, - "PubSubAdminWrapperStats.getInstance should return a different instance with the different MetricsRepository and same prefix params"); - Assert.assertNotEquals( - PubSubAdminWrapperStats.getInstance(repo2, prefix2), - statsForRepo1AndPrefix1, - "PubSubAdminWrapperStats.getInstance should return a different instance with the different MetricsRepository and different prefix params"); - } -} diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/system/store/MetaStoreWriterTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/system/store/MetaStoreWriterTest.java index 1093d8b839..cb402b3cc1 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/system/store/MetaStoreWriterTest.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/system/store/MetaStoreWriterTest.java @@ -14,8 +14,8 @@ import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.helix.HelixReadOnlyZKSharedSchemaRepository; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.pubsub.PubSubTopicRepository; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.systemstore.schemas.StoreMetaKey; import com.linkedin.venice.systemstore.schemas.StoreMetaValue; import com.linkedin.venice.utils.VeniceResourceCloseResult; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestAdminToolEndToEnd.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestAdminToolEndToEnd.java index df8f5554f9..a69bfeb32d 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestAdminToolEndToEnd.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestAdminToolEndToEnd.java @@ -19,10 +19,10 @@ import com.linkedin.venice.integration.utils.ServiceFactory; import com.linkedin.venice.integration.utils.VeniceClusterWrapper; import com.linkedin.venice.integration.utils.VeniceServerWrapper; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.meta.Version; import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.utils.TestUtils; import com.linkedin.venice.utils.Time; import com.linkedin.venice.utils.Utils; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestClusterLevelConfigForActiveActiveReplication.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestClusterLevelConfigForActiveActiveReplication.java index 7b75aafec6..362e47aa2d 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestClusterLevelConfigForActiveActiveReplication.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestClusterLevelConfigForActiveActiveReplication.java @@ -16,10 +16,10 @@ import com.linkedin.venice.common.VeniceSystemStoreUtils; import com.linkedin.venice.controllerapi.UpdateStoreQueryParams; import com.linkedin.venice.integration.utils.D2TestUtils; -import com.linkedin.venice.kafka.TopicManager; -import com.linkedin.venice.kafka.TopicManagerRepository; import com.linkedin.venice.meta.Store; import com.linkedin.venice.meta.Version; +import com.linkedin.venice.pubsub.manager.TopicManager; +import com.linkedin.venice.pubsub.manager.TopicManagerRepository; import com.linkedin.venice.utils.TestUtils; import com.linkedin.venice.utils.Time; import com.linkedin.venice.utils.Utils; @@ -229,7 +229,7 @@ private TopicManagerRepository prepareCluster( TopicManager mockedTopicManager = mock(TopicManager.class); TopicManagerRepository mockedTopicManageRepository = mock(TopicManagerRepository.class); - doReturn(mockedTopicManager).when(mockedTopicManageRepository).getTopicManager(); + doReturn(mockedTopicManager).when(mockedTopicManageRepository).getLocalTopicManager(); doReturn(mockedTopicManager).when(mockedTopicManageRepository).getTopicManager(any(String.class)); doReturn(mockedTopicManager).when(mockedTopicManageRepository).getTopicManager(anyString()); veniceAdmin.setTopicManagerRepository(mockedTopicManageRepository); diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestClusterLevelConfigForNativeReplication.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestClusterLevelConfigForNativeReplication.java index 9d1df5d02e..aa64d0a333 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestClusterLevelConfigForNativeReplication.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestClusterLevelConfigForNativeReplication.java @@ -10,9 +10,9 @@ import static org.mockito.Mockito.mock; import com.linkedin.venice.controllerapi.UpdateStoreQueryParams; -import com.linkedin.venice.kafka.TopicManager; -import com.linkedin.venice.kafka.TopicManagerRepository; import com.linkedin.venice.meta.Version; +import com.linkedin.venice.pubsub.manager.TopicManager; +import com.linkedin.venice.pubsub.manager.TopicManagerRepository; import com.linkedin.venice.utils.Utils; import java.io.IOException; import java.util.Optional; @@ -50,7 +50,7 @@ public void testClusterLevelNativeReplicationConfigForNewStores() { TopicManager mockedTopicManager = mock(TopicManager.class); TopicManagerRepository mockedTopicManageRepository = mock(TopicManagerRepository.class); - doReturn(mockedTopicManager).when(mockedTopicManageRepository).getTopicManager(); + doReturn(mockedTopicManager).when(mockedTopicManageRepository).getLocalTopicManager(); doReturn(mockedTopicManager).when(mockedTopicManageRepository).getTopicManager(any(String.class)); doReturn(mockedTopicManager).when(mockedTopicManageRepository).getTopicManager(anyString()); veniceAdmin.setTopicManagerRepository(mockedTopicManageRepository); diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestDelayedRebalance.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestDelayedRebalance.java index c8a08090b3..b333c9a025 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestDelayedRebalance.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestDelayedRebalance.java @@ -32,7 +32,7 @@ public class TestDelayedRebalance { long delayRebalanceMS = testTimeOutMS * 2; int minActiveReplica = replicaFactor - 1; - @BeforeMethod + @BeforeMethod(alwaysRun = true) public void setUp() { // Start a cluster with enabling delayed rebalance. VeniceClusterCreateOptions options = new VeniceClusterCreateOptions.Builder().numberOfControllers(1) @@ -46,13 +46,13 @@ public void setUp() { cluster = ServiceFactory.getVeniceCluster(options); } - @AfterMethod + @AfterMethod(alwaysRun = true) public void cleanUp() { cluster.close(); } @Test(timeOut = 60 * Time.MS_PER_SECOND) - public void testFailOneServerWithDelayedRebalance() throws InterruptedException { + public void testFailOneServerWithDelayedRebalance() { // Test the case that fail one server with enable delayed rebalance. Helix will not move the partition to other // server. // After restart the failed server, replica would be recovered correctly. @@ -112,7 +112,7 @@ public void testFailOneServerWithDelayedRebalanceTimeout() throws InterruptedExc .getHelixStateByInstanceId(Utils.getHelixNodeIdentifier(Utils.getHostName(), failServerPort))); } - @Test + @Test(timeOut = 3 * Time.MS_PER_MINUTE) public void testModifyDelayedRebalanceTime() { // Test the case that set the shorter delayed time for a cluster, to check whether helix will do the rebalance // earlier. @@ -140,7 +140,7 @@ public void testModifyDelayedRebalanceTime() { }); } - @Test() + @Test(timeOut = 3 * Time.MS_PER_MINUTE) public void testDisableRebalanceTemporarily() throws InterruptedException { // Test the cases that fail one server after disabling delayed rebalance of the cluster. Helix will move the // partition immediately. @@ -167,7 +167,7 @@ public void testDisableRebalanceTemporarily() throws InterruptedException { }); } - @Test + @Test(timeOut = 3 * Time.MS_PER_MINUTE) public void testEnableDelayedRebalance() throws InterruptedException { String topicName = createVersionAndPushData(); // disable delayed rebalance diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestDeleteStoreDeletesRealtimeTopic.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestDeleteStoreDeletesRealtimeTopic.java index ebde83d1fc..4a7228b6be 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestDeleteStoreDeletesRealtimeTopic.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestDeleteStoreDeletesRealtimeTopic.java @@ -1,6 +1,6 @@ package com.linkedin.venice.controller; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_OPERATION_TIMEOUT_MS; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE; import static com.linkedin.venice.utils.IntegrationTestPushUtils.getSamzaProducer; import static com.linkedin.venice.utils.IntegrationTestPushUtils.makeStoreHybrid; import static com.linkedin.venice.utils.IntegrationTestPushUtils.sendStreamingRecord; @@ -16,11 +16,11 @@ import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.integration.utils.ServiceFactory; import com.linkedin.venice.integration.utils.VeniceClusterWrapper; -import com.linkedin.venice.kafka.TopicManagerRepository; import com.linkedin.venice.meta.Version; import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.exceptions.PubSubTopicDoesNotExistException; +import com.linkedin.venice.pubsub.manager.TopicManagerRepository; import com.linkedin.venice.utils.IntegrationTestPushUtils; import com.linkedin.venice.utils.TestUtils; import com.linkedin.venice.utils.Time; @@ -53,7 +53,7 @@ public void setUp() { controllerClient = ControllerClient.constructClusterControllerClient(venice.getClusterName(), venice.getRandomRouterURL()); topicManagerRepository = IntegrationTestPushUtils.getTopicManagerRepo( - DEFAULT_KAFKA_OPERATION_TIMEOUT_MS, + PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE, 100, 0l, venice.getPubSubBrokerWrapper(), @@ -105,7 +105,7 @@ public void deletingHybridStoreDeletesRealtimeTopic() { // verify realtime topic exists PubSubTopic rtTopic = pubSubTopicRepository.getTopic(Version.composeRealTimeTopic(storeName)); - assertTrue(topicManagerRepository.getTopicManager().containsTopicAndAllPartitionsAreOnline(rtTopic)); + assertTrue(topicManagerRepository.getLocalTopicManager().containsTopicAndAllPartitionsAreOnline(rtTopic)); // disable store TestUtils.assertCommand( @@ -126,11 +126,11 @@ public void deletingHybridStoreDeletesRealtimeTopic() { // verify realtime topic does not exist PubSubTopic realTimeTopicName = pubSubTopicRepository.getTopic(Version.composeRealTimeTopic(storeName)); try { - boolean isTruncated = topicManagerRepository.getTopicManager().isTopicTruncated(realTimeTopicName, 60000); + boolean isTruncated = topicManagerRepository.getLocalTopicManager().isTopicTruncated(realTimeTopicName, 60000); assertTrue( isTruncated, "Real-time buffer topic should be truncated: " + realTimeTopicName + " but retention is set to: " - + topicManagerRepository.getTopicManager().getTopicRetention(realTimeTopicName) + "."); + + topicManagerRepository.getLocalTopicManager().getTopicRetention(realTimeTopicName) + "."); LOGGER.info("Confirmed truncation of real-time topic: {}", realTimeTopicName); } catch (PubSubTopicDoesNotExistException e) { LOGGER diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestParentControllerWithMultiDataCenter.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestParentControllerWithMultiDataCenter.java index ad1f843d71..16d76b2bf3 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestParentControllerWithMultiDataCenter.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestParentControllerWithMultiDataCenter.java @@ -16,12 +16,12 @@ import com.linkedin.venice.integration.utils.ServiceFactory; import com.linkedin.venice.integration.utils.VeniceMultiClusterWrapper; import com.linkedin.venice.integration.utils.VeniceTwoLayerMultiRegionMultiClusterWrapper; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.meta.BufferReplayPolicy; import com.linkedin.venice.meta.Store; import com.linkedin.venice.meta.StoreInfo; import com.linkedin.venice.meta.Version; import com.linkedin.venice.pubsub.PubSubTopicRepository; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.schema.rmd.RmdSchemaEntry; import com.linkedin.venice.schema.rmd.RmdSchemaGenerator; import com.linkedin.venice.utils.TestUtils; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestTopicRequestOnHybridDelete.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestTopicRequestOnHybridDelete.java index a614e6a63a..19a8a998db 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestTopicRequestOnHybridDelete.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestTopicRequestOnHybridDelete.java @@ -1,7 +1,7 @@ package com.linkedin.venice.controller; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_OPERATION_TIMEOUT_MS; import static com.linkedin.venice.meta.Version.composeRealTimeTopic; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE; import static com.linkedin.venice.utils.IntegrationTestPushUtils.getSamzaProducer; import static com.linkedin.venice.utils.IntegrationTestPushUtils.makeStoreHybrid; import static com.linkedin.venice.utils.IntegrationTestPushUtils.sendStreamingRecord; @@ -19,11 +19,11 @@ import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.integration.utils.ServiceFactory; import com.linkedin.venice.integration.utils.VeniceClusterWrapper; -import com.linkedin.venice.kafka.TopicManager; -import com.linkedin.venice.kafka.TopicManagerRepository; import com.linkedin.venice.meta.Version; import com.linkedin.venice.meta.VersionStatus; import com.linkedin.venice.pubsub.PubSubTopicRepository; +import com.linkedin.venice.pubsub.manager.TopicManager; +import com.linkedin.venice.pubsub.manager.TopicManagerRepository; import com.linkedin.venice.pushmonitor.ExecutionStatus; import com.linkedin.venice.utils.IntegrationTestPushUtils; import com.linkedin.venice.utils.TestUtils; @@ -186,13 +186,13 @@ public void serverRestartOnHybridStoreKeepsVersionOnline() { public void deleteStoreAfterStartedPushAllowsNewPush() { ControllerClient controllerClient = new ControllerClient(venice.getClusterName(), venice.getRandomRouterURL()); try (TopicManagerRepository topicManagerRepository = IntegrationTestPushUtils.getTopicManagerRepo( - DEFAULT_KAFKA_OPERATION_TIMEOUT_MS, + PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE, 100, 0l, venice.getPubSubBrokerWrapper(), pubSubTopicRepository)) { - TopicManager topicManager = topicManagerRepository.getTopicManager(); + TopicManager topicManager = topicManagerRepository.getLocalTopicManager(); String storeName = Utils.getUniqueString("hybrid-store"); venice.getNewStore(storeName); diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestVeniceHelixAdminWithSharedEnvironment.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestVeniceHelixAdminWithSharedEnvironment.java index 029ef02b05..432086bd3d 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestVeniceHelixAdminWithSharedEnvironment.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestVeniceHelixAdminWithSharedEnvironment.java @@ -25,8 +25,6 @@ import com.linkedin.venice.helix.SafeHelixManager; import com.linkedin.venice.helix.ZkStoreConfigAccessor; import com.linkedin.venice.integration.utils.D2TestUtils; -import com.linkedin.venice.kafka.TopicManager; -import com.linkedin.venice.kafka.TopicManagerRepository; import com.linkedin.venice.meta.DataReplicationPolicy; import com.linkedin.venice.meta.LiveClusterConfig; import com.linkedin.venice.meta.OfflinePushStrategy; @@ -47,6 +45,8 @@ import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.exceptions.PubSubOpTimeoutException; +import com.linkedin.venice.pubsub.manager.TopicManager; +import com.linkedin.venice.pubsub.manager.TopicManagerRepository; import com.linkedin.venice.pushmonitor.ExecutionStatus; import com.linkedin.venice.pushmonitor.KillOfflinePushMessage; import com.linkedin.venice.pushmonitor.PushMonitor; @@ -561,7 +561,7 @@ public void testAddVersionAndStartIngestionTopicCreationTimeout() { doThrow(new PubSubOpTimeoutException("mock timeout")).when(mockedTopicManager) .createTopic(any(), anyInt(), anyInt(), anyBoolean(), anyBoolean(), any(), eq(true)); TopicManagerRepository mockedTopicManageRepository = mock(TopicManagerRepository.class); - doReturn(mockedTopicManager).when(mockedTopicManageRepository).getTopicManager(); + doReturn(mockedTopicManager).when(mockedTopicManageRepository).getLocalTopicManager(); doReturn(mockedTopicManager).when(mockedTopicManageRepository).getTopicManager(any(String.class)); doReturn(mockedTopicManager).when(mockedTopicManageRepository).getTopicManager(anyString()); veniceAdmin.setTopicManagerRepository(mockedTopicManageRepository); @@ -1590,7 +1590,7 @@ public void testAddVersionWithRemoteKafkaBootstrapServers() { TopicManager mockedTopicManager = mock(TopicManager.class); TopicManagerRepository mockedTopicManageRepository = mock(TopicManagerRepository.class); - doReturn(mockedTopicManager).when(mockedTopicManageRepository).getTopicManager(); + doReturn(mockedTopicManager).when(mockedTopicManageRepository).getLocalTopicManager(); doReturn(mockedTopicManager).when(mockedTopicManageRepository).getTopicManager(any(String.class)); doReturn(mockedTopicManager).when(mockedTopicManageRepository).getTopicManager(anyString()); veniceAdmin.setTopicManagerRepository(mockedTopicManageRepository); @@ -1744,7 +1744,7 @@ public void testVersionLevelActiveActiveReplicationConfig() { TopicManager mockedTopicManager = mock(TopicManager.class); TopicManagerRepository mockedTopicManageRepository = mock(TopicManagerRepository.class); - doReturn(mockedTopicManager).when(mockedTopicManageRepository).getTopicManager(); + doReturn(mockedTopicManager).when(mockedTopicManageRepository).getLocalTopicManager(); doReturn(mockedTopicManager).when(mockedTopicManageRepository).getTopicManager(any(String.class)); doReturn(mockedTopicManager).when(mockedTopicManageRepository).getTopicManager(anyString()); veniceAdmin.setTopicManagerRepository(mockedTopicManageRepository); diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/kafka/consumer/AdminConsumptionTaskIntegrationTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/kafka/consumer/AdminConsumptionTaskIntegrationTest.java index 63f10b9e84..a01482270b 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/kafka/consumer/AdminConsumptionTaskIntegrationTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/kafka/consumer/AdminConsumptionTaskIntegrationTest.java @@ -1,7 +1,7 @@ package com.linkedin.venice.controller.kafka.consumer; import static com.linkedin.venice.integration.utils.VeniceClusterWrapperConstants.STANDALONE_REGION_NAME; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_OPERATION_TIMEOUT_MS; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE; import com.linkedin.venice.controller.kafka.AdminTopicUtils; import com.linkedin.venice.controller.kafka.protocol.admin.AdminOperation; @@ -17,10 +17,10 @@ import com.linkedin.venice.integration.utils.VeniceControllerCreateOptions; import com.linkedin.venice.integration.utils.VeniceControllerWrapper; import com.linkedin.venice.integration.utils.ZkServerWrapper; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.pubsub.PubSubProducerAdapterFactory; import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.utils.IntegrationTestPushUtils; import com.linkedin.venice.utils.TestUtils; import com.linkedin.venice.utils.Time; @@ -61,12 +61,12 @@ public void testSkipMessageEndToEnd() throws ExecutionException, InterruptedExce TopicManager topicManager = IntegrationTestPushUtils .getTopicManagerRepo( - DEFAULT_KAFKA_OPERATION_TIMEOUT_MS, + PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE, 100, 0l, pubSubBrokerWrapper, pubSubTopicRepository) - .getTopicManager()) { + .getLocalTopicManager()) { PubSubTopic adminTopic = pubSubTopicRepository.getTopic(AdminTopicUtils.getTopicNameFromClusterName(clusterName)); topicManager.createTopic(adminTopic, 1, 1, true); String storeName = "test-store"; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestEmptyPush.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestEmptyPush.java index d386e32f8d..1567b54b65 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestEmptyPush.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestEmptyPush.java @@ -3,7 +3,7 @@ import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_METRIC_COLLECTION_ENABLED; import static com.linkedin.venice.hadoop.VenicePushJobConstants.SEND_CONTROL_MESSAGES_DIRECTLY; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_OPERATION_TIMEOUT_MS; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE; import static com.linkedin.venice.utils.IntegrationTestPushUtils.defaultVPJProps; import static com.linkedin.venice.utils.IntegrationTestPushUtils.runVPJ; import static com.linkedin.venice.utils.IntegrationTestPushUtils.sendStreamingRecord; @@ -27,10 +27,10 @@ import com.linkedin.venice.integration.utils.ServiceFactory; import com.linkedin.venice.integration.utils.VeniceClusterCreateOptions; import com.linkedin.venice.integration.utils.VeniceClusterWrapper; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.meta.Store; import com.linkedin.venice.meta.Version; import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.utils.DataProviderUtils; import com.linkedin.venice.utils.DictionaryUtils; import com.linkedin.venice.utils.IntegrationTestPushUtils; @@ -108,12 +108,12 @@ public void testEmptyPushByChangingCompressionStrategyForHybridStore() throws IO TopicManager topicManager = IntegrationTestPushUtils .getTopicManagerRepo( - DEFAULT_KAFKA_OPERATION_TIMEOUT_MS, + PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE, 100, 0l, venice.getPubSubBrokerWrapper(), venice.getPubSubTopicRepository()) - .getTopicManager()) { + .getTopicManager(venice.getPubSubBrokerWrapper().getAddress())) { controllerClient.createNewStore(storeName, "owner", STRING_SCHEMA.toString(), STRING_SCHEMA.toString()); controllerClient.updateStore( storeName, diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestHybrid.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestHybrid.java index aeb8a5b4b2..9427a47ebf 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestHybrid.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestHybrid.java @@ -14,9 +14,9 @@ import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFER_VERSION_SWAP; import static com.linkedin.venice.integration.utils.VeniceClusterWrapper.DEFAULT_KEY_SCHEMA; import static com.linkedin.venice.integration.utils.VeniceClusterWrapper.DEFAULT_VALUE_SCHEMA; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_OPERATION_TIMEOUT_MS; import static com.linkedin.venice.meta.BufferReplayPolicy.REWIND_FROM_EOP; import static com.linkedin.venice.meta.BufferReplayPolicy.REWIND_FROM_SOP; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE; import static com.linkedin.venice.router.api.VenicePathParser.TYPE_STORAGE; import static com.linkedin.venice.utils.IntegrationTestPushUtils.createStoreForJob; import static com.linkedin.venice.utils.IntegrationTestPushUtils.defaultVPJProps; @@ -59,7 +59,6 @@ import com.linkedin.venice.integration.utils.VeniceControllerWrapper; import com.linkedin.venice.integration.utils.VeniceServerWrapper; import com.linkedin.venice.integration.utils.ZkServerWrapper; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.kafka.protocol.GUID; import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; import com.linkedin.venice.kafka.protocol.LeaderMetadata; @@ -83,6 +82,7 @@ import com.linkedin.venice.producer.online.OnlineProducerFactory; import com.linkedin.venice.pubsub.PubSubProducerAdapterFactory; import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.pushmonitor.ExecutionStatus; import com.linkedin.venice.pushmonitor.OfflinePushStatus; import com.linkedin.venice.pushmonitor.PartitionStatus; @@ -98,6 +98,7 @@ import com.linkedin.venice.utils.DataProviderUtils; import com.linkedin.venice.utils.IntegrationTestPushUtils; import com.linkedin.venice.utils.Pair; +import com.linkedin.venice.utils.StoreUtils; import com.linkedin.venice.utils.TestMockTime; import com.linkedin.venice.utils.TestUtils; import com.linkedin.venice.utils.TestWriteUtils; @@ -199,12 +200,12 @@ public void testHybridInitializationOnMultiColo() throws IOException { TopicManager topicManager = IntegrationTestPushUtils .getTopicManagerRepo( - DEFAULT_KAFKA_OPERATION_TIMEOUT_MS, + PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE, 100, 0l, venice.getPubSubBrokerWrapper(), venice.getPubSubTopicRepository()) - .getTopicManager()) { + .getLocalTopicManager()) { long streamingRewindSeconds = 25L; long streamingMessageLag = 2L; final String storeName = Utils.getUniqueString("multi-colo-hybrid-store"); @@ -256,7 +257,7 @@ public void testHybridInitializationOnMultiColo() throws IOException { assertEquals( topicManager.getTopicRetention( sharedVenice.getPubSubTopicRepository().getTopic(Version.composeRealTimeTopic(storeName))), - TopicManager.getExpectedRetentionTimeInMs(store, hybridStoreConfig), + StoreUtils.getExpectedRetentionTimeInMs(store, hybridStoreConfig), "RT retention not configured properly"); // Make sure RT retention is updated when the rewind time is updated long newStreamingRewindSeconds = 600; @@ -266,7 +267,7 @@ public void testHybridInitializationOnMultiColo() throws IOException { assertEquals( topicManager.getTopicRetention( sharedVenice.getPubSubTopicRepository().getTopic(Version.composeRealTimeTopic(storeName))), - TopicManager.getExpectedRetentionTimeInMs(store, hybridStoreConfig), + StoreUtils.getExpectedRetentionTimeInMs(store, hybridStoreConfig), "RT retention not updated properly"); } } @@ -332,12 +333,12 @@ public void testHybridEndToEnd(boolean multiDivStream, boolean chunkingEnabled, TopicManager topicManager = IntegrationTestPushUtils .getTopicManagerRepo( - DEFAULT_KAFKA_OPERATION_TIMEOUT_MS, + PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE, 100, 0l, venice.getPubSubBrokerWrapper(), sharedVenice.getPubSubTopicRepository()) - .getTopicManager()) { + .getLocalTopicManager()) { Cache cacheNothingCache = Mockito.mock(Cache.class); Mockito.when(cacheNothingCache.getIfPresent(Mockito.any())).thenReturn(null); diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestHybridQuota.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestHybridQuota.java index 66148e58d4..e573d112f5 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestHybridQuota.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestHybridQuota.java @@ -6,7 +6,7 @@ import static com.linkedin.venice.ConfigKeys.SERVER_CONSUMER_POOL_SIZE_PER_KAFKA_CLUSTER; import static com.linkedin.venice.ConfigKeys.SERVER_PROMOTION_TO_LEADER_REPLICA_DELAY_SECONDS; import static com.linkedin.venice.ConfigKeys.SSL_TO_KAFKA_LEGACY; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_OPERATION_TIMEOUT_MS; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE; import static com.linkedin.venice.utils.IntegrationTestPushUtils.createStoreForJob; import static com.linkedin.venice.utils.IntegrationTestPushUtils.defaultVPJProps; import static com.linkedin.venice.utils.IntegrationTestPushUtils.getSamzaProducer; @@ -29,11 +29,11 @@ import com.linkedin.venice.helix.ZkClientFactory; import com.linkedin.venice.integration.utils.ServiceFactory; import com.linkedin.venice.integration.utils.VeniceClusterWrapper; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.meta.PersistenceType; import com.linkedin.venice.meta.Store; import com.linkedin.venice.meta.Version; import com.linkedin.venice.meta.VersionImpl; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.pushmonitor.HybridStoreQuotaStatus; import com.linkedin.venice.utils.IntegrationTestPushUtils; import com.linkedin.venice.utils.TestUtils; @@ -134,12 +134,12 @@ public void testHybridStoreQuota(boolean chunkingEnabled, boolean isStreamReproc TopicManager topicManager = IntegrationTestPushUtils .getTopicManagerRepo( - DEFAULT_KAFKA_OPERATION_TIMEOUT_MS, + PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE, 100L, 0L, sharedVenice.getPubSubBrokerWrapper(), sharedVenice.getPubSubTopicRepository()) - .getTopicManager()) { + .getLocalTopicManager()) { // Setting the hybrid store quota here will cause the VPJ push failed. ControllerResponse response = controllerClient.updateStore( diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestHybridStoreDeletion.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestHybridStoreDeletion.java index 1fa057eb37..360f20d286 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestHybridStoreDeletion.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestHybridStoreDeletion.java @@ -11,7 +11,6 @@ import static com.linkedin.venice.ConfigKeys.SERVER_PROMOTION_TO_LEADER_REPLICA_DELAY_SECONDS; import static com.linkedin.venice.ConfigKeys.SERVER_SHARED_CONSUMER_ASSIGNMENT_STRATEGY; import static com.linkedin.venice.ConfigKeys.SSL_TO_KAFKA_LEGACY; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_OPERATION_TIMEOUT_MS; import static com.linkedin.venice.utils.IntegrationTestPushUtils.getSamzaProducer; import static com.linkedin.venice.utils.IntegrationTestPushUtils.sendCustomSizeStreamingRecord; import static com.linkedin.venice.utils.TestWriteUtils.STRING_SCHEMA; @@ -31,11 +30,12 @@ import com.linkedin.venice.integration.utils.VeniceControllerCreateOptions; import com.linkedin.venice.integration.utils.VeniceControllerWrapper; import com.linkedin.venice.integration.utils.ZkServerWrapper; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.meta.PersistenceType; import com.linkedin.venice.meta.Store; import com.linkedin.venice.meta.Version; +import com.linkedin.venice.pubsub.PubSubConstants; import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.utils.IntegrationTestPushUtils; import com.linkedin.venice.utils.TestUtils; import com.linkedin.venice.utils.Time; @@ -146,12 +146,12 @@ public void testHybridStoreRTDeletionWhileIngesting() { TopicManager topicManager = IntegrationTestPushUtils .getTopicManagerRepo( - DEFAULT_KAFKA_OPERATION_TIMEOUT_MS, + PubSubConstants.PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE, 100, 0l, veniceCluster.getPubSubBrokerWrapper(), veniceCluster.getPubSubTopicRepository()) - .getTopicManager()) { + .getLocalTopicManager()) { createStoresAndVersions(controllerClient, storeNames, streamingRewindSeconds, streamingMessageLag); diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestPushJobWithNativeReplication.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestPushJobWithNativeReplication.java index e450596869..0da43e03c9 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestPushJobWithNativeReplication.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestPushJobWithNativeReplication.java @@ -298,7 +298,7 @@ public void testNativeReplicationForBatchPush(int recordCount, int partitionCoun long latestOffsetInVersionTopic = childDataCenter.getRandomController() .getVeniceAdmin() .getTopicManager() - .getPartitionLatestOffsetAndRetry(versionTopicPartition, 5); + .getLatestOffsetWithRetries(versionTopicPartition, 5); // Get the offset metadata of the selected partition from storage node StorageMetadataService metadataService = serverInRemoteFabric.getStorageMetadataService(); OffsetRecord offsetRecord = metadataService.getLastOffset(versionTopic, partitionId); @@ -358,8 +358,9 @@ public void testNativeReplicationWithLeadershipHandover() throws Exception { ClientConfig.defaultGenericClientConfig(storeName).setVeniceURL(routerUrl))) { for (int i = 1; i <= recordCount; ++i) { String expected = "test_name_" + i; - String actual = client.get(Integer.toString(i)).get().toString(); - Assert.assertEquals(actual, expected); + Object val = client.get(Integer.toString(i)).get(); + Assert.assertNotNull(val, "Value should not be null for key " + i); + Assert.assertEquals(val.toString(), expected); } } }); diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestStuckConsumerRepair.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestStuckConsumerRepair.java index 8ed72dc1c7..50d83ae91d 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestStuckConsumerRepair.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestStuckConsumerRepair.java @@ -15,7 +15,7 @@ import static com.linkedin.venice.ConfigKeys.SERVER_STUCK_CONSUMER_REPAIR_INTERVAL_SECOND; import static com.linkedin.venice.ConfigKeys.SERVER_STUCK_CONSUMER_REPAIR_THRESHOLD_SECOND; import static com.linkedin.venice.ConfigKeys.SSL_TO_KAFKA_LEGACY; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_OPERATION_TIMEOUT_MS; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE; import static com.linkedin.venice.utils.IntegrationTestPushUtils.createStoreForJob; import static com.linkedin.venice.utils.IntegrationTestPushUtils.defaultVPJProps; import static com.linkedin.venice.utils.IntegrationTestPushUtils.getSamzaProducer; @@ -39,9 +39,9 @@ import com.linkedin.venice.integration.utils.ServiceFactory; import com.linkedin.venice.integration.utils.VeniceClusterWrapper; import com.linkedin.venice.integration.utils.VeniceServerWrapper; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.meta.PersistenceType; import com.linkedin.venice.meta.Version; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.utils.IntegrationTestPushUtils; import com.linkedin.venice.utils.TestUtils; import com.linkedin.venice.utils.TestWriteUtils; @@ -164,12 +164,12 @@ public void testStuckConsumerRepair() throws Exception { TopicManager topicManager = IntegrationTestPushUtils .getTopicManagerRepo( - DEFAULT_KAFKA_OPERATION_TIMEOUT_MS, + PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE, 100, 0l, venice.getPubSubBrokerWrapper(), sharedVenice.getPubSubTopicRepository()) - .getTopicManager()) { + .getLocalTopicManager()) { Cache cacheNothingCache = Mockito.mock(Cache.class); Mockito.when(cacheNothingCache.getIfPresent(Mockito.any())).thenReturn(null); diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/hadoop/input/kafka/TestKafkaInputFormat.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/hadoop/input/kafka/TestKafkaInputFormat.java index f63f876a26..373141ed47 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/hadoop/input/kafka/TestKafkaInputFormat.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/hadoop/input/kafka/TestKafkaInputFormat.java @@ -3,13 +3,13 @@ import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_TOPIC; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_OPERATION_TIMEOUT_MS; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE; import com.linkedin.venice.integration.utils.PubSubBrokerWrapper; import com.linkedin.venice.integration.utils.ServiceFactory; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.pubsub.PubSubProducerAdapterFactory; import com.linkedin.venice.pubsub.PubSubTopicRepository; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.utils.IntegrationTestPushUtils; import com.linkedin.venice.utils.Time; import com.linkedin.venice.utils.Utils; @@ -44,12 +44,12 @@ public void setUp() { manager = IntegrationTestPushUtils .getTopicManagerRepo( - DEFAULT_KAFKA_OPERATION_TIMEOUT_MS, + PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE, 100L, 24 * Time.MS_PER_HOUR, pubSubBrokerWrapper, pubSubTopicRepository) - .getTopicManager(); + .getLocalTopicManager(); } @AfterClass diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/hadoop/input/kafka/TestKafkaInputRecordReader.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/hadoop/input/kafka/TestKafkaInputRecordReader.java index 8212984748..6f1a63ecae 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/hadoop/input/kafka/TestKafkaInputRecordReader.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/hadoop/input/kafka/TestKafkaInputRecordReader.java @@ -3,15 +3,15 @@ import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_TOPIC; import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_SOURCE_KEY_SCHEMA_STRING_PROP; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_OPERATION_TIMEOUT_MS; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE; import com.linkedin.venice.hadoop.input.kafka.avro.KafkaInputMapperKey; import com.linkedin.venice.hadoop.input.kafka.avro.KafkaInputMapperValue; import com.linkedin.venice.hadoop.input.kafka.avro.MapperValueType; import com.linkedin.venice.integration.utils.PubSubBrokerWrapper; import com.linkedin.venice.integration.utils.ServiceFactory; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.pubsub.PubSubTopicRepository; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.storage.protocol.ChunkedKeySuffix; import com.linkedin.venice.utils.ByteUtils; import com.linkedin.venice.utils.IntegrationTestPushUtils; @@ -43,12 +43,12 @@ public void setUp() { manager = IntegrationTestPushUtils .getTopicManagerRepo( - DEFAULT_KAFKA_OPERATION_TIMEOUT_MS, + PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE, 100L, 24 * Time.MS_PER_HOUR, pubSubBrokerWrapper, pubSubTopicRepository) - .getTopicManager(); + .getLocalTopicManager(); } @AfterClass diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/integration/utils/VeniceServerWrapper.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/integration/utils/VeniceServerWrapper.java index c95fa57162..e2459297d8 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/integration/utils/VeniceServerWrapper.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/integration/utils/VeniceServerWrapper.java @@ -31,6 +31,7 @@ import static com.linkedin.venice.ConfigKeys.SERVER_PARTITION_GRACEFUL_DROP_DELAY_IN_SECONDS; import static com.linkedin.venice.ConfigKeys.SERVER_PROMOTION_TO_LEADER_REPLICA_DELAY_SECONDS; import static com.linkedin.venice.ConfigKeys.SERVER_REST_SERVICE_STORAGE_THREAD_NUM; +import static com.linkedin.venice.ConfigKeys.SERVER_SOURCE_TOPIC_OFFSET_CHECK_INTERVAL_MS; import static com.linkedin.venice.ConfigKeys.SERVER_SSL_HANDSHAKE_THREAD_POOL_SIZE; import static com.linkedin.venice.ConfigKeys.SYSTEM_SCHEMA_CLUSTER_NAME; import static com.linkedin.venice.ConfigKeys.SYSTEM_SCHEMA_INITIALIZATION_AT_START_TIME_ENABLED; @@ -241,6 +242,7 @@ static StatefulServiceProvider generateService( .put(CLUSTER_DISCOVERY_D2_SERVICE, VeniceRouterWrapper.CLUSTER_DISCOVERY_D2_SERVICE_NAME) .put(SERVER_SSL_HANDSHAKE_THREAD_POOL_SIZE, 10) .put(SYSTEM_SCHEMA_INITIALIZATION_AT_START_TIME_ENABLED, true) + .put(SERVER_SOURCE_TOPIC_OFFSET_CHECK_INTERVAL_MS, 100) .put(LOCAL_CONTROLLER_D2_SERVICE_NAME, VeniceControllerWrapper.D2_SERVICE_NAME) .put(LOCAL_D2_ZK_HOST, zkAddress) .put( diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/kafka/KafkaConsumptionTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/kafka/KafkaConsumptionTest.java index df9b92d345..be92d8d3c2 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/kafka/KafkaConsumptionTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/kafka/KafkaConsumptionTest.java @@ -1,7 +1,7 @@ package com.linkedin.venice.kafka; import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_OPERATION_TIMEOUT_MS; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE; import static com.linkedin.venice.utils.TestUtils.waitForNonDeterministicCompletion; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -35,6 +35,7 @@ import com.linkedin.venice.pubsub.api.PubSubProducerAdapter; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.serialization.avro.OptimizedKafkaValueSerializer; import com.linkedin.venice.stats.TehutiUtils; import com.linkedin.venice.throttle.EventThrottler; @@ -105,12 +106,12 @@ public void setUp() { topicManager = IntegrationTestPushUtils .getTopicManagerRepo( - DEFAULT_KAFKA_OPERATION_TIMEOUT_MS, + PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE, 100L, MIN_COMPACTION_LAG, localPubSubBroker, pubSubTopicRepository) - .getTopicManager(); + .getTopicManager(localPubSubBroker.getAddress()); Cache cacheNothingCache = mock(Cache.class); Mockito.when(cacheNothingCache.getIfPresent(Mockito.any())).thenReturn(null); topicManager.setTopicConfigCache(cacheNothingCache); @@ -121,12 +122,12 @@ public void setUp() { remoteTopicManager = IntegrationTestPushUtils .getTopicManagerRepo( - DEFAULT_KAFKA_OPERATION_TIMEOUT_MS, + PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE, 100L, MIN_COMPACTION_LAG, remotePubSubBroker, pubSubTopicRepository) - .getTopicManager(); + .getTopicManager(remotePubSubBroker.getAddress()); Cache remoteCacheNothingCache = mock(Cache.class); Mockito.when(remoteCacheNothingCache.getIfPresent(Mockito.any())).thenReturn(null); remoteTopicManager.setTopicConfigCache(remoteCacheNothingCache); diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherTest.java deleted file mode 100644 index a6efdd8bb3..0000000000 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/kafka/partitionoffset/PartitionOffsetFetcherTest.java +++ /dev/null @@ -1,64 +0,0 @@ -package com.linkedin.venice.kafka.partitionoffset; - -import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; - -import com.linkedin.venice.integration.utils.PubSubBrokerWrapper; -import com.linkedin.venice.integration.utils.ServiceFactory; -import com.linkedin.venice.pubsub.PubSubAdminAdapterFactory; -import com.linkedin.venice.pubsub.PubSubConsumerAdapterFactory; -import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl; -import com.linkedin.venice.pubsub.PubSubTopicRepository; -import com.linkedin.venice.pubsub.api.PubSubTopicPartition; -import com.linkedin.venice.pubsub.api.exceptions.PubSubTopicDoesNotExistException; -import com.linkedin.venice.utils.Time; -import com.linkedin.venice.utils.Utils; -import com.linkedin.venice.utils.VeniceProperties; -import com.linkedin.venice.utils.lazy.Lazy; -import java.util.Optional; -import java.util.Properties; -import org.testng.Assert; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; - - -public class PartitionOffsetFetcherTest { - private PubSubBrokerWrapper pubSubBrokerWrapper; - - private final PubSubTopicRepository pubSubTopicRepository = new PubSubTopicRepository(); - - @BeforeClass - public void setUp() { - this.pubSubBrokerWrapper = ServiceFactory.getPubSubBroker(); - } - - @AfterClass - public void close() { - this.pubSubBrokerWrapper.close(); - } - - @Test - public void testGetPartitionLatestOffsetAndRetry() { - - PubSubAdminAdapterFactory pubSubAdminAdapterFactory = - pubSubBrokerWrapper.getPubSubClientsFactory().getAdminAdapterFactory(); - PubSubConsumerAdapterFactory pubSubConsumerAdapterFactory = - pubSubBrokerWrapper.getPubSubClientsFactory().getConsumerAdapterFactory(); - Properties properties = new Properties(); - properties.setProperty(KAFKA_BOOTSTRAP_SERVERS, pubSubBrokerWrapper.getAddress()); - try (PartitionOffsetFetcher fetcher = PartitionOffsetFetcherFactory.createDefaultPartitionOffsetFetcher( - pubSubConsumerAdapterFactory, - new VeniceProperties(properties), - pubSubBrokerWrapper.getAddress(), - Lazy.of(() -> pubSubAdminAdapterFactory.create(new VeniceProperties(properties), pubSubTopicRepository)), - Time.MS_PER_SECOND, - Optional.empty())) { - String topic = Utils.getUniqueString("topic") + "_v1"; - PubSubTopicPartition pubSubTopicPartition = - new PubSubTopicPartitionImpl(pubSubTopicRepository.getTopic(topic), 0); - Assert.assertThrows( - PubSubTopicDoesNotExistException.class, - () -> fetcher.getPartitionLatestOffsetAndRetry(pubSubTopicPartition, 1)); - } - } -} diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/pubsub/api/consumer/PubSubConsumerAdapterTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/pubsub/api/consumer/PubSubConsumerAdapterTest.java index c2bc40a249..09872014ac 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/pubsub/api/consumer/PubSubConsumerAdapterTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/pubsub/api/consumer/PubSubConsumerAdapterTest.java @@ -1,5 +1,8 @@ package com.linkedin.venice.pubsub.api.consumer; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_CONSUMER_CHECK_TOPIC_EXISTENCE; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_CONSUMER_POSITION_RESET_STRATEGY; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_CONSUMER_POSITION_RESET_STRATEGY_DEFAULT_VALUE; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; @@ -37,6 +40,7 @@ import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -110,7 +114,9 @@ public void setUpMethod() { properties.setProperty( PubSubConstants.PUBSUB_CONSUMER_API_DEFAULT_TIMEOUT_MS, String.valueOf(PUBSUB_CONSUMER_API_DEFAULT_TIMEOUT_MS)); - properties.setProperty(PubSubConstants.PUBSUB_CONSUMER_CHECK_TOPIC_EXISTENCE, "true"); + properties.setProperty(PUBSUB_CONSUMER_CHECK_TOPIC_EXISTENCE, "true"); + properties + .setProperty(PUBSUB_CONSUMER_POSITION_RESET_STRATEGY, PUBSUB_CONSUMER_POSITION_RESET_STRATEGY_DEFAULT_VALUE); properties.putAll(pubSubBrokerWrapper.getAdditionalConfig()); properties.putAll(pubSubBrokerWrapper.getMergeableConfigs()); VeniceProperties veniceProperties = new VeniceProperties(properties); @@ -875,7 +881,7 @@ public void testResetOffsetForExistingTopicWithValidPartitionButNoSubscription() } // Test: poll works as expected when called on an existing topic with a valid partition and subscription. - // poll should not block for longer than the specified timeout even consumer is subscribed to multiple + // poll should not block for longer than the specified timeout even when consumer is subscribed to multiple // topic-partitions and some topic-partitions do not exist. @Test(timeOut = 3 * Time.MS_PER_MINUTE) public void testPollPauseResume() throws ExecutionException, InterruptedException, TimeoutException { @@ -1203,6 +1209,23 @@ public void testPollPauseResume() throws ExecutionException, InterruptedExceptio assertTrue( elapsedTime <= PUBSUB_OP_TIMEOUT_WITH_VARIANCE, "Batch unsubscribe should not block for longer than the timeout"); + + // We use the "earliest" offset reset policy in prod. This means that if we subscribe with an offset + // greater than the end offset, the consumer will seek to the beginning of the partition. Subsequent polls + // will return the first message in the partition. + assertFalse(pubSubConsumerAdapter.hasAnySubscription(), "Should be subscribed to the topic and partition"); + pubSubConsumerAdapter.subscribe(partitionA0, endOffsets.get(partitionA0)); // sub will add +1 to the offset + messages = Collections.emptyMap(); + while (messages.isEmpty()) { + messages = pubSubConsumerAdapter.poll(pollTimeout); + if (messages.isEmpty() || messages.get(partitionA0) == null || messages.get(partitionA0).isEmpty()) { + messages = Collections.emptyMap(); + } + } + assertTrue(messages.containsKey(partitionA0), "Should have messages for A0"); + assertTrue(messages.get(partitionA0).size() > 0, "Should have messages for A0"); + // offset should be at the first message + assertEquals((long) messages.get(partitionA0).get(0).getOffset(), 0, "Poll should start from the beginning"); } // Note: The following test may not work for non-Kafka PubSub implementations. diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/pubsub/manager/TopicManagerE2ETest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/pubsub/manager/TopicManagerE2ETest.java new file mode 100644 index 0000000000..c76a30eef7 --- /dev/null +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/pubsub/manager/TopicManagerE2ETest.java @@ -0,0 +1,429 @@ +package com.linkedin.venice.pubsub.manager; + +import static com.linkedin.venice.utils.TestUtils.waitForNonDeterministicAssertion; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertThrows; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.expectThrows; + +import com.linkedin.venice.ConfigKeys; +import com.linkedin.venice.exceptions.VeniceException; +import com.linkedin.venice.integration.utils.PubSubBrokerWrapper; +import com.linkedin.venice.integration.utils.ServiceFactory; +import com.linkedin.venice.pubsub.PubSubClientsFactory; +import com.linkedin.venice.pubsub.PubSubConstants; +import com.linkedin.venice.pubsub.PubSubTopicConfiguration; +import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl; +import com.linkedin.venice.pubsub.PubSubTopicRepository; +import com.linkedin.venice.pubsub.api.PubSubAdminAdapter; +import com.linkedin.venice.pubsub.api.PubSubConsumerAdapter; +import com.linkedin.venice.pubsub.api.PubSubMessageDeserializer; +import com.linkedin.venice.pubsub.api.PubSubProduceResult; +import com.linkedin.venice.pubsub.api.PubSubProducerAdapter; +import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.api.PubSubTopicPartition; +import com.linkedin.venice.pubsub.api.exceptions.PubSubTopicDoesNotExistException; +import com.linkedin.venice.stats.StatsErrorCode; +import com.linkedin.venice.utils.PubSubHelper; +import com.linkedin.venice.utils.PubSubHelper.MutablePubSubMessage; +import com.linkedin.venice.utils.Time; +import com.linkedin.venice.utils.Utils; +import com.linkedin.venice.utils.VeniceProperties; +import com.linkedin.venice.utils.lazy.Lazy; +import io.tehuti.metrics.MetricsRepository; +import java.time.Duration; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Properties; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + + +public class TopicManagerE2ETest { + // timeout for pub-sub operations + private static final Duration PUBSUB_OP_TIMEOUT = Duration.ofSeconds(15); + // add a variance of 5 seconds to the timeout to account for fluctuations in the test environment + private static final long PUBSUB_OP_TIMEOUT_WITH_VARIANCE = PUBSUB_OP_TIMEOUT.toMillis() + 5000; + // timeout for pub-sub consumer APIs which do not have a timeout parameter + private static final int PUBSUB_CONSUMER_API_DEFAULT_TIMEOUT_MS = 10_000; + // add a variance of 5 seconds to the timeout to account for fluctuations in the test environment + private static final long PUBSUB_CONSUMER_API_DEFAULT_TIMEOUT_MS_WITH_VARIANCE = + PUBSUB_CONSUMER_API_DEFAULT_TIMEOUT_MS + 5000; + private static final int REPLICATION_FACTOR = 1; + private static final boolean IS_LOG_COMPACTED = false; + private static final int MIN_IN_SYNC_REPLICAS = 1; + private static final long RETENTION_IN_MS = Duration.ofDays(3).toMillis(); + private static final long MIN_LOG_COMPACTION_LAG_MS = Duration.ofDays(1).toMillis(); + private static final long MAX_LOG_COMPACTION_LAG_MS = Duration.ofDays(2).toMillis(); + private static final PubSubTopicConfiguration TOPIC_CONFIGURATION = new PubSubTopicConfiguration( + Optional.of(RETENTION_IN_MS), + IS_LOG_COMPACTED, + Optional.of(MIN_IN_SYNC_REPLICAS), + MIN_LOG_COMPACTION_LAG_MS, + Optional.of(MAX_LOG_COMPACTION_LAG_MS)); + + private PubSubBrokerWrapper pubSubBrokerWrapper; + private Lazy pubSubAdminAdapterLazy; + private Lazy pubSubProducerAdapterLazy; + private Lazy pubSubConsumerAdapterLazy; + private PubSubMessageDeserializer pubSubMessageDeserializer; + private PubSubTopicRepository pubSubTopicRepository; + private PubSubClientsFactory pubSubClientsFactory; + private TopicManagerRepository topicManagerRepository; + private TopicManager topicManager; + private TopicManagerContext.Builder topicManagerContextBuilder; + private MetricsRepository metricsRepository; + + @BeforeClass(alwaysRun = true) + public void setUp() { + pubSubBrokerWrapper = ServiceFactory.getPubSubBroker(); + pubSubMessageDeserializer = PubSubMessageDeserializer.getInstance(); + pubSubTopicRepository = new PubSubTopicRepository(); + pubSubClientsFactory = pubSubBrokerWrapper.getPubSubClientsFactory(); + } + + @AfterClass(alwaysRun = true) + public void tearDown() { + Utils.closeQuietlyWithErrorLogged(pubSubBrokerWrapper); + } + + @BeforeMethod(alwaysRun = true) + public void setUpMethod() { + String clientId = Utils.getUniqueString("TopicManageE2EITest"); + Properties properties = new Properties(); + properties.setProperty(ConfigKeys.KAFKA_BOOTSTRAP_SERVERS, pubSubBrokerWrapper.getAddress()); + properties.setProperty( + PubSubConstants.PUBSUB_CONSUMER_API_DEFAULT_TIMEOUT_MS, + String.valueOf(PUBSUB_CONSUMER_API_DEFAULT_TIMEOUT_MS)); + properties.setProperty(PubSubConstants.PUBSUB_CONSUMER_CHECK_TOPIC_EXISTENCE, "true"); + properties.putAll(pubSubBrokerWrapper.getAdditionalConfig()); + properties.putAll(pubSubBrokerWrapper.getMergeableConfigs()); + VeniceProperties veniceProperties = new VeniceProperties(properties); + pubSubProducerAdapterLazy = + Lazy.of(() -> pubSubClientsFactory.getProducerAdapterFactory().create(veniceProperties, clientId, null)); + pubSubAdminAdapterLazy = + Lazy.of(() -> pubSubClientsFactory.getAdminAdapterFactory().create(veniceProperties, pubSubTopicRepository)); + pubSubConsumerAdapterLazy = Lazy.of( + () -> pubSubClientsFactory.getConsumerAdapterFactory() + .create(veniceProperties, false, pubSubMessageDeserializer, clientId)); + + metricsRepository = new MetricsRepository(); + topicManagerContextBuilder = new TopicManagerContext.Builder().setPubSubTopicRepository(pubSubTopicRepository) + .setMetricsRepository(metricsRepository) + .setTopicMetadataFetcherConsumerPoolSize(2) + .setTopicMetadataFetcherThreadPoolSize(6) + .setTopicOffsetCheckIntervalMs(100) + .setPubSubPropertiesSupplier(k -> veniceProperties) + .setPubSubAdminAdapterFactory(pubSubClientsFactory.getAdminAdapterFactory()) + .setPubSubConsumerAdapterFactory(pubSubClientsFactory.getConsumerAdapterFactory()); + + topicManagerRepository = + new TopicManagerRepository(topicManagerContextBuilder.build(), pubSubBrokerWrapper.getAddress()); + topicManager = topicManagerRepository.getLocalTopicManager(); + } + + @AfterMethod(alwaysRun = true) + public void tearDownMethod() { + if (pubSubProducerAdapterLazy.isPresent()) { + pubSubProducerAdapterLazy.get().close(0, false); + } + if (pubSubAdminAdapterLazy.isPresent()) { + Utils.closeQuietlyWithErrorLogged(pubSubAdminAdapterLazy.get()); + } + if (pubSubConsumerAdapterLazy.isPresent()) { + Utils.closeQuietlyWithErrorLogged(pubSubConsumerAdapterLazy.get()); + } + + if (topicManagerRepository != null) { + Utils.closeQuietlyWithErrorLogged(topicManagerRepository); + } + } + + @Test(timeOut = 5 * Time.MS_PER_MINUTE, invocationCount = 1) + public void testAsyncApis() throws ExecutionException, InterruptedException, TimeoutException { + int numPartitions = 3; + int replicationFactor = 1; + boolean isEternalTopic = true; + PubSubTopic testTopic = pubSubTopicRepository.getTopic(Utils.getUniqueString("testTopic")); + PubSubTopic nonExistentTopic = pubSubTopicRepository.getTopic(Utils.getUniqueString("nonExistentTopic")); + assertFalse(topicManager.containsTopic(testTopic)); + assertFalse(topicManager.containsTopic(nonExistentTopic)); + topicManager.createTopic(testTopic, numPartitions, replicationFactor, isEternalTopic); + waitForNonDeterministicAssertion(1, TimeUnit.MINUTES, () -> topicManager.containsTopic(testTopic)); + + int numMessages = 250; + PubSubProducerAdapter pubSubProducerAdapter = pubSubProducerAdapterLazy.get(); + CompletableFuture lastMessageFuture = null; + // list of messages + Map messages = new HashMap<>(numMessages); + for (int i = 0; i < numMessages; i++) { + MutablePubSubMessage message = PubSubHelper.getDummyPubSubMessage(false); + message.getValue().getProducerMetadata().setMessageTimestamp(i); + messages.put(i, message); + lastMessageFuture = + pubSubProducerAdapter.sendMessage(testTopic.getName(), 0, message.getKey(), message.getValue(), null, null); + lastMessageFuture.whenComplete((result, throwable) -> { + if (throwable == null) { + message.setOffset(result.getOffset()); + } + }); + } + assertNotNull(lastMessageFuture, "Last message future should not be null"); + lastMessageFuture.get(1, TimeUnit.MINUTES); + assertEquals(messages.size(), numMessages); + + final AtomicInteger successfulRequests = new AtomicInteger(0); + List tasks = new ArrayList<>(); + + Runnable getPartitionCountTask = () -> { + try { + int actualNumPartitions = topicManager.getPartitionCount(testTopic); + assertEquals(actualNumPartitions, numPartitions); + successfulRequests.incrementAndGet(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }; + tasks.add(getPartitionCountTask); + + // get partition count for non-existent topic + Runnable getPartitionCountForNonExistentTopicTask = () -> { + try { + assertNull(topicManager.getPartitionCount(nonExistentTopic)); + successfulRequests.incrementAndGet(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }; + tasks.add(getPartitionCountForNonExistentTopicTask); + + // contains topic + Runnable containsTopicTask = () -> { + try { + assertTrue(topicManager.containsTopic(testTopic)); + successfulRequests.incrementAndGet(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }; + tasks.add(containsTopicTask); + + // contains topic for non-existent topic + Runnable containsNonExistentTopicTask = () -> { + try { + assertFalse(topicManager.containsTopic(nonExistentTopic)); + successfulRequests.incrementAndGet(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }; + tasks.add(containsNonExistentTopicTask); + + Runnable getLatestOffsetWithRetriesTask = () -> { + try { + long latestOffset = topicManager.getLatestOffsetWithRetries(new PubSubTopicPartitionImpl(testTopic, 0), 1); + assertEquals(latestOffset, numMessages); + successfulRequests.incrementAndGet(); + } catch (Exception e) { + throw new RuntimeException(e); + } + }; + tasks.add(getLatestOffsetWithRetriesTask); + + ExecutorService executorService = Executors.newFixedThreadPool(8); + + List vwFutures = new ArrayList<>(); + + int totalTasks = 1024; + for (int i = 0; i < totalTasks; i++) { + Future future = executorService.submit(tasks.get(i % tasks.size())); + vwFutures.add(future); + } + + int failedRequests = 0; + for (Future future: vwFutures) { + try { + future.get(1, TimeUnit.MINUTES); + } catch (Exception e) { + failedRequests++; + } + } + System.out.println("successfulRequests: " + successfulRequests.get()); + // total should be equal to the number of tasks + assertEquals(successfulRequests.get() + failedRequests, totalTasks); + } + + @Test(timeOut = 3 * Time.MS_PER_MINUTE) + public void testMetadataApisForNonExistentTopics() throws ExecutionException, InterruptedException, TimeoutException { + PubSubTopic nonExistentTopic = pubSubTopicRepository.getTopic(Utils.getUniqueString("nonExistentTopic")); + assertFalse(topicManager.containsTopicCached(nonExistentTopic)); + assertFalse(topicManager.containsTopic(nonExistentTopic)); + Map nonExistentTopicLatestOffsets = topicManager.getTopicLatestOffsets(nonExistentTopic); + assertNotNull(nonExistentTopicLatestOffsets); + assertEquals(nonExistentTopicLatestOffsets.size(), 0); + assertThrows(PubSubTopicDoesNotExistException.class, () -> topicManager.getPartitionCount(nonExistentTopic)); + PubSubTopicPartitionImpl nonExistentTopicPartition = new PubSubTopicPartitionImpl(nonExistentTopic, 0); + assertThrows( + PubSubTopicDoesNotExistException.class, + () -> topicManager.getOffsetByTime(nonExistentTopicPartition, System.currentTimeMillis())); + assertThrows( + PubSubTopicDoesNotExistException.class, + () -> topicManager.getProducerTimestampOfLastDataMessageWithRetries(nonExistentTopicPartition, 1)); + assertEquals( + topicManager.getProducerTimestampOfLastDataMessageCached(nonExistentTopicPartition), + StatsErrorCode.LAG_MEASUREMENT_FAILURE.code); + topicManager.invalidateCache(nonExistentTopic).get(1, TimeUnit.MINUTES); // should not throw an exception + assertThrows( + PubSubTopicDoesNotExistException.class, + () -> topicManager.getLatestOffsetWithRetries(new PubSubTopicPartitionImpl(nonExistentTopic, 0), 1)); + assertEquals(topicManager.getLatestOffsetCached(nonExistentTopic, 1), StatsErrorCode.LAG_MEASUREMENT_FAILURE.code); + } + + @Test(timeOut = 3 * Time.MS_PER_MINUTE) + public void testMetadataApisForExistingTopics() throws ExecutionException, InterruptedException, TimeoutException { + int numPartitions = 35; + int replicationFactor = 1; + boolean isEternalTopic = true; + PubSubTopic existingTopic = pubSubTopicRepository.getTopic(Utils.getUniqueString("existingTopic")); + + // topic should not exist + assertFalse(topicManager.containsTopic(existingTopic)); + assertFalse(topicManager.containsTopicCached(existingTopic)); + + // create the topic + topicManager.createTopic(existingTopic, numPartitions, replicationFactor, isEternalTopic); + + // topic should exist + waitForNonDeterministicAssertion(1, TimeUnit.MINUTES, () -> { + assertTrue(topicManager.containsTopicCached(existingTopic)); + assertTrue(topicManager.containsTopic(existingTopic)); + }); + + // when there are no messages, the latest offset should be 0 + Map latestOffsets = topicManager.getTopicLatestOffsets(existingTopic); + assertNotNull(latestOffsets); + assertEquals(latestOffsets.size(), numPartitions); + for (int i = 0; i < numPartitions; i++) { + assertEquals((long) latestOffsets.get(i), 0L); + } + assertEquals(topicManager.getPartitionCount(existingTopic), numPartitions); + + PubSubTopicPartition p0 = new PubSubTopicPartitionImpl(existingTopic, 0); + PubSubTopicPartition p1 = new PubSubTopicPartitionImpl(existingTopic, 1); + PubSubTopicPartition p2 = new PubSubTopicPartitionImpl(existingTopic, 2); + PubSubTopicPartition p3 = new PubSubTopicPartitionImpl(existingTopic, 3); + + // produce messages to the topic-partitions: p0, p1, p2 + PubSubProducerAdapter pubSubProducerAdapter = pubSubProducerAdapterLazy.get(); + List p0Messages = PubSubHelper.produceMessages(pubSubProducerAdapter, p0, 10, 2, false); + List p1Messages = PubSubHelper.produceMessages(pubSubProducerAdapter, p1, 14, 2, false); + List p2Messages = PubSubHelper.produceMessages(pubSubProducerAdapter, p2, 19, 2, false); + + // get the latest offsets + latestOffsets = topicManager.getTopicLatestOffsets(existingTopic); + assertNotNull(latestOffsets); + assertEquals(latestOffsets.size(), numPartitions); + assertEquals((long) latestOffsets.get(0), p0Messages.size()); + assertEquals(topicManager.getLatestOffsetWithRetries(p0, 5), p0Messages.size()); + assertEquals(topicManager.getLatestOffsetCached(p0.getPubSubTopic(), 0), p0Messages.size()); + + assertEquals((long) latestOffsets.get(1), p1Messages.size()); + assertEquals(topicManager.getLatestOffsetWithRetries(p1, 5), p1Messages.size()); + assertEquals(topicManager.getLatestOffsetCached(p1.getPubSubTopic(), 1), p1Messages.size()); + + assertEquals((long) latestOffsets.get(2), p2Messages.size()); + assertEquals(topicManager.getLatestOffsetWithRetries(p2, 5), p2Messages.size()); + assertEquals(topicManager.getLatestOffsetCached(p2.getPubSubTopic(), 2), p2Messages.size()); + + // except for the first 3 partitions, the latest offset should be 0 + for (int i = 3; i < numPartitions; i++) { + assertEquals((long) latestOffsets.get(i), 0L); + assertEquals(topicManager.getLatestOffsetWithRetries(new PubSubTopicPartitionImpl(existingTopic, i), 5), 0L); + assertEquals(topicManager.getLatestOffsetCached(existingTopic, i), 0L); + } + + // if timestamp is greater than the latest message timestamp, the offset returned should be the latest offset + long timestamp = System.currentTimeMillis(); + assertEquals(topicManager.getOffsetByTime(p0, timestamp), p0Messages.size()); + + // If the provided timestamp is less than or equal to the timestamp of a message, + // the offset returned should correspond to that message. + long p0M4Ts = p0Messages.get(4).getTimestampAfterProduce(); + assertEquals(topicManager.getOffsetByTime(p0, p0M4Ts), 5); + + long p0TsBeforeM0 = p0Messages.get(0).getTimestampBeforeProduce(); + assertEquals(topicManager.getOffsetByTime(p0, p0TsBeforeM0), 0); + + // test getProducerTimestampOfLastDataMessage + long p0LastDataMessageTs = + p0Messages.get(p0Messages.size() - 1).getValue().getProducerMetadata().getMessageTimestamp(); + long p1LastDataMessageTs = + p1Messages.get(p1Messages.size() - 1).getValue().getProducerMetadata().getMessageTimestamp(); + long p2LastDataMessageTs = + p2Messages.get(p2Messages.size() - 1).getValue().getProducerMetadata().getMessageTimestamp(); + long p3LastDataMessageTs = PubSubConstants.PUBSUB_NO_PRODUCER_TIME_IN_EMPTY_TOPIC_PARTITION; + assertEquals(topicManager.getProducerTimestampOfLastDataMessageWithRetries(p0, 5), p0LastDataMessageTs); + assertEquals(topicManager.getProducerTimestampOfLastDataMessageWithRetries(p1, 5), p1LastDataMessageTs); + assertEquals(topicManager.getProducerTimestampOfLastDataMessageWithRetries(p2, 5), p2LastDataMessageTs); + assertEquals(topicManager.getProducerTimestampOfLastDataMessageWithRetries(p3, 5), p3LastDataMessageTs); + PubSubHelper.produceMessages(pubSubProducerAdapter, p0, 5, 1, true); + PubSubHelper.produceMessages(pubSubProducerAdapter, p1, 13, 1, true); + PubSubHelper.produceMessages(pubSubProducerAdapter, p2, 21, 1, true); + PubSubHelper.produceMessages(pubSubProducerAdapter, p3, 25, 1, true); + assertEquals(topicManager.getProducerTimestampOfLastDataMessageWithRetries(p0, 5), p0LastDataMessageTs); + assertEquals(topicManager.getProducerTimestampOfLastDataMessageCached(p0), p0LastDataMessageTs); + assertEquals(topicManager.getProducerTimestampOfLastDataMessageWithRetries(p1, 5), p1LastDataMessageTs); + assertEquals(topicManager.getProducerTimestampOfLastDataMessageCached(p1), p1LastDataMessageTs); + assertEquals(topicManager.getProducerTimestampOfLastDataMessageWithRetries(p2, 5), p2LastDataMessageTs); + assertEquals(topicManager.getProducerTimestampOfLastDataMessageCached(p2), p2LastDataMessageTs); + Throwable exception = + expectThrows(VeniceException.class, () -> topicManager.getProducerTimestampOfLastDataMessageWithRetries(p3, 5)); + assertTrue(exception.getMessage().contains("No data message found in topic-partition: ")); + Throwable exception2 = + expectThrows(VeniceException.class, () -> topicManager.getProducerTimestampOfLastDataMessageCached(p3)); + assertTrue(exception2.getMessage().contains("No data message found in topic-partition: ")); + } + + @Test(timeOut = 3 * Time.MS_PER_MINUTE) + public void testClose() throws InterruptedException { + PubSubTopic nonExistentTopic = pubSubTopicRepository.getTopic(Utils.getUniqueString("nonExistentTopic")); + assertFalse(topicManager.containsTopic(nonExistentTopic)); + CountDownLatch latch = new CountDownLatch(1); + Runnable[] tasks = { () -> { + latch.countDown(); + topicManager.getLatestOffsetCached(nonExistentTopic, 1); + }, () -> { + latch.countDown(); + topicManager.getLatestOffsetWithRetries(new PubSubTopicPartitionImpl(nonExistentTopic, 0), 1); + } }; + + ExecutorService executorService = Executors.newFixedThreadPool(5); + for (int i = 0; i < 20; i++) { + executorService.submit(tasks[i % tasks.length]); + } + latch.await(); + Thread.sleep(100); + topicManager.close(); + // call close again and it should not throw an exception + topicManager.close(); + } + +} diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/kafka/TopicManagerIntegrationTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/pubsub/manager/TopicManagerIntegrationTest.java similarity index 88% rename from internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/kafka/TopicManagerIntegrationTest.java rename to internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/pubsub/manager/TopicManagerIntegrationTest.java index 511525cd7b..8c15526a6c 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/kafka/TopicManagerIntegrationTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/pubsub/manager/TopicManagerIntegrationTest.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.kafka; +package com.linkedin.venice.pubsub.manager; import static com.linkedin.venice.integration.utils.VeniceClusterWrapperConstants.STANDALONE_REGION_NAME; @@ -38,7 +38,7 @@ protected void createTopicManager() { new PubSubBrokerConfigs.Builder().setMockTime(mockTime).setRegionName(STANDALONE_REGION_NAME).build()); topicManager = IntegrationTestPushUtils .getTopicManagerRepo(500L, 100L, MIN_COMPACTION_LAG, pubSubBrokerWrapper, new PubSubTopicRepository()) - .getTopicManager(); + .getLocalTopicManager(); } protected PubSubProducerAdapter createPubSubProducerAdapter() { @@ -56,7 +56,7 @@ public void testRaceCondition() throws ExecutionException, InterruptedException produceRandomPubSubMessage(topic, true, timestamp); // This timestamp is expected to be retrieved produceRandomPubSubMessage(topic, false, timestamp + 1000L); // produce a control message - long retrievedTimestamp = topicManager.getProducerTimestampOfLastDataRecord(pubSubTopicPartition, 1); + long retrievedTimestamp = topicManager.getProducerTimestampOfLastDataMessageWithRetries(pubSubTopicPartition, 1); Assert.assertEquals(retrievedTimestamp, timestamp); // Produce more data records to this topic partition @@ -73,10 +73,10 @@ public void testRaceCondition() throws ExecutionException, InterruptedException ExecutorService executorService = Executors.newFixedThreadPool(numberOfThreads); Future[] vwFutures = new Future[numberOfThreads]; // Put all topic manager calls related to partition offset fetcher with admin and consumer here. - Runnable[] tasks = { () -> topicManager.getPartitionOffsetByTime(pubSubTopicPartition, checkTimestamp), - () -> topicManager.getProducerTimestampOfLastDataRecord(pubSubTopicPartition, 1), - () -> topicManager.partitionsFor(topic), - () -> topicManager.getPartitionLatestOffsetAndRetry(pubSubTopicPartition, 1), + Runnable[] tasks = { () -> topicManager.getOffsetByTime(pubSubTopicPartition, checkTimestamp), + () -> topicManager.getProducerTimestampOfLastDataMessageWithRetries(pubSubTopicPartition, 1), + () -> topicManager.getPartitionCount(topic), + () -> topicManager.getLatestOffsetWithRetries(pubSubTopicPartition, 1), () -> topicManager.getTopicLatestOffsets(topic) }; for (int i = 0; i < numberOfThreads; i++) { diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/utils/IntegrationTestPushUtils.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/utils/IntegrationTestPushUtils.java index 636db0402b..4f16f8827b 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/utils/IntegrationTestPushUtils.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/utils/IntegrationTestPushUtils.java @@ -37,11 +37,12 @@ import com.linkedin.venice.integration.utils.VeniceControllerWrapper; import com.linkedin.venice.integration.utils.VeniceMultiClusterWrapper; import com.linkedin.venice.integration.utils.VeniceTwoLayerMultiRegionMultiClusterWrapper; -import com.linkedin.venice.kafka.TopicManagerRepository; import com.linkedin.venice.meta.Store; import com.linkedin.venice.meta.Version; import com.linkedin.venice.pubsub.PubSubProducerAdapterFactory; import com.linkedin.venice.pubsub.PubSubTopicRepository; +import com.linkedin.venice.pubsub.manager.TopicManagerContext; +import com.linkedin.venice.pubsub.manager.TopicManagerRepository; import com.linkedin.venice.samza.VeniceObjectWithTimestamp; import com.linkedin.venice.samza.VeniceSystemFactory; import com.linkedin.venice.writer.VeniceWriterFactory; @@ -374,16 +375,18 @@ public static TopicManagerRepository getTopicManagerRepo( String pubSubBootstrapServers = pubSubBrokerWrapper.getAddress(); properties.putAll(PubSubBrokerWrapper.getBrokerDetailsForClients(Collections.singletonList(pubSubBrokerWrapper))); properties.put(KAFKA_BOOTSTRAP_SERVERS, pubSubBootstrapServers); - return TopicManagerRepository.builder() - .setPubSubProperties(k -> new VeniceProperties(properties)) - .setPubSubTopicRepository(pubSubTopicRepository) - .setLocalKafkaBootstrapServers(pubSubBootstrapServers) - .setPubSubConsumerAdapterFactory(pubSubBrokerWrapper.getPubSubClientsFactory().getConsumerAdapterFactory()) - .setPubSubAdminAdapterFactory(pubSubBrokerWrapper.getPubSubClientsFactory().getAdminAdapterFactory()) - .setKafkaOperationTimeoutMs(kafkaOperationTimeoutMs) - .setTopicDeletionStatusPollIntervalMs(topicDeletionStatusPollIntervalMs) - .setTopicMinLogCompactionLagMs(topicMinLogCompactionLagMs) - .build(); + + TopicManagerContext topicManagerContext = + new TopicManagerContext.Builder().setPubSubPropertiesSupplier(k -> new VeniceProperties(properties)) + .setPubSubTopicRepository(pubSubTopicRepository) + .setPubSubConsumerAdapterFactory(pubSubBrokerWrapper.getPubSubClientsFactory().getConsumerAdapterFactory()) + .setPubSubAdminAdapterFactory(pubSubBrokerWrapper.getPubSubClientsFactory().getAdminAdapterFactory()) + .setPubSubOperationTimeoutMs(kafkaOperationTimeoutMs) + .setTopicDeletionStatusPollIntervalMs(topicDeletionStatusPollIntervalMs) + .setTopicMinLogCompactionLagMs(topicMinLogCompactionLagMs) + .build(); + + return new TopicManagerRepository(topicManagerContext, pubSubBootstrapServers); } public static VeniceWriterFactory getVeniceWriterFactory( diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/writer/PubSubSharedProducerAdapterFactoryTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/writer/PubSubSharedProducerAdapterFactoryTest.java index a9ecd959ac..e83ffaf238 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/writer/PubSubSharedProducerAdapterFactoryTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/writer/PubSubSharedProducerAdapterFactoryTest.java @@ -1,7 +1,7 @@ package com.linkedin.venice.writer; import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_OPERATION_TIMEOUT_MS; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE; import static com.linkedin.venice.pubsub.adapter.kafka.producer.ApacheKafkaProducerConfig.KAFKA_BUFFER_MEMORY; import static org.mockito.Mockito.mock; @@ -9,7 +9,6 @@ import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.integration.utils.PubSubBrokerWrapper; import com.linkedin.venice.integration.utils.ServiceFactory; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; import com.linkedin.venice.kafka.protocol.enums.MessageType; import com.linkedin.venice.message.KafkaKey; @@ -25,6 +24,7 @@ import com.linkedin.venice.pubsub.api.PubSubProducerAdapter; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.serialization.avro.KafkaValueSerializer; import com.linkedin.venice.utils.IntegrationTestPushUtils; import com.linkedin.venice.utils.TestUtils; @@ -55,9 +55,15 @@ public class PubSubSharedProducerAdapterFactoryTest { public void setUp() { pubSubBrokerWrapper = ServiceFactory.getPubSubBroker(); pubSubConsumerAdapterFactory = pubSubBrokerWrapper.getPubSubClientsFactory().getConsumerAdapterFactory(); - topicManager = IntegrationTestPushUtils - .getTopicManagerRepo(DEFAULT_KAFKA_OPERATION_TIMEOUT_MS, 100, 0L, pubSubBrokerWrapper, pubSubTopicRepository) - .getTopicManager(); + topicManager = + IntegrationTestPushUtils + .getTopicManagerRepo( + PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE, + 100, + 0L, + pubSubBrokerWrapper, + pubSubTopicRepository) + .getLocalTopicManager(); } @AfterClass diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/writer/VeniceWriterTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/writer/VeniceWriterTest.java index e1ede61ebc..90cccaeffe 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/writer/VeniceWriterTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/writer/VeniceWriterTest.java @@ -1,6 +1,6 @@ package com.linkedin.venice.writer; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_OPERATION_TIMEOUT_MS; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE; import static com.linkedin.venice.utils.Time.MS_PER_SECOND; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; @@ -10,7 +10,6 @@ import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.integration.utils.PubSubBrokerWrapper; import com.linkedin.venice.integration.utils.ServiceFactory; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; import com.linkedin.venice.kafka.protocol.ProducerMetadata; import com.linkedin.venice.kafka.protocol.enums.MessageType; @@ -25,6 +24,7 @@ import com.linkedin.venice.pubsub.api.PubSubMessageDeserializer; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.serialization.avro.KafkaValueSerializer; import com.linkedin.venice.serialization.avro.OptimizedKafkaValueSerializer; import com.linkedin.venice.utils.DataProviderUtils; @@ -69,9 +69,15 @@ public void setUp() { pubSubBrokerWrapper = ServiceFactory.getPubSubBroker(); pubSubConsumerAdapterFactory = pubSubBrokerWrapper.getPubSubClientsFactory().getConsumerAdapterFactory(); pubSubProducerAdapterFactory = pubSubBrokerWrapper.getPubSubClientsFactory().getProducerAdapterFactory(); - topicManager = IntegrationTestPushUtils - .getTopicManagerRepo(DEFAULT_KAFKA_OPERATION_TIMEOUT_MS, 100L, 0L, pubSubBrokerWrapper, pubSubTopicRepository) - .getTopicManager(); + topicManager = + IntegrationTestPushUtils + .getTopicManagerRepo( + PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE, + 100L, + 0L, + pubSubBrokerWrapper, + pubSubTopicRepository) + .getLocalTopicManager(); } @AfterClass diff --git a/internal/venice-test-common/src/main/java/com/linkedin/venice/unit/kafka/MockInMemoryAdminAdapter.java b/internal/venice-test-common/src/main/java/com/linkedin/venice/unit/kafka/MockInMemoryAdminAdapter.java index a40564e22d..a3b9ad5651 100644 --- a/internal/venice-test-common/src/main/java/com/linkedin/venice/unit/kafka/MockInMemoryAdminAdapter.java +++ b/internal/venice-test-common/src/main/java/com/linkedin/venice/unit/kafka/MockInMemoryAdminAdapter.java @@ -85,7 +85,7 @@ public Map getAllTopicRetentions() { if (retentionMs.isPresent()) { retentions.put(entry.getKey(), retentionMs.get()); } else { - retentions.put(entry.getKey(), PubSubConstants.UNKNOWN_TOPIC_RETENTION); + retentions.put(entry.getKey(), PubSubConstants.PUBSUB_TOPIC_UNKNOWN_RETENTION); } } return retentions; diff --git a/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/PubSubHelper.java b/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/PubSubHelper.java index f3e5cc011c..eeab8882e1 100644 --- a/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/PubSubHelper.java +++ b/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/PubSubHelper.java @@ -6,11 +6,25 @@ import com.linkedin.venice.kafka.protocol.Put; import com.linkedin.venice.kafka.protocol.enums.MessageType; import com.linkedin.venice.message.KafkaKey; +import com.linkedin.venice.pubsub.api.PubSubProducerAdapter; +import com.linkedin.venice.pubsub.api.PubSubTopicPartition; import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; public class PubSubHelper { public static KafkaKey getDummyKey() { + return getDummyKey(false); + } + + public static KafkaKey getDummyKey(boolean isControlMessage) { + if (isControlMessage) { + return new KafkaKey(MessageType.CONTROL_MESSAGE, Utils.getUniqueString("dummyCmKey").getBytes()); + } return new KafkaKey(MessageType.PUT, Utils.getUniqueString("dummyKey").getBytes()); } @@ -31,4 +45,110 @@ public static KafkaMessageEnvelope getDummyValue(long producerMessageTimestamp) value.payloadUnion = put; return value; } + + public static MutablePubSubMessage getDummyPubSubMessage(boolean isControlMessage) { + return new MutablePubSubMessage().setKey(getDummyKey(isControlMessage)).setValue(getDummyValue()); + } + + public static List produceMessages( + PubSubProducerAdapter pubSubProducerAdapter, + PubSubTopicPartition topicPartition, + int messageCount, + long delayBetweenMessagesInMs, + boolean controlMessages) throws InterruptedException, ExecutionException, TimeoutException { + List messages = new ArrayList<>(messageCount); + for (int i = 0; i < messageCount; i++) { + MutablePubSubMessage message = PubSubHelper.getDummyPubSubMessage(controlMessages); + message.getValue().getProducerMetadata().setMessageTimestamp(i); // logical ts + message.setTimestampBeforeProduce(System.currentTimeMillis()); + messages.add(message); + pubSubProducerAdapter + .sendMessage( + topicPartition.getTopicName(), + topicPartition.getPartitionNumber(), + message.getKey(), + message.getValue(), + null, + null) + .whenComplete((result, throwable) -> { + if (throwable == null) { + message.setOffset(result.getOffset()); + message.setTimestampAfterProduce(System.currentTimeMillis()); + try { + TimeUnit.MILLISECONDS.sleep(delayBetweenMessagesInMs); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + }) + .get(10, TimeUnit.SECONDS); + TimeUnit.MILLISECONDS.sleep(delayBetweenMessagesInMs); + } + return messages; + } + + // mutable publish-sub message + public static class MutablePubSubMessage { + private KafkaKey key; + private KafkaMessageEnvelope value; + private PubSubTopicPartition topicPartition; + private long offset; + private long timestampBeforeProduce; + private long timestampAfterProduce; + + public KafkaKey getKey() { + return key; + } + + public KafkaMessageEnvelope getValue() { + return value; + } + + public PubSubTopicPartition getTopicPartition() { + return topicPartition; + } + + public Long getOffset() { + return offset; + } + + public MutablePubSubMessage setKey(KafkaKey key) { + this.key = key; + return this; + } + + public MutablePubSubMessage setValue(KafkaMessageEnvelope value) { + this.value = value; + return this; + } + + public MutablePubSubMessage setTopicPartition(PubSubTopicPartition topicPartition) { + this.topicPartition = topicPartition; + return this; + } + + public MutablePubSubMessage setOffset(long offset) { + this.offset = offset; + return this; + } + + public MutablePubSubMessage setTimestampBeforeProduce(long timestampBeforeProduce) { + this.timestampBeforeProduce = timestampBeforeProduce; + return this; + } + + public MutablePubSubMessage setTimestampAfterProduce(long timestampAfterProduce) { + this.timestampAfterProduce = timestampAfterProduce; + return this; + } + + public long getTimestampBeforeProduce() { + return timestampBeforeProduce; + } + + public long getTimestampAfterProduce() { + return timestampAfterProduce; + } + } } diff --git a/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/TestUtils.java b/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/TestUtils.java index bdcccdf9ce..86c02386d0 100644 --- a/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/TestUtils.java +++ b/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/TestUtils.java @@ -45,7 +45,6 @@ import com.linkedin.venice.helix.HelixReadOnlySchemaRepository; import com.linkedin.venice.helix.SafeHelixManager; import com.linkedin.venice.helix.VeniceOfflinePushMonitorAccessor; -import com.linkedin.venice.kafka.TopicManagerRepository; import com.linkedin.venice.kafka.protocol.state.PartitionState; import com.linkedin.venice.meta.IngestionMode; import com.linkedin.venice.meta.Instance; @@ -68,6 +67,7 @@ import com.linkedin.venice.pubsub.adapter.kafka.producer.ApacheKafkaProducerAdapterFactory; import com.linkedin.venice.pubsub.adapter.kafka.producer.SharedKafkaProducerAdapterFactory; import com.linkedin.venice.pubsub.api.PubSubTopicType; +import com.linkedin.venice.pubsub.manager.TopicManagerRepository; import com.linkedin.venice.pushmonitor.ExecutionStatus; import com.linkedin.venice.serialization.avro.AvroProtocolDefinition; import com.linkedin.venice.serialization.avro.InternalAvroSpecificSerializer; diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/Admin.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/Admin.java index d99aaeb25d..3ce76ba524 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/Admin.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/Admin.java @@ -13,7 +13,6 @@ import com.linkedin.venice.helix.HelixReadOnlyZKSharedSchemaRepository; import com.linkedin.venice.helix.HelixReadOnlyZKSharedSystemStoreRepository; import com.linkedin.venice.helix.Replica; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.meta.Instance; import com.linkedin.venice.meta.RegionPushDetails; import com.linkedin.venice.meta.RoutersClusterConfig; @@ -26,6 +25,7 @@ import com.linkedin.venice.meta.Version; import com.linkedin.venice.persona.StoragePersona; import com.linkedin.venice.pubsub.PubSubConsumerAdapterFactory; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.pushmonitor.ExecutionStatus; import com.linkedin.venice.pushstatushelper.PushStatusStoreReader; import com.linkedin.venice.pushstatushelper.PushStatusStoreWriter; @@ -635,7 +635,7 @@ void updateRoutersClusterConfig( VeniceWriterFactory getVeniceWriterFactory(); - PubSubConsumerAdapterFactory getVeniceConsumerFactory(); + PubSubConsumerAdapterFactory getPubSubConsumerAdapterFactory(); VeniceProperties getPubSubSSLProperties(String pubSubBrokerAddress); diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java index e2e3dee701..31666db60f 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java @@ -68,8 +68,8 @@ import static com.linkedin.venice.SSLConfig.DEFAULT_CONTROLLER_SSL_ENABLED; import static com.linkedin.venice.VeniceConstants.DEFAULT_PER_ROUTER_READ_QUOTA; import static com.linkedin.venice.VeniceConstants.DEFAULT_SSL_FACTORY_CLASS_NAME; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_MIN_LOG_COMPACTION_LAG_MS; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_REPLICATION_FACTOR; +import static com.linkedin.venice.pubsub.PubSubConstants.DEFAULT_KAFKA_MIN_LOG_COMPACTION_LAG_MS; +import static com.linkedin.venice.pubsub.PubSubConstants.DEFAULT_KAFKA_REPLICATION_FACTOR; import com.linkedin.venice.SSLConfig; import com.linkedin.venice.exceptions.ConfigurationException; diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerConfig.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerConfig.java index b0af8fc2fc..5e0b42705c 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerConfig.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerConfig.java @@ -1,6 +1,5 @@ package com.linkedin.venice.controller; -import static com.linkedin.venice.ConfigConstants.DEFAULT_TOPIC_DELETION_STATUS_POLL_INTERVAL_MS; import static com.linkedin.venice.ConfigKeys.ACTIVE_ACTIVE_ENABLED_ON_CONTROLLER; import static com.linkedin.venice.ConfigKeys.ACTIVE_ACTIVE_REAL_TIME_SOURCE_FABRIC_LIST; import static com.linkedin.venice.ConfigKeys.ADMIN_CHECK_READ_METHOD_FOR_KAFKA; @@ -82,6 +81,8 @@ import static com.linkedin.venice.ConfigKeys.PARENT_CONTROLLER_WAITING_TIME_FOR_CONSUMPTION_MS; import static com.linkedin.venice.ConfigKeys.PARENT_KAFKA_CLUSTER_FABRIC_LIST; import static com.linkedin.venice.ConfigKeys.PARTICIPANT_MESSAGE_STORE_ENABLED; +import static com.linkedin.venice.ConfigKeys.PUBSUB_TOPIC_MANAGER_METADATA_FETCHER_CONSUMER_POOL_SIZE; +import static com.linkedin.venice.ConfigKeys.PUBSUB_TOPIC_MANAGER_METADATA_FETCHER_THREAD_POOL_SIZE; import static com.linkedin.venice.ConfigKeys.PUB_SUB_ADMIN_ADAPTER_FACTORY_CLASS; import static com.linkedin.venice.ConfigKeys.PUB_SUB_CONSUMER_ADAPTER_FACTORY_CLASS; import static com.linkedin.venice.ConfigKeys.PUB_SUB_PRODUCER_ADAPTER_FACTORY_CLASS; @@ -100,6 +101,8 @@ import static com.linkedin.venice.ConfigKeys.UNREGISTER_METRIC_FOR_DELETED_STORE_ENABLED; import static com.linkedin.venice.ConfigKeys.USE_PUSH_STATUS_STORE_FOR_INCREMENTAL_PUSH; import static com.linkedin.venice.ConfigKeys.VENICE_STORAGE_CLUSTER_LEADER_HAAS; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_TOPIC_DELETION_STATUS_POLL_INTERVAL_MS_DEFAULT_VALUE; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_TOPIC_MANAGER_METADATA_FETCHER_CONSUMER_POOL_SIZE_DEFAULT_VALUE; import com.linkedin.venice.authorization.DefaultIdentityParser; import com.linkedin.venice.client.store.ClientConfig; @@ -177,6 +180,8 @@ public class VeniceControllerConfig extends VeniceControllerClusterConfig { private final int topicCleanupDelayFactor; private final int topicManagerKafkaOperationTimeOutMs; + private final int topicManagerMetadataFetcherConsumerPoolSize; + private final int topicManagerMetadataFetcherThreadPoolSize; private final int minNumberOfUnusedKafkaTopicsToPreserve; private final int minNumberOfStoreVersionsToPreserve; private final int parentControllerMaxErroredTopicNumToKeep; @@ -423,6 +428,11 @@ public VeniceControllerConfig(VeniceProperties props) { props.getLong(CONTROLLER_DISABLED_REPLICA_ENABLER_INTERVAL_MS, TimeUnit.HOURS.toMillis(16)); this.topicManagerKafkaOperationTimeOutMs = props.getInt(TOPIC_MANAGER_KAFKA_OPERATION_TIMEOUT_MS, 30 * Time.MS_PER_SECOND); + this.topicManagerMetadataFetcherConsumerPoolSize = props.getInt( + PUBSUB_TOPIC_MANAGER_METADATA_FETCHER_CONSUMER_POOL_SIZE, + PUBSUB_TOPIC_MANAGER_METADATA_FETCHER_CONSUMER_POOL_SIZE_DEFAULT_VALUE); + this.topicManagerMetadataFetcherThreadPoolSize = props + .getInt(PUBSUB_TOPIC_MANAGER_METADATA_FETCHER_THREAD_POOL_SIZE, topicManagerMetadataFetcherConsumerPoolSize); this.minNumberOfUnusedKafkaTopicsToPreserve = props.getInt(MIN_NUMBER_OF_UNUSED_KAFKA_TOPICS_TO_PRESERVE, 2); this.minNumberOfStoreVersionsToPreserve = props.getInt(MIN_NUMBER_OF_STORE_VERSIONS_TO_PRESERVE, 2); @@ -443,8 +453,8 @@ public VeniceControllerConfig(VeniceProperties props) { } this.systemSchemaClusterName = props.getString(CONTROLLER_SYSTEM_SCHEMA_CLUSTER_NAME, ""); this.earlyDeleteBackUpEnabled = props.getBoolean(CONTROLLER_EARLY_DELETE_BACKUP_ENABLED, true); - this.topicDeletionStatusPollIntervalMs = - props.getInt(TOPIC_DELETION_STATUS_POLL_INTERVAL_MS, DEFAULT_TOPIC_DELETION_STATUS_POLL_INTERVAL_MS); // 2s + this.topicDeletionStatusPollIntervalMs = props + .getInt(TOPIC_DELETION_STATUS_POLL_INTERVAL_MS, PUBSUB_TOPIC_DELETION_STATUS_POLL_INTERVAL_MS_DEFAULT_VALUE); // 2s this.isControllerClusterLeaderHAAS = props.getBoolean(CONTROLLER_CLUSTER_LEADER_HAAS, false); this.isVeniceClusterLeaderHAAS = props.getBoolean(VENICE_STORAGE_CLUSTER_LEADER_HAAS, false); this.controllerHAASSuperClusterName = props.getString(CONTROLLER_HAAS_SUPER_CLUSTER_NAME, ""); @@ -747,6 +757,14 @@ public int getTopicManagerKafkaOperationTimeOutMs() { return topicManagerKafkaOperationTimeOutMs; } + public int getTopicManagerMetadataFetcherConsumerPoolSize() { + return topicManagerMetadataFetcherConsumerPoolSize; + } + + public int getTopicManagerMetadataFetcherThreadPoolSize() { + return topicManagerMetadataFetcherThreadPoolSize; + } + public int getMinNumberOfUnusedKafkaTopicsToPreserve() { return minNumberOfUnusedKafkaTopicsToPreserve; } diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceHelixAdmin.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceHelixAdmin.java index 72faef5f8e..a006a24b55 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceHelixAdmin.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceHelixAdmin.java @@ -1,6 +1,5 @@ package com.linkedin.venice.controller; -import static com.linkedin.venice.ConfigConstants.DEFAULT_TOPIC_DELETION_STATUS_POLL_INTERVAL_MS; import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; import static com.linkedin.venice.ConfigKeys.KAFKA_MIN_IN_SYNC_REPLICAS; import static com.linkedin.venice.ConfigKeys.KAFKA_OVER_SSL; @@ -8,8 +7,6 @@ import static com.linkedin.venice.ConfigKeys.SSL_KAFKA_BOOTSTRAP_SERVERS; import static com.linkedin.venice.ConfigKeys.SSL_TO_KAFKA_LEGACY; import static com.linkedin.venice.controller.UserSystemStoreLifeCycleHelper.AUTO_META_SYSTEM_STORE_PUSH_ID_PREFIX; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_MIN_LOG_COMPACTION_LAG_MS; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_OPERATION_TIMEOUT_MS; import static com.linkedin.venice.meta.HybridStoreConfigImpl.DEFAULT_HYBRID_OFFSET_LAG_THRESHOLD; import static com.linkedin.venice.meta.HybridStoreConfigImpl.DEFAULT_HYBRID_TIME_LAG_THRESHOLD; import static com.linkedin.venice.meta.HybridStoreConfigImpl.DEFAULT_REWIND_TIME_IN_SECONDS; @@ -104,8 +101,6 @@ import com.linkedin.venice.helix.ZkRoutersClusterManager; import com.linkedin.venice.helix.ZkStoreConfigAccessor; import com.linkedin.venice.ingestion.control.RealTimeTopicSwitcher; -import com.linkedin.venice.kafka.TopicManager; -import com.linkedin.venice.kafka.TopicManagerRepository; import com.linkedin.venice.kafka.protocol.enums.ControlMessageType; import com.linkedin.venice.meta.BackupStrategy; import com.linkedin.venice.meta.BufferReplayPolicy; @@ -156,6 +151,9 @@ import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.exceptions.PubSubOpTimeoutException; import com.linkedin.venice.pubsub.api.exceptions.PubSubTopicDoesNotExistException; +import com.linkedin.venice.pubsub.manager.TopicManager; +import com.linkedin.venice.pubsub.manager.TopicManagerContext; +import com.linkedin.venice.pubsub.manager.TopicManagerRepository; import com.linkedin.venice.pushmonitor.ExecutionStatus; import com.linkedin.venice.pushmonitor.ExecutionStatusWithDetails; import com.linkedin.venice.pushmonitor.KillOfflinePushMessage; @@ -202,6 +200,7 @@ import com.linkedin.venice.utils.RegionUtils; import com.linkedin.venice.utils.RetryUtils; import com.linkedin.venice.utils.SslUtils; +import com.linkedin.venice.utils.StoreUtils; import com.linkedin.venice.utils.Time; import com.linkedin.venice.utils.Utils; import com.linkedin.venice.utils.VeniceProperties; @@ -348,7 +347,7 @@ public class VeniceHelixAdmin implements Admin, StoreCleaner { private final long deprecatedJobTopicMaxRetentionMs; private final HelixReadOnlyStoreConfigRepository storeConfigRepo; private final VeniceWriterFactory veniceWriterFactory; - private final PubSubConsumerAdapterFactory veniceConsumerFactory; + private final PubSubConsumerAdapterFactory pubSubConsumerAdapterFactory; private final int minNumberOfStoreVersionsToPreserve; private final StoreGraveyard storeGraveyard; private final Map participantMessageStoreRTTMap; @@ -501,18 +500,19 @@ public VeniceHelixAdmin( this.zkClient.subscribeStateChanges(new ZkClientStatusStats(metricsRepository, "controller-zk-client")); this.adapterSerializer = new HelixAdapterSerializer(); - this.veniceConsumerFactory = pubSubClientsFactory.getConsumerAdapterFactory(); - this.topicManagerRepository = TopicManagerRepository.builder() - .setPubSubTopicRepository(pubSubTopicRepository) - .setMetricsRepository(metricsRepository) - .setLocalKafkaBootstrapServers(getKafkaBootstrapServers(isSslToKafka())) - .setTopicDeletionStatusPollIntervalMs(DEFAULT_TOPIC_DELETION_STATUS_POLL_INTERVAL_MS) - .setTopicMinLogCompactionLagMs(DEFAULT_KAFKA_MIN_LOG_COMPACTION_LAG_MS) - .setKafkaOperationTimeoutMs(DEFAULT_KAFKA_OPERATION_TIMEOUT_MS) - .setPubSubProperties(this::getPubSubSSLPropertiesFromControllerConfig) - .setPubSubAdminAdapterFactory(pubSubClientsFactory.getAdminAdapterFactory()) - .setPubSubConsumerAdapterFactory(veniceConsumerFactory) - .build(); + this.pubSubConsumerAdapterFactory = pubSubClientsFactory.getConsumerAdapterFactory(); + + TopicManagerContext topicManagerContext = + new TopicManagerContext.Builder().setPubSubTopicRepository(pubSubTopicRepository) + .setMetricsRepository(metricsRepository) + .setPubSubPropertiesSupplier(this::getPubSubSSLPropertiesFromControllerConfig) + .setPubSubAdminAdapterFactory(pubSubClientsFactory.getAdminAdapterFactory()) + .setPubSubConsumerAdapterFactory(pubSubConsumerAdapterFactory) + .setTopicMetadataFetcherConsumerPoolSize(commonConfig.getTopicManagerMetadataFetcherConsumerPoolSize()) + .setTopicMetadataFetcherThreadPoolSize(commonConfig.getTopicManagerMetadataFetcherThreadPoolSize()) + .build(); + this.topicManagerRepository = + new TopicManagerRepository(topicManagerContext, getKafkaBootstrapServers(isSslToKafka())); this.allowlistAccessor = new ZkAllowlistAccessor(zkClient, adapterSerializer); this.executionIdAccessor = new ZkExecutionIdAccessor(zkClient, adapterSerializer); @@ -528,7 +528,7 @@ public VeniceHelixAdmin( pubSubClientsFactory.getProducerAdapterFactory(), null); this.realTimeTopicSwitcher = new RealTimeTopicSwitcher( - topicManagerRepository.getTopicManager(), + topicManagerRepository.getLocalTopicManager(), veniceWriterFactory, commonConfig.getProps(), pubSubTopicRepository); @@ -551,7 +551,7 @@ public VeniceHelixAdmin( commonConfig.getRefreshAttemptsForZkReconnect(), commonConfig.getRefreshIntervalForZkReconnectInMs()); metaStoreWriter = new MetaStoreWriter( - topicManagerRepository.getTopicManager(), + topicManagerRepository.getLocalTopicManager(), veniceWriterFactory, zkSharedSchemaRepository, pubSubTopicRepository, @@ -2566,7 +2566,7 @@ private Pair addVersion( realTimeTopic, numberOfPartitions, clusterConfig.getKafkaReplicationFactorRTTopics(), - TopicManager.getExpectedRetentionTimeInMs(store, store.getHybridStoreConfig()), + StoreUtils.getExpectedRetentionTimeInMs(store, store.getHybridStoreConfig()), false, // Note: do not enable RT compaction! Might make jobs in Online/Offline model stuck clusterConfig.getMinInSyncReplicasRealTimeTopics(), @@ -2575,9 +2575,9 @@ private Pair addVersion( // If real-time topic already exists, check whether its retention time is correct. PubSubTopicConfiguration pubSubTopicConfiguration = getTopicManager().getCachedTopicConfig(realTimeTopic); - long topicRetentionTimeInMs = getTopicManager().getTopicRetention(pubSubTopicConfiguration); + long topicRetentionTimeInMs = TopicManager.getTopicRetention(pubSubTopicConfiguration); long expectedRetentionTimeMs = - TopicManager.getExpectedRetentionTimeInMs(store, store.getHybridStoreConfig()); + StoreUtils.getExpectedRetentionTimeInMs(store, store.getHybridStoreConfig()); if (topicRetentionTimeInMs != expectedRetentionTimeMs) { getTopicManager() .updateTopicRetention(realTimeTopic, expectedRetentionTimeMs, pubSubTopicConfiguration); @@ -3504,12 +3504,12 @@ private boolean truncateKafkaTopic(TopicManager topicManager, String kafkaTopicN LOGGER.info( "Topic {} does not exist in Kafka cluster {}, will skip the truncation", kafkaTopicName, - topicManager.getPubSubBootstrapServers()); + topicManager.getPubSubClusterAddress()); } catch (Exception e) { LOGGER.warn( "Unable to update the retention for topic {} in Kafka cluster {}, will skip the truncation", kafkaTopicName, - topicManager.getPubSubBootstrapServers(), + topicManager.getPubSubClusterAddress(), e); } return false; @@ -3850,7 +3850,7 @@ void preCheckStorePartitionCountUpdate(String clusterName, Store store, int newP } PubSubTopic realTimeTopic = pubSubTopicRepository.getTopic(Version.composeRealTimeTopic(store.getName())); if (topicManager.containsTopic(realTimeTopic) - && topicManager.partitionsFor(realTimeTopic).size() == newPartitionCount) { + && topicManager.getPartitionCount(realTimeTopic) == newPartitionCount) { LOGGER.info("Allow updating store " + store.getName() + " partition count to " + newPartitionCount); return; } @@ -4435,7 +4435,7 @@ private void internalUpdateStore(String clusterName, String storeName, UpdateSto if (getTopicManager().containsTopicAndAllPartitionsAreOnline(rtTopic)) { // RT already exists, ensure the retention is correct getTopicManager() - .updateTopicRetention(rtTopic, TopicManager.getExpectedRetentionTimeInMs(store, finalHybridConfig)); + .updateTopicRetention(rtTopic, StoreUtils.getExpectedRetentionTimeInMs(store, finalHybridConfig)); } } return store; @@ -4591,7 +4591,7 @@ && getTopicManager().containsTopicAndAllPartitionsAreOnline(rtTopic)) { // Ensure the topic retention is rolled back too getTopicManager().updateTopicRetention( rtTopic, - TopicManager.getExpectedRetentionTimeInMs(originalStore, originalStore.getHybridStoreConfig())); + StoreUtils.getExpectedRetentionTimeInMs(originalStore, originalStore.getHybridStoreConfig())); } LOGGER.info( "Successfully rolled back changes to store: {} in cluster: {}. Will now throw the original exception: {}.", @@ -6005,7 +6005,7 @@ TopicManagerRepository getTopicManagerRepository() { */ @Override public TopicManager getTopicManager() { - return this.topicManagerRepository.getTopicManager(); + return this.topicManagerRepository.getLocalTopicManager(); } /** @@ -6674,8 +6674,8 @@ public VeniceWriterFactory getVeniceWriterFactory() { * @return a PubSubClientFactory object used by the Venice controller to create Pubsub clients. */ @Override - public PubSubConsumerAdapterFactory getVeniceConsumerFactory() { - return veniceConsumerFactory; + public PubSubConsumerAdapterFactory getPubSubConsumerAdapterFactory() { + return pubSubConsumerAdapterFactory; } @Override diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceParentHelixAdmin.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceParentHelixAdmin.java index 38256dfa4b..c6b5cf2c71 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceParentHelixAdmin.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceParentHelixAdmin.java @@ -160,7 +160,6 @@ import com.linkedin.venice.helix.Replica; import com.linkedin.venice.helix.StoragePersonaRepository; import com.linkedin.venice.helix.ZkStoreConfigAccessor; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.meta.BackupStrategy; import com.linkedin.venice.meta.BufferReplayPolicy; import com.linkedin.venice.meta.DataReplicationPolicy; @@ -186,6 +185,7 @@ import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubProduceResult; import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.pushmonitor.ExecutionStatus; import com.linkedin.venice.pushstatushelper.PushStatusStoreReader; import com.linkedin.venice.pushstatushelper.PushStatusStoreWriter; @@ -4052,11 +4052,11 @@ public VeniceWriterFactory getVeniceWriterFactory() { } /** - * @see VeniceHelixAdmin#getVeniceConsumerFactory() + * @see VeniceHelixAdmin#getPubSubConsumerAdapterFactory() */ @Override - public PubSubConsumerAdapterFactory getVeniceConsumerFactory() { - return getVeniceHelixAdmin().getVeniceConsumerFactory(); + public PubSubConsumerAdapterFactory getPubSubConsumerAdapterFactory() { + return getVeniceHelixAdmin().getPubSubConsumerAdapterFactory(); } @Override diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/kafka/TopicCleanupService.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/kafka/TopicCleanupService.java index 09ef46fe6b..ced37b4355 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/kafka/TopicCleanupService.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/kafka/TopicCleanupService.java @@ -5,10 +5,10 @@ import com.linkedin.venice.controller.VeniceControllerMultiClusterConfig; import com.linkedin.venice.controller.stats.TopicCleanupServiceStats; import com.linkedin.venice.exceptions.VeniceException; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.meta.Version; import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.service.AbstractVeniceService; import com.linkedin.venice.utils.Time; import com.linkedin.venice.utils.Utils; @@ -252,7 +252,7 @@ private void populateDeprecatedTopicQueue(PriorityQueue topics) { private void refreshMultiDataCenterStoreToVersionTopicCountMap(Set localTopics) { if (localDatacenter == null) { - String localPubSubBootstrapServer = getTopicManager().getPubSubBootstrapServers(); + String localPubSubBootstrapServer = getTopicManager().getPubSubClusterAddress(); for (String childFabric: childFabricList) { if (localPubSubBootstrapServer.equals(multiClusterConfigs.getChildDataCenterKafkaUrlMap().get(childFabric))) { localDatacenter = childFabric; diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/kafka/TopicCleanupServiceForParentController.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/kafka/TopicCleanupServiceForParentController.java index 95806a4d62..da029874aa 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/kafka/TopicCleanupServiceForParentController.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/kafka/TopicCleanupServiceForParentController.java @@ -4,9 +4,9 @@ import com.linkedin.venice.controller.VeniceControllerMultiClusterConfig; import com.linkedin.venice.controller.stats.TopicCleanupServiceStats; import com.linkedin.venice.exceptions.VeniceException; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.manager.TopicManager; import java.util.HashMap; import java.util.Map; import java.util.Set; @@ -51,7 +51,7 @@ private void cleanupVeniceTopics(TopicManager topicManager) { if (getAdmin().isTopicTruncatedBasedOnRetention(retention)) { // Topic may be deleted after delay int remainingFactor = storeToCountdownForDeletion.merge( - topic.getName() + "_" + topicManager.getPubSubBootstrapServers(), + topic.getName() + "_" + topicManager.getPubSubClusterAddress(), delayFactor, (oldVal, givenVal) -> oldVal - 1); if (remainingFactor > 0) { @@ -65,7 +65,7 @@ private void cleanupVeniceTopics(TopicManager topicManager) { "Retention policy for topic: {} is: {} ms, and it is deprecated, will delete it now.", topic, retention); - storeToCountdownForDeletion.remove(topic + "_" + topicManager.getPubSubBootstrapServers()); + storeToCountdownForDeletion.remove(topic + "_" + topicManager.getPubSubClusterAddress()); try { topicManager.ensureTopicIsDeletedAndBlockWithRetry(topic); } catch (VeniceException e) { diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/kafka/consumer/AdminConsumerService.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/kafka/consumer/AdminConsumerService.java index f64aeb94ad..9fa13521cb 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/kafka/consumer/AdminConsumerService.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/kafka/consumer/AdminConsumerService.java @@ -67,7 +67,7 @@ public AdminConsumerService( remoteKafkaServerUrl = Optional.empty(); } this.localKafkaServerUrl = admin.getKafkaBootstrapServers(admin.isSslToKafka()); - this.consumerFactory = admin.getVeniceConsumerFactory(); + this.consumerFactory = admin.getPubSubConsumerAdapterFactory(); } @Override diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/kafka/consumer/AdminConsumptionTask.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/kafka/consumer/AdminConsumptionTask.java index 36fd9373c7..d38eb4770c 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/kafka/consumer/AdminConsumptionTask.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/kafka/consumer/AdminConsumptionTask.java @@ -14,7 +14,6 @@ import com.linkedin.venice.exceptions.validation.DataValidationException; import com.linkedin.venice.exceptions.validation.DuplicateDataException; import com.linkedin.venice.exceptions.validation.MissingDataException; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.kafka.protocol.GUID; import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; import com.linkedin.venice.kafka.protocol.ProducerMetadata; @@ -30,6 +29,7 @@ import com.linkedin.venice.pubsub.api.PubSubMessageDeserializer; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.utils.DaemonThreadFactory; import com.linkedin.venice.utils.LatencyUtils; import com.linkedin.venice.utils.Pair; @@ -960,7 +960,7 @@ private void recordConsumptionLag() { PubSubTopicPartition adminTopicPartition = new PubSubTopicPartitionImpl(pubSubTopic, AdminTopicUtils.ADMIN_TOPIC_PARTITION_ID); long sourceAdminTopicEndOffset = - sourceKafkaClusterTopicManager.getPartitionLatestOffsetAndRetry(adminTopicPartition, 10) - 1; + sourceKafkaClusterTopicManager.getLatestOffsetWithRetries(adminTopicPartition, 10) - 1; /** * If the first consumer poll returns nothing, "lastConsumedOffset" will remain as {@link #UNASSIGNED_VALUE}, so a * huge lag will be reported, but actually that's not case since consumer is subscribed to the last checkpoint offset. diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/ControllerRoutes.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/ControllerRoutes.java index 191658b8c0..b4a79fd505 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/ControllerRoutes.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/ControllerRoutes.java @@ -19,11 +19,11 @@ import com.linkedin.venice.controllerapi.LeaderControllerResponse; import com.linkedin.venice.controllerapi.PubSubTopicConfigResponse; import com.linkedin.venice.exceptions.ErrorType; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.meta.Instance; import com.linkedin.venice.pubsub.PubSubTopicConfiguration; import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.utils.Utils; import java.util.Optional; import org.apache.http.HttpStatus; diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/StoresRoutes.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/StoresRoutes.java index df535ef2cf..4583b70fa1 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/StoresRoutes.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/StoresRoutes.java @@ -86,7 +86,6 @@ import com.linkedin.venice.exceptions.ResourceStillExistsException; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.exceptions.VeniceNoStoreException; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.meta.RegionPushDetails; import com.linkedin.venice.meta.Store; import com.linkedin.venice.meta.StoreDataAudit; @@ -97,6 +96,7 @@ import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.exceptions.PubSubTopicDoesNotExistException; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.systemstore.schemas.StoreProperties; import com.linkedin.venice.utils.Utils; import java.util.ArrayList; diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/ingestion/control/RealTimeTopicSwitcher.java b/services/venice-controller/src/main/java/com/linkedin/venice/ingestion/control/RealTimeTopicSwitcher.java index f3559d65a8..d6f8e70131 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/ingestion/control/RealTimeTopicSwitcher.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/ingestion/control/RealTimeTopicSwitcher.java @@ -4,18 +4,20 @@ import static com.linkedin.venice.ConfigKeys.KAFKA_REPLICATION_FACTOR; import static com.linkedin.venice.ConfigKeys.KAFKA_REPLICATION_FACTOR_RT_TOPICS; import static com.linkedin.venice.VeniceConstants.REWIND_TIME_DECIDED_BY_SERVER; -import static com.linkedin.venice.kafka.TopicManager.DEFAULT_KAFKA_REPLICATION_FACTOR; +import static com.linkedin.venice.pubsub.PubSubConstants.DEFAULT_KAFKA_REPLICATION_FACTOR; import com.linkedin.venice.ConfigKeys; import com.linkedin.venice.exceptions.VeniceException; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.meta.DataReplicationPolicy; import com.linkedin.venice.meta.HybridStoreConfig; import com.linkedin.venice.meta.Store; import com.linkedin.venice.meta.Version; +import com.linkedin.venice.pubsub.PubSubConstants; import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.exceptions.PubSubTopicDoesNotExistException; +import com.linkedin.venice.pubsub.manager.TopicManager; +import com.linkedin.venice.utils.StoreUtils; import com.linkedin.venice.utils.SystemTime; import com.linkedin.venice.utils.Time; import com.linkedin.venice.utils.VeniceProperties; @@ -95,7 +97,7 @@ void sendTopicSwitch( throw new PubSubTopicDoesNotExistException( errorPrefix + " topic " + topicWhereToSendTheTopicSwitch + " does not exist."); } - int destinationPartitionCount = getTopicManager().partitionsFor(topicWhereToSendTheTopicSwitch).size(); + int destinationPartitionCount = getTopicManager().getPartitionCount(topicWhereToSendTheTopicSwitch); List sourceClusters = new ArrayList<>(); if (!remoteKafkaUrls.isEmpty()) { sourceClusters.addAll(remoteKafkaUrls); @@ -159,7 +161,7 @@ void ensurePreconditions( srcTopicName, partitionCount, replicationFactor, - TopicManager.getExpectedRetentionTimeInMs(store, hybridStoreConfig.get()), + StoreUtils.getExpectedRetentionTimeInMs(store, hybridStoreConfig.get()), false, // Note: do not enable RT compaction! Might make jobs in Online/Offline model stuck minISR, false); @@ -168,7 +170,7 @@ void ensurePreconditions( * If real-time topic already exists, check whether its retention time is correct. */ long topicRetentionTimeInMs = getTopicManager().getTopicRetention(srcTopicName); - long expectedRetentionTimeMs = TopicManager.getExpectedRetentionTimeInMs(store, hybridStoreConfig.get()); + long expectedRetentionTimeMs = StoreUtils.getExpectedRetentionTimeInMs(store, hybridStoreConfig.get()); if (topicRetentionTimeInMs != expectedRetentionTimeMs) { getTopicManager().updateTopicRetention(srcTopicName, expectedRetentionTimeMs); } @@ -189,7 +191,7 @@ long getRewindStartTime( long rewindTimeInMs = hybridStoreConfig.get().getRewindTimeInSeconds() * Time.MS_PER_SECOND; if (version.getDataRecoveryVersionConfig() != null) { // Override the user rewind if the version is under data recovery to avoid data loss when user have short rewind. - rewindTimeInMs = Math.min(TopicManager.BUFFER_REPLAY_MINIMAL_SAFETY_MARGIN, rewindTimeInMs); + rewindTimeInMs = Math.min(PubSubConstants.BUFFER_REPLAY_MINIMAL_SAFETY_MARGIN, rewindTimeInMs); } switch (hybridStoreConfig.get().getBufferReplayPolicy()) { // TODO to get a more deterministic timestamp across colo we could use the timestamp from the SOP/EOP control diff --git a/services/venice-controller/src/test/java/com/linkedin/venice/controller/AbstractTestVeniceParentHelixAdmin.java b/services/venice-controller/src/test/java/com/linkedin/venice/controller/AbstractTestVeniceParentHelixAdmin.java index 33611e57c4..b44710b9eb 100644 --- a/services/venice-controller/src/test/java/com/linkedin/venice/controller/AbstractTestVeniceParentHelixAdmin.java +++ b/services/venice-controller/src/test/java/com/linkedin/venice/controller/AbstractTestVeniceParentHelixAdmin.java @@ -22,12 +22,12 @@ import com.linkedin.venice.helix.StoragePersonaRepository; import com.linkedin.venice.helix.ZkRoutersClusterManager; import com.linkedin.venice.helix.ZkStoreConfigAccessor; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.meta.HybridStoreConfig; import com.linkedin.venice.meta.OfflinePushStrategy; import com.linkedin.venice.meta.Store; import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.schema.SchemaEntry; import com.linkedin.venice.utils.TestUtils; import com.linkedin.venice.utils.Time; diff --git a/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestVeniceHelixAdminWithoutCluster.java b/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestVeniceHelixAdminWithoutCluster.java index bcb4b01478..c052635862 100644 --- a/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestVeniceHelixAdminWithoutCluster.java +++ b/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestVeniceHelixAdminWithoutCluster.java @@ -10,7 +10,6 @@ import com.linkedin.venice.common.VeniceSystemStoreType; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.helix.ZkStoreConfigAccessor; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.meta.BufferReplayPolicy; import com.linkedin.venice.meta.DataReplicationPolicy; import com.linkedin.venice.meta.HybridStoreConfig; @@ -21,6 +20,7 @@ import com.linkedin.venice.meta.Version; import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.utils.TestUtils; import com.linkedin.venice.utils.Utils; import java.util.Collections; diff --git a/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestVeniceParentHelixAdmin.java b/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestVeniceParentHelixAdmin.java index d99cffd181..00216c5cc0 100644 --- a/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestVeniceParentHelixAdmin.java +++ b/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestVeniceParentHelixAdmin.java @@ -52,7 +52,6 @@ import com.linkedin.venice.exceptions.VeniceStoreAlreadyExistsException; import com.linkedin.venice.exceptions.VeniceUnsupportedOperationException; import com.linkedin.venice.helix.HelixReadWriteStoreRepository; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.meta.BufferReplayPolicy; import com.linkedin.venice.meta.DataReplicationPolicy; import com.linkedin.venice.meta.HybridStoreConfigImpl; @@ -71,6 +70,7 @@ import com.linkedin.venice.partitioner.InvalidKeySchemaPartitioner; import com.linkedin.venice.pubsub.adapter.SimplePubSubProduceResultImpl; import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.pushmonitor.ExecutionStatus; import com.linkedin.venice.pushmonitor.OfflinePushStatus; import com.linkedin.venice.pushmonitor.PartitionStatus; diff --git a/services/venice-controller/src/test/java/com/linkedin/venice/controller/kafka/TestTopicCleanupService.java b/services/venice-controller/src/test/java/com/linkedin/venice/controller/kafka/TestTopicCleanupService.java index 11ee3a60c1..7760f1de4b 100644 --- a/services/venice-controller/src/test/java/com/linkedin/venice/controller/kafka/TestTopicCleanupService.java +++ b/services/venice-controller/src/test/java/com/linkedin/venice/controller/kafka/TestTopicCleanupService.java @@ -17,11 +17,11 @@ import com.linkedin.venice.controller.stats.TopicCleanupServiceStats; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.helix.HelixReadOnlyStoreConfigRepository; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.meta.StoreConfig; import com.linkedin.venice.meta.Version; import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.utils.TestUtils; import com.linkedin.venice.utils.Utils; import java.util.Arrays; @@ -70,7 +70,7 @@ public void setUp() { dataCenterToBootstrapServerMap.put("local", "local"); dataCenterToBootstrapServerMap.put("remote", "remote"); doReturn(dataCenterToBootstrapServerMap).when(veniceControllerMultiClusterConfig).getChildDataCenterKafkaUrlMap(); - doReturn("local").when(topicManager).getPubSubBootstrapServers(); + doReturn("local").when(topicManager).getPubSubClusterAddress(); remoteTopicManager = mock(TopicManager.class); doReturn(remoteTopicManager).when(admin).getTopicManager("remote"); doReturn(Collections.emptyMap()).when(remoteTopicManager).getAllTopicRetentions(); @@ -394,7 +394,7 @@ public void testCleanVeniceTopicsBlockRTTopicDeletionWhenMisconfigured() { doReturn(storeTopics).when(remoteTopicManager).getAllTopicRetentions(); doReturn(Optional.of(new StoreConfig(storeName))).when(storeConfigRepository).getStoreConfig(storeName); blockedTopicCleanupService.cleanupVeniceTopics(); - verify(topicManager, atLeastOnce()).getPubSubBootstrapServers(); + verify(topicManager, atLeastOnce()).getPubSubClusterAddress(); verify(topicManager, never()).ensureTopicIsDeletedAndBlockWithRetry(getPubSubTopic(storeName, "_rt")); verify(topicCleanupServiceStats, atLeastOnce()).recordDeletableTopicsCount(1); verify(topicCleanupServiceStats, atLeastOnce()).recordTopicDeletionError(); diff --git a/services/venice-controller/src/test/java/com/linkedin/venice/controller/kafka/TestTopicCleanupServiceForMultiKafkaClusters.java b/services/venice-controller/src/test/java/com/linkedin/venice/controller/kafka/TestTopicCleanupServiceForMultiKafkaClusters.java index 24036313ec..1fe3889f68 100644 --- a/services/venice-controller/src/test/java/com/linkedin/venice/controller/kafka/TestTopicCleanupServiceForMultiKafkaClusters.java +++ b/services/venice-controller/src/test/java/com/linkedin/venice/controller/kafka/TestTopicCleanupServiceForMultiKafkaClusters.java @@ -9,9 +9,9 @@ import com.linkedin.venice.controller.VeniceControllerConfig; import com.linkedin.venice.controller.VeniceControllerMultiClusterConfig; import com.linkedin.venice.controller.stats.TopicCleanupServiceStats; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.manager.TopicManager; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -55,10 +55,10 @@ public void setUp() { admin = mock(Admin.class); topicManager1 = mock(TopicManager.class); - doReturn(kafkaClusterServerUrl1).when(topicManager1).getPubSubBootstrapServers(); + doReturn(kafkaClusterServerUrl1).when(topicManager1).getPubSubClusterAddress(); doReturn(topicManager1).when(admin).getTopicManager(kafkaClusterServerUrl1); topicManager2 = mock(TopicManager.class); - doReturn(kafkaClusterServerUrl2).when(topicManager2).getPubSubBootstrapServers(); + doReturn(kafkaClusterServerUrl2).when(topicManager2).getPubSubClusterAddress(); doReturn(topicManager2).when(admin).getTopicManager(kafkaClusterServerUrl2); TopicCleanupServiceStats topicCleanupServiceStats = mock(TopicCleanupServiceStats.class); diff --git a/services/venice-controller/src/test/java/com/linkedin/venice/controller/kafka/TestTopicCleanupServiceForParentController.java b/services/venice-controller/src/test/java/com/linkedin/venice/controller/kafka/TestTopicCleanupServiceForParentController.java index 235ab9b310..2aa66cb180 100644 --- a/services/venice-controller/src/test/java/com/linkedin/venice/controller/kafka/TestTopicCleanupServiceForParentController.java +++ b/services/venice-controller/src/test/java/com/linkedin/venice/controller/kafka/TestTopicCleanupServiceForParentController.java @@ -9,9 +9,9 @@ import com.linkedin.venice.controller.VeniceControllerConfig; import com.linkedin.venice.controller.VeniceControllerMultiClusterConfig; import com.linkedin.venice.controller.stats.TopicCleanupServiceStats; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.manager.TopicManager; import java.util.HashMap; import java.util.Map; import java.util.concurrent.ExecutionException; diff --git a/services/venice-controller/src/test/java/com/linkedin/venice/controller/kafka/consumer/AdminConsumptionTaskTest.java b/services/venice-controller/src/test/java/com/linkedin/venice/controller/kafka/consumer/AdminConsumptionTaskTest.java index 6bd6c3ba8c..019afc0744 100644 --- a/services/venice-controller/src/test/java/com/linkedin/venice/controller/kafka/consumer/AdminConsumptionTaskTest.java +++ b/services/venice-controller/src/test/java/com/linkedin/venice/controller/kafka/consumer/AdminConsumptionTaskTest.java @@ -59,7 +59,6 @@ import com.linkedin.venice.controller.stats.AdminConsumptionStats; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.guid.GuidUtils; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; import com.linkedin.venice.kafka.protocol.state.PartitionState; import com.linkedin.venice.kafka.protocol.state.ProducerPartitionState; @@ -78,6 +77,7 @@ import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; import com.linkedin.venice.pubsub.api.exceptions.PubSubOpTimeoutException; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.serialization.DefaultSerializer; import com.linkedin.venice.serialization.avro.AvroProtocolDefinition; import com.linkedin.venice.serialization.avro.OptimizedKafkaValueSerializer; diff --git a/services/venice-controller/src/test/java/com/linkedin/venice/controller/kafka/consumer/TestAdminConsumerService.java b/services/venice-controller/src/test/java/com/linkedin/venice/controller/kafka/consumer/TestAdminConsumerService.java index ec408a564b..9629005f65 100644 --- a/services/venice-controller/src/test/java/com/linkedin/venice/controller/kafka/consumer/TestAdminConsumerService.java +++ b/services/venice-controller/src/test/java/com/linkedin/venice/controller/kafka/consumer/TestAdminConsumerService.java @@ -66,7 +66,7 @@ public void testMultipleAdminConsumerServiceWithSameMetricsRepo() { VeniceHelixAdmin admin = mock(VeniceHelixAdmin.class); doReturn(mock(ZkClient.class)).when(admin).getZkClient(); doReturn(mock(HelixAdapterSerializer.class)).when(admin).getAdapterSerializer(); - doReturn(consumerFactory).when(admin).getVeniceConsumerFactory(); + doReturn(consumerFactory).when(admin).getPubSubConsumerAdapterFactory(); doReturn("localhost:1234").when(admin).getKafkaBootstrapServers(true); doReturn(true).when(admin).isSslToKafka(); diff --git a/services/venice-controller/src/test/java/com/linkedin/venice/ingestion/control/RealTimeTopicSwitcherRewindTest.java b/services/venice-controller/src/test/java/com/linkedin/venice/ingestion/control/RealTimeTopicSwitcherRewindTest.java index b79bbe0e0c..14a035a38c 100644 --- a/services/venice-controller/src/test/java/com/linkedin/venice/ingestion/control/RealTimeTopicSwitcherRewindTest.java +++ b/services/venice-controller/src/test/java/com/linkedin/venice/ingestion/control/RealTimeTopicSwitcherRewindTest.java @@ -11,7 +11,6 @@ import static org.testng.Assert.fail; import com.linkedin.venice.exceptions.VeniceException; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.meta.BufferReplayPolicy; import com.linkedin.venice.meta.DataReplicationPolicy; import com.linkedin.venice.meta.HybridStoreConfig; @@ -20,6 +19,7 @@ import com.linkedin.venice.meta.Version; import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.utils.TestMockTime; import com.linkedin.venice.utils.TestUtils; import com.linkedin.venice.utils.Time; diff --git a/services/venice-controller/src/test/java/com/linkedin/venice/ingestion/control/RealTimeTopicSwitcherTest.java b/services/venice-controller/src/test/java/com/linkedin/venice/ingestion/control/RealTimeTopicSwitcherTest.java index 02d31f3e85..aa58530814 100644 --- a/services/venice-controller/src/test/java/com/linkedin/venice/ingestion/control/RealTimeTopicSwitcherTest.java +++ b/services/venice-controller/src/test/java/com/linkedin/venice/ingestion/control/RealTimeTopicSwitcherTest.java @@ -15,7 +15,6 @@ import static org.mockito.Mockito.when; import com.linkedin.venice.ConfigKeys; -import com.linkedin.venice.kafka.TopicManager; import com.linkedin.venice.meta.DataReplicationPolicy; import com.linkedin.venice.meta.HybridStoreConfig; import com.linkedin.venice.meta.Store; @@ -25,6 +24,7 @@ import com.linkedin.venice.meta.ViewConfigImpl; import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubTopic; +import com.linkedin.venice.pubsub.manager.TopicManager; import com.linkedin.venice.utils.VeniceProperties; import com.linkedin.venice.writer.VeniceWriter; import com.linkedin.venice.writer.VeniceWriterFactory; @@ -36,7 +36,6 @@ import java.util.Map; import java.util.Optional; import java.util.Properties; -import org.apache.kafka.common.PartitionInfo; import org.mockito.ArgumentCaptor; import org.testng.Assert; import org.testng.annotations.BeforeMethod; @@ -76,7 +75,6 @@ public void testPrepareAndStartReplication() { PubSubTopic destTopic = pubSubTopicRepository.getTopic("testTopic_v1"); Store mockStore = mock(Store.class); HybridStoreConfig mockHybridConfig = mock(HybridStoreConfig.class); - List partitionInfos = new ArrayList<>(); VeniceWriter mockVeniceWriter = mock(VeniceWriter.class); doReturn(true).when(mockStore).isHybrid(); @@ -88,8 +86,6 @@ public void testPrepareAndStartReplication() { doReturn(REWIND_FROM_EOP).when(mockHybridConfig).getBufferReplayPolicy(); doReturn(true).when(mockTopicManager).containsTopicAndAllPartitionsAreOnline(srcTopic); doReturn(true).when(mockTopicManager).containsTopicAndAllPartitionsAreOnline(destTopic); - doReturn(partitionInfos).when(mockTopicManager).partitionsFor(srcTopic); - doReturn(partitionInfos).when(mockTopicManager).partitionsFor(destTopic); doReturn(mockVeniceWriter).when(mockVeniceWriterFactory).createVeniceWriter(any(VeniceWriterOptions.class)); leaderStorageNodeReplicator.switchToRealTimeTopic( @@ -108,7 +104,6 @@ public void testPrepareAndStartReplicationWithNativeReplication() { PubSubTopic destTopic = pubSubTopicRepository.getTopic("testTopic_v1"); Store mockStore = mock(Store.class); HybridStoreConfig mockHybridConfig = mock(HybridStoreConfig.class); - List partitionInfos = new ArrayList<>(); VeniceWriter mockVeniceWriter = mock(VeniceWriter.class); doReturn(true).when(mockStore).isHybrid(); @@ -122,8 +117,6 @@ public void testPrepareAndStartReplicationWithNativeReplication() { doReturn(DataReplicationPolicy.AGGREGATE).when(mockHybridConfig).getDataReplicationPolicy(); doReturn(true).when(mockTopicManager).containsTopicAndAllPartitionsAreOnline(srcTopic); doReturn(true).when(mockTopicManager).containsTopicAndAllPartitionsAreOnline(destTopic); - doReturn(partitionInfos).when(mockTopicManager).partitionsFor(srcTopic); - doReturn(partitionInfos).when(mockTopicManager).partitionsFor(destTopic); doReturn(mockVeniceWriter).when(mockVeniceWriterFactory).createVeniceWriter(any(VeniceWriterOptions.class)); leaderStorageNodeReplicator.switchToRealTimeTopic( From b99012066558d54ef009f31db8abae1f02b7bb69 Mon Sep 17 00:00:00 2001 From: Sushant Mane Date: Mon, 12 Feb 2024 07:17:06 -0800 Subject: [PATCH 2/8] Cleanup test --- .../pubsub/manager/TopicManagerE2ETest.java | 209 +++++++++--------- 1 file changed, 103 insertions(+), 106 deletions(-) diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/pubsub/manager/TopicManagerE2ETest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/pubsub/manager/TopicManagerE2ETest.java index c76a30eef7..84ee70ccb0 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/pubsub/manager/TopicManagerE2ETest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/pubsub/manager/TopicManagerE2ETest.java @@ -4,7 +4,6 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; -import static org.testng.Assert.assertNull; import static org.testng.Assert.assertThrows; import static org.testng.Assert.assertTrue; import static org.testng.Assert.expectThrows; @@ -15,13 +14,11 @@ import com.linkedin.venice.integration.utils.ServiceFactory; import com.linkedin.venice.pubsub.PubSubClientsFactory; import com.linkedin.venice.pubsub.PubSubConstants; -import com.linkedin.venice.pubsub.PubSubTopicConfiguration; import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl; import com.linkedin.venice.pubsub.PubSubTopicRepository; import com.linkedin.venice.pubsub.api.PubSubAdminAdapter; import com.linkedin.venice.pubsub.api.PubSubConsumerAdapter; import com.linkedin.venice.pubsub.api.PubSubMessageDeserializer; -import com.linkedin.venice.pubsub.api.PubSubProduceResult; import com.linkedin.venice.pubsub.api.PubSubProducerAdapter; import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.pubsub.api.PubSubTopicPartition; @@ -34,14 +31,11 @@ import com.linkedin.venice.utils.VeniceProperties; import com.linkedin.venice.utils.lazy.Lazy; import io.tehuti.metrics.MetricsRepository; -import java.time.Duration; import java.util.ArrayList; -import java.util.HashMap; +import java.util.Arrays; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Properties; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -58,27 +52,7 @@ public class TopicManagerE2ETest { - // timeout for pub-sub operations - private static final Duration PUBSUB_OP_TIMEOUT = Duration.ofSeconds(15); - // add a variance of 5 seconds to the timeout to account for fluctuations in the test environment - private static final long PUBSUB_OP_TIMEOUT_WITH_VARIANCE = PUBSUB_OP_TIMEOUT.toMillis() + 5000; - // timeout for pub-sub consumer APIs which do not have a timeout parameter private static final int PUBSUB_CONSUMER_API_DEFAULT_TIMEOUT_MS = 10_000; - // add a variance of 5 seconds to the timeout to account for fluctuations in the test environment - private static final long PUBSUB_CONSUMER_API_DEFAULT_TIMEOUT_MS_WITH_VARIANCE = - PUBSUB_CONSUMER_API_DEFAULT_TIMEOUT_MS + 5000; - private static final int REPLICATION_FACTOR = 1; - private static final boolean IS_LOG_COMPACTED = false; - private static final int MIN_IN_SYNC_REPLICAS = 1; - private static final long RETENTION_IN_MS = Duration.ofDays(3).toMillis(); - private static final long MIN_LOG_COMPACTION_LAG_MS = Duration.ofDays(1).toMillis(); - private static final long MAX_LOG_COMPACTION_LAG_MS = Duration.ofDays(2).toMillis(); - private static final PubSubTopicConfiguration TOPIC_CONFIGURATION = new PubSubTopicConfiguration( - Optional.of(RETENTION_IN_MS), - IS_LOG_COMPACTED, - Optional.of(MIN_IN_SYNC_REPLICAS), - MIN_LOG_COMPACTION_LAG_MS, - Optional.of(MAX_LOG_COMPACTION_LAG_MS)); private PubSubBrokerWrapper pubSubBrokerWrapper; private Lazy pubSubAdminAdapterLazy; @@ -171,88 +145,97 @@ public void testAsyncApis() throws ExecutionException, InterruptedException, Tim int numMessages = 250; PubSubProducerAdapter pubSubProducerAdapter = pubSubProducerAdapterLazy.get(); - CompletableFuture lastMessageFuture = null; - // list of messages - Map messages = new HashMap<>(numMessages); - for (int i = 0; i < numMessages; i++) { - MutablePubSubMessage message = PubSubHelper.getDummyPubSubMessage(false); - message.getValue().getProducerMetadata().setMessageTimestamp(i); - messages.put(i, message); - lastMessageFuture = - pubSubProducerAdapter.sendMessage(testTopic.getName(), 0, message.getKey(), message.getValue(), null, null); - lastMessageFuture.whenComplete((result, throwable) -> { - if (throwable == null) { - message.setOffset(result.getOffset()); - } - }); - } - assertNotNull(lastMessageFuture, "Last message future should not be null"); - lastMessageFuture.get(1, TimeUnit.MINUTES); - assertEquals(messages.size(), numMessages); + PubSubTopicPartition topicPartition = new PubSubTopicPartitionImpl(testTopic, 0); + List messages = + PubSubHelper.produceMessages(pubSubProducerAdapter, topicPartition, numMessages, 2, false); + long timeBeforeProduce = messages.get(0).getTimestampBeforeProduce() - 10; + long tsOfLastDataMessage = messages.get(messages.size() - 1).getValue().getProducerMetadata().getMessageTimestamp(); + PubSubTopicPartition nonExistentTopicPartition = new PubSubTopicPartitionImpl(nonExistentTopic, 0); final AtomicInteger successfulRequests = new AtomicInteger(0); - List tasks = new ArrayList<>(); - - Runnable getPartitionCountTask = () -> { - try { - int actualNumPartitions = topicManager.getPartitionCount(testTopic); - assertEquals(actualNumPartitions, numPartitions); - successfulRequests.incrementAndGet(); - } catch (Exception e) { - throw new RuntimeException(e); - } - }; - tasks.add(getPartitionCountTask); - - // get partition count for non-existent topic - Runnable getPartitionCountForNonExistentTopicTask = () -> { - try { - assertNull(topicManager.getPartitionCount(nonExistentTopic)); - successfulRequests.incrementAndGet(); - } catch (Exception e) { - throw new RuntimeException(e); - } - }; - tasks.add(getPartitionCountForNonExistentTopicTask); + // get partition count for an existing topic + Runnable t1 = getAssertionTask( + () -> assertEquals(topicManager.getPartitionCount(testTopic), numPartitions), + successfulRequests); + // get partition count for a non-existent topic + Runnable t2 = getAssertionTask( + () -> assertThrows( + PubSubTopicDoesNotExistException.class, + () -> topicManager.getPartitionCount(nonExistentTopic)), + successfulRequests); // contains topic - Runnable containsTopicTask = () -> { - try { - assertTrue(topicManager.containsTopic(testTopic)); - successfulRequests.incrementAndGet(); - } catch (Exception e) { - throw new RuntimeException(e); - } - }; - tasks.add(containsTopicTask); - + Runnable t3 = getAssertionTask(() -> assertTrue(topicManager.containsTopic(testTopic)), successfulRequests); // contains topic for non-existent topic - Runnable containsNonExistentTopicTask = () -> { - try { - assertFalse(topicManager.containsTopic(nonExistentTopic)); - successfulRequests.incrementAndGet(); - } catch (Exception e) { - throw new RuntimeException(e); - } - }; - tasks.add(containsNonExistentTopicTask); - - Runnable getLatestOffsetWithRetriesTask = () -> { - try { - long latestOffset = topicManager.getLatestOffsetWithRetries(new PubSubTopicPartitionImpl(testTopic, 0), 1); - assertEquals(latestOffset, numMessages); - successfulRequests.incrementAndGet(); - } catch (Exception e) { - throw new RuntimeException(e); - } - }; - tasks.add(getLatestOffsetWithRetriesTask); - + Runnable t4 = getAssertionTask(() -> assertFalse(topicManager.containsTopic(nonExistentTopic)), successfulRequests); + // contains topic cached + Runnable t5 = getAssertionTask(() -> assertTrue(topicManager.containsTopicCached(testTopic)), successfulRequests); + // contains topic cached for non-existent topic + Runnable t6 = + getAssertionTask(() -> assertFalse(topicManager.containsTopicCached(nonExistentTopic)), successfulRequests); + // get latest offset with retries for an existing topic + Runnable t7 = getAssertionTask( + () -> assertEquals(topicManager.getLatestOffsetWithRetries(topicPartition, 1), numMessages), + successfulRequests); + // get latest offset with retries for a non-existent topic + Runnable t8 = getAssertionTask( + () -> assertThrows( + PubSubTopicDoesNotExistException.class, + () -> topicManager.getLatestOffsetWithRetries(nonExistentTopicPartition, 1)), + successfulRequests); + // get latest offset cached for an existing topic + Runnable t9 = getAssertionTask( + () -> assertEquals(topicManager.getLatestOffsetCached(testTopic, 0), numMessages), + successfulRequests); + // get latest offset cached for a non-existent topic + Runnable t10 = getAssertionTask( + () -> assertEquals( + topicManager.getLatestOffsetCached(nonExistentTopic, 0), + StatsErrorCode.LAG_MEASUREMENT_FAILURE.code), + successfulRequests); + // get producer timestamp of last data message with retries for an existing topic + Runnable t11 = getAssertionTask( + () -> assertEquals( + topicManager.getProducerTimestampOfLastDataMessageWithRetries(topicPartition, 1), + tsOfLastDataMessage), + successfulRequests); + // get producer timestamp of last data message with retries for a non-existent topic + Runnable t12 = getAssertionTask( + () -> assertThrows( + PubSubTopicDoesNotExistException.class, + () -> topicManager.getProducerTimestampOfLastDataMessageWithRetries(nonExistentTopicPartition, 1)), + successfulRequests); + // get producer timestamp of last data message cached for an existing topic + Runnable t13 = getAssertionTask( + () -> assertEquals( + topicManager.getProducerTimestampOfLastDataMessageCached(topicPartition), + tsOfLastDataMessage), + successfulRequests); + // get producer timestamp of last data message cached for a non-existent topic + Runnable t14 = getAssertionTask( + () -> assertEquals( + topicManager.getProducerTimestampOfLastDataMessageCached(nonExistentTopicPartition), + StatsErrorCode.LAG_MEASUREMENT_FAILURE.code), + successfulRequests); + // get offset by time for an existing topic + Runnable t15 = getAssertionTask( + () -> assertEquals(topicManager.getOffsetByTime(topicPartition, System.currentTimeMillis()), numMessages), + successfulRequests); + // get offset by time for a non-existent topic + Runnable t16 = getAssertionTask( + () -> assertThrows( + PubSubTopicDoesNotExistException.class, + () -> topicManager.getOffsetByTime(nonExistentTopicPartition, tsOfLastDataMessage)), + successfulRequests); + // get offset by time for an existing topic: first message + Runnable t17 = getAssertionTask( + () -> assertEquals(topicManager.getOffsetByTime(topicPartition, timeBeforeProduce), 0), + successfulRequests); + + List tasks = Arrays.asList(t1, t2, t3, t4, t5, t6, t7, t8, t9, t10, t11, t12, t13, t14, t15, t16, t17); ExecutorService executorService = Executors.newFixedThreadPool(8); - List vwFutures = new ArrayList<>(); - - int totalTasks = 1024; + int totalTasks = 100; for (int i = 0; i < totalTasks; i++) { Future future = executorService.submit(tasks.get(i % tasks.size())); vwFutures.add(future); @@ -261,14 +244,28 @@ public void testAsyncApis() throws ExecutionException, InterruptedException, Tim int failedRequests = 0; for (Future future: vwFutures) { try { - future.get(1, TimeUnit.MINUTES); + future.get(3, TimeUnit.MINUTES); } catch (Exception e) { failedRequests++; } } - System.out.println("successfulRequests: " + successfulRequests.get()); - // total should be equal to the number of tasks - assertEquals(successfulRequests.get() + failedRequests, totalTasks); + assertEquals(failedRequests, 0); + assertEquals(successfulRequests.get(), totalTasks); + } + + private static Runnable getAssertionTask(Runnable runnable, AtomicInteger successfulRequests) { + return () -> { + try { + runnable.run(); + successfulRequests.incrementAndGet(); + } catch (AssertionError e) { + e.printStackTrace(); + throw new AssertionError("Assertion failed: " + e.getMessage(), e); + } catch (Exception e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + }; } @Test(timeOut = 3 * Time.MS_PER_MINUTE) From aed9c6a3d946fd6c2dcd636549e9c1c0bb78fe62 Mon Sep 17 00:00:00 2001 From: Sushant Mane Date: Mon, 12 Feb 2024 13:34:41 -0800 Subject: [PATCH 3/8] Address review comment --- .../kafka/consumer/StoreIngestionTask.java | 5 +- .../venice/pubsub/manager/TopicManager.java | 2 +- .../manager/TopicManagerRepository.java | 13 +- .../pubsub/manager/TopicMetadataFetcher.java | 11 ++ .../pubsub/manager/TopicManagerE2ETest.java | 162 ++++++++---------- 5 files changed, 94 insertions(+), 99 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 3e758c9fde..a61f9ac0a2 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java @@ -1523,10 +1523,7 @@ private void internalClose(boolean doFlush) { } if (topicManagerRepository != null) { - Collection topicManagers = topicManagerRepository.getAllTopicManagers(); - for (TopicManager topicManager: topicManagers) { - topicManager.invalidateCache(versionTopic); - } + topicManagerRepository.invalidateTopicManagerCaches(versionTopic); } close(); diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java index 21b4bd5bdd..f2ad9a5aea 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java @@ -714,7 +714,7 @@ public void close() { logger.warn("{} is already closed", this); } try { - closeAsync().get(600, TimeUnit.SECONDS); + closeAsync().get(2, TimeUnit.MINUTES); } catch (InterruptedException e) { Thread.currentThread().interrupt(); logger.error("Interrupted while closing: {}", this, e); diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerRepository.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerRepository.java index 94d0be66e5..4a5b12198a 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerRepository.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerRepository.java @@ -1,5 +1,6 @@ package com.linkedin.venice.pubsub.manager; +import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap; import java.io.Closeable; import java.util.ArrayList; @@ -47,10 +48,20 @@ public TopicManager getTopicManager(String pubSubAddress) { return topicManagers.computeIfAbsent(pubSubAddress, this::createTopicManager); } - public Collection getAllTopicManagers() { + Collection getAllTopicManagers() { return topicManagers.values(); } + /** + * Invalidates the cache for the given PubSub topic across all TopicManagers in the repository. + * @param pubSubTopic the PubSub topic to invalidate + */ + public void invalidateTopicManagerCaches(PubSubTopic pubSubTopic) { + for (TopicManager topicManager: getAllTopicManagers()) { + topicManager.invalidateCache(pubSubTopic); + } + } + @Override public void close() { long startTime = System.currentTimeMillis(); diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java index e1ed57e77f..0f621ed892 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java @@ -665,6 +665,12 @@ void updateCacheAsync( return; } + // check if the value has been already updated by another thread; if so, release the lock and return + if (cachedValue != null && cachedValue.getExpiryTimeNs() > System.nanoTime()) { + cachedValue.releaseUpdateLock(); + return; + } + completableFutureSupplier.get().whenComplete((value, throwable) -> { if (throwable != null) { cache.remove(key); @@ -713,6 +719,11 @@ boolean tryAcquireUpdateLock() { return isUpdateInProgress.compareAndSet(false, true); } + // release the lock + boolean releaseUpdateLock() { + return isUpdateInProgress.compareAndSet(true, false); + } + void updateValue(T value) { this.value = value; this.expiryTimeNs = System.nanoTime() + cachedEntryTtlInNs; // update the expiry time diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/pubsub/manager/TopicManagerE2ETest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/pubsub/manager/TopicManagerE2ETest.java index 84ee70ccb0..40748d93ae 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/pubsub/manager/TopicManagerE2ETest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/pubsub/manager/TopicManagerE2ETest.java @@ -36,11 +36,11 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; @@ -131,8 +131,8 @@ public void tearDownMethod() { } } - @Test(timeOut = 5 * Time.MS_PER_MINUTE, invocationCount = 1) - public void testAsyncApis() throws ExecutionException, InterruptedException, TimeoutException { + @Test(timeOut = 5 * Time.MS_PER_MINUTE) + public void testConcurrentApiExecution() throws ExecutionException, InterruptedException, TimeoutException { int numPartitions = 3; int replicationFactor = 1; boolean isEternalTopic = true; @@ -152,122 +152,98 @@ public void testAsyncApis() throws ExecutionException, InterruptedException, Tim long tsOfLastDataMessage = messages.get(messages.size() - 1).getValue().getProducerMetadata().getMessageTimestamp(); PubSubTopicPartition nonExistentTopicPartition = new PubSubTopicPartitionImpl(nonExistentTopic, 0); - final AtomicInteger successfulRequests = new AtomicInteger(0); // get partition count for an existing topic - Runnable t1 = getAssertionTask( - () -> assertEquals(topicManager.getPartitionCount(testTopic), numPartitions), - successfulRequests); + Runnable t1 = () -> assertEquals(topicManager.getPartitionCount(testTopic), numPartitions); + // get partition count for a non-existent topic - Runnable t2 = getAssertionTask( - () -> assertThrows( - PubSubTopicDoesNotExistException.class, - () -> topicManager.getPartitionCount(nonExistentTopic)), - successfulRequests); + Runnable t2 = () -> assertThrows( + PubSubTopicDoesNotExistException.class, + () -> topicManager.getPartitionCount(nonExistentTopic)); + // contains topic - Runnable t3 = getAssertionTask(() -> assertTrue(topicManager.containsTopic(testTopic)), successfulRequests); + Runnable t3 = () -> assertTrue(topicManager.containsTopic(testTopic)); + // contains topic for non-existent topic - Runnable t4 = getAssertionTask(() -> assertFalse(topicManager.containsTopic(nonExistentTopic)), successfulRequests); + Runnable t4 = () -> assertFalse(topicManager.containsTopic(nonExistentTopic)); + // contains topic cached - Runnable t5 = getAssertionTask(() -> assertTrue(topicManager.containsTopicCached(testTopic)), successfulRequests); + Runnable t5 = () -> assertTrue(topicManager.containsTopicCached(testTopic)); + // contains topic cached for non-existent topic - Runnable t6 = - getAssertionTask(() -> assertFalse(topicManager.containsTopicCached(nonExistentTopic)), successfulRequests); + Runnable t6 = () -> assertFalse(topicManager.containsTopicCached(nonExistentTopic)); + // get latest offset with retries for an existing topic - Runnable t7 = getAssertionTask( - () -> assertEquals(topicManager.getLatestOffsetWithRetries(topicPartition, 1), numMessages), - successfulRequests); + Runnable t7 = () -> assertEquals(topicManager.getLatestOffsetWithRetries(topicPartition, 1), numMessages); + // get latest offset with retries for a non-existent topic - Runnable t8 = getAssertionTask( - () -> assertThrows( - PubSubTopicDoesNotExistException.class, - () -> topicManager.getLatestOffsetWithRetries(nonExistentTopicPartition, 1)), - successfulRequests); + Runnable t8 = () -> assertThrows( + PubSubTopicDoesNotExistException.class, + () -> topicManager.getLatestOffsetWithRetries(nonExistentTopicPartition, 1)); + // get latest offset cached for an existing topic - Runnable t9 = getAssertionTask( - () -> assertEquals(topicManager.getLatestOffsetCached(testTopic, 0), numMessages), - successfulRequests); + Runnable t9 = () -> assertEquals(topicManager.getLatestOffsetCached(testTopic, 0), numMessages); + // get latest offset cached for a non-existent topic - Runnable t10 = getAssertionTask( - () -> assertEquals( - topicManager.getLatestOffsetCached(nonExistentTopic, 0), - StatsErrorCode.LAG_MEASUREMENT_FAILURE.code), - successfulRequests); + Runnable t10 = () -> assertEquals( + topicManager.getLatestOffsetCached(nonExistentTopic, 0), + StatsErrorCode.LAG_MEASUREMENT_FAILURE.code); + // get producer timestamp of last data message with retries for an existing topic - Runnable t11 = getAssertionTask( - () -> assertEquals( - topicManager.getProducerTimestampOfLastDataMessageWithRetries(topicPartition, 1), - tsOfLastDataMessage), - successfulRequests); + Runnable t11 = () -> assertEquals( + topicManager.getProducerTimestampOfLastDataMessageWithRetries(topicPartition, 1), + tsOfLastDataMessage); + // get producer timestamp of last data message with retries for a non-existent topic - Runnable t12 = getAssertionTask( - () -> assertThrows( - PubSubTopicDoesNotExistException.class, - () -> topicManager.getProducerTimestampOfLastDataMessageWithRetries(nonExistentTopicPartition, 1)), - successfulRequests); + Runnable t12 = () -> assertThrows( + PubSubTopicDoesNotExistException.class, + () -> topicManager.getProducerTimestampOfLastDataMessageWithRetries(nonExistentTopicPartition, 1)); + // get producer timestamp of last data message cached for an existing topic - Runnable t13 = getAssertionTask( - () -> assertEquals( - topicManager.getProducerTimestampOfLastDataMessageCached(topicPartition), - tsOfLastDataMessage), - successfulRequests); + Runnable t13 = () -> assertEquals( + topicManager.getProducerTimestampOfLastDataMessageCached(topicPartition), + tsOfLastDataMessage); + // get producer timestamp of last data message cached for a non-existent topic - Runnable t14 = getAssertionTask( - () -> assertEquals( - topicManager.getProducerTimestampOfLastDataMessageCached(nonExistentTopicPartition), - StatsErrorCode.LAG_MEASUREMENT_FAILURE.code), - successfulRequests); + Runnable t14 = () -> assertEquals( + topicManager.getProducerTimestampOfLastDataMessageCached(nonExistentTopicPartition), + StatsErrorCode.LAG_MEASUREMENT_FAILURE.code); + // get offset by time for an existing topic - Runnable t15 = getAssertionTask( - () -> assertEquals(topicManager.getOffsetByTime(topicPartition, System.currentTimeMillis()), numMessages), - successfulRequests); + Runnable t15 = + () -> assertEquals(topicManager.getOffsetByTime(topicPartition, System.currentTimeMillis()), numMessages); + // get offset by time for a non-existent topic - Runnable t16 = getAssertionTask( - () -> assertThrows( - PubSubTopicDoesNotExistException.class, - () -> topicManager.getOffsetByTime(nonExistentTopicPartition, tsOfLastDataMessage)), - successfulRequests); + Runnable t16 = () -> assertThrows( + PubSubTopicDoesNotExistException.class, + () -> topicManager.getOffsetByTime(nonExistentTopicPartition, tsOfLastDataMessage)); + // get offset by time for an existing topic: first message - Runnable t17 = getAssertionTask( - () -> assertEquals(topicManager.getOffsetByTime(topicPartition, timeBeforeProduce), 0), - successfulRequests); + Runnable t17 = () -> assertEquals(topicManager.getOffsetByTime(topicPartition, timeBeforeProduce), 0); - List tasks = Arrays.asList(t1, t2, t3, t4, t5, t6, t7, t8, t9, t10, t11, t12, t13, t14, t15, t16, t17); + // invalidate cache for an existing topic + Runnable t18 = () -> topicManager.invalidateCache(testTopic); + + // invalidate cache for a non-existent topic + Runnable t19 = () -> topicManager.invalidateCache(nonExistentTopic); + + List tasks = + Arrays.asList(t1, t2, t3, t4, t5, t6, t7, t8, t9, t10, t11, t12, t13, t14, t15, t16, t17, t18, t19); + + final AtomicInteger successfulRequests = new AtomicInteger(0); ExecutorService executorService = Executors.newFixedThreadPool(8); - List vwFutures = new ArrayList<>(); + List> futures = new ArrayList<>(); int totalTasks = 100; for (int i = 0; i < totalTasks; i++) { - Future future = executorService.submit(tasks.get(i % tasks.size())); - vwFutures.add(future); + int finalI = i; + futures.add( + CompletableFuture.runAsync(() -> tasks.get(finalI % tasks.size()).run(), executorService) + .thenAccept(v -> successfulRequests.incrementAndGet())); } - - int failedRequests = 0; - for (Future future: vwFutures) { - try { - future.get(3, TimeUnit.MINUTES); - } catch (Exception e) { - failedRequests++; - } - } - assertEquals(failedRequests, 0); + CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).get(5, TimeUnit.MINUTES); assertEquals(successfulRequests.get(), totalTasks); } - private static Runnable getAssertionTask(Runnable runnable, AtomicInteger successfulRequests) { - return () -> { - try { - runnable.run(); - successfulRequests.incrementAndGet(); - } catch (AssertionError e) { - e.printStackTrace(); - throw new AssertionError("Assertion failed: " + e.getMessage(), e); - } catch (Exception e) { - e.printStackTrace(); - throw new RuntimeException(e); - } - }; - } - @Test(timeOut = 3 * Time.MS_PER_MINUTE) public void testMetadataApisForNonExistentTopics() throws ExecutionException, InterruptedException, TimeoutException { PubSubTopic nonExistentTopic = pubSubTopicRepository.getTopic(Utils.getUniqueString("nonExistentTopic")); From 70be6aa13f323a5edbbb557b02f6999d178f4d8d Mon Sep 17 00:00:00 2001 From: Sushant Mane Date: Wed, 14 Feb 2024 13:35:26 -0800 Subject: [PATCH 4/8] Remove blocking close for TM and TMR --- .../venice/pubsub/manager/TopicManager.java | 15 +----------- .../manager/TopicManagerRepository.java | 23 ++----------------- .../manager/TopicManagerRepositoryTest.java | 11 ++++----- 3 files changed, 8 insertions(+), 41 deletions(-) diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java index f2ad9a5aea..2b02bc5023 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java @@ -47,9 +47,7 @@ import java.util.Properties; import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -713,18 +711,7 @@ public void close() { if (isClosed.get()) { logger.warn("{} is already closed", this); } - try { - closeAsync().get(2, TimeUnit.MINUTES); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - logger.error("Interrupted while closing: {}", this, e); - } catch (ExecutionException | TimeoutException e) { - logger.error("Failed to close: {}", this, e); - } - } - - CompletableFuture closeAsync() { - return CompletableFuture.runAsync(() -> { + CompletableFuture.runAsync(() -> { if (isClosed.compareAndSet(false, true)) { logger.info("Closing {}", this); Utils.closeQuietlyWithErrorLogged(topicMetadataFetcher); diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerRepository.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerRepository.java index 4a5b12198a..ac5c9fcee3 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerRepository.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerRepository.java @@ -3,15 +3,9 @@ import com.linkedin.venice.pubsub.api.PubSubTopic; import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap; import java.io.Closeable; -import java.util.ArrayList; import java.util.Collection; -import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -64,22 +58,9 @@ public void invalidateTopicManagerCaches(PubSubTopic pubSubTopic) { @Override public void close() { - long startTime = System.currentTimeMillis(); - List> closeFutures = new ArrayList<>(topicManagers.size()); for (TopicManager topicManager: topicManagers.values()) { - closeFutures.add(topicManager.closeAsync()); - } - try { - CompletableFuture.allOf(closeFutures.toArray(new CompletableFuture[0])).get(2, TimeUnit.MINUTES); - LOGGER.info( - "All TopicManagers in the TopicManagerRepository have been closed in {} ms", - System.currentTimeMillis() - startTime); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - LOGGER.error("Interrupted while waiting for TopicManagers to close", e); - } catch (ExecutionException | TimeoutException e) { - // log and ignore exception - LOGGER.error("Error when closing TopicManagerRepository", e); + topicManager.close(); } + LOGGER.info("TopicManagerRepository has started closing all TopicManagers in the repository."); } } diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicManagerRepositoryTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicManagerRepositoryTest.java index f424c6ae75..55f0d6ac68 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicManagerRepositoryTest.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicManagerRepositoryTest.java @@ -1,15 +1,14 @@ package com.linkedin.venice.pubsub.manager; +import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; -import java.util.concurrent.CompletableFuture; import org.testng.annotations.Test; @@ -54,10 +53,10 @@ public void testGetTopicManagerIsCreatedIfNotExists() { assertTrue(topicManagerRepositorySpy.getAllTopicManagers().contains(remoteTopicManager)); // test close - when(localTopicManager.closeAsync()).thenReturn(CompletableFuture.completedFuture(null)); - when(remoteTopicManager.closeAsync()).thenReturn(CompletableFuture.completedFuture(null)); + doNothing().when(localTopicManager).close(); + doNothing().when(remoteTopicManager).close(); topicManagerRepositorySpy.close(); - verify(localTopicManager).closeAsync(); - verify(remoteTopicManager).closeAsync(); + verify(localTopicManager).close(); + verify(remoteTopicManager).close(); } } From 3031915584ca2d6fc4701384556fbb13e3925719 Mon Sep 17 00:00:00 2001 From: Sushant Mane Date: Wed, 14 Feb 2024 15:09:13 -0800 Subject: [PATCH 5/8] Address Gaojie's review comments --- .../ActiveActiveStoreIngestionTask.java | 8 ++++---- .../LeaderFollowerStoreIngestionTask.java | 20 +++++++++---------- .../venice/pubsub/manager/TopicManager.java | 1 + 3 files changed, 15 insertions(+), 14 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java index e6dea86639..48e2d5700a 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java @@ -1395,12 +1395,12 @@ public long getRegionHybridOffsetLag(int regionId) { return 0; } - // Consumer might not existed after the consumption state is created, but before attaching the corresponding + // Consumer might not exist after the consumption state is created, but before attaching the corresponding // consumer. - long offsetLagOptional = + long lagBasedOnMetrics = getPartitionOffsetLagBasedOnMetrics(kafkaSourceAddress, currentLeaderTopic, pcs.getUserPartition()); - if (offsetLagOptional >= 0) { - return offsetLagOptional; + if (lagBasedOnMetrics >= 0) { + return lagBasedOnMetrics; } // Fall back to calculate offset lag in the old way diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java index 2921a64d1a..fad7ae8601 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/LeaderFollowerStoreIngestionTask.java @@ -2568,10 +2568,10 @@ public long getBatchReplicationLag() { String sourceKafkaURL = getSourceKafkaUrlForOffsetLagMeasurement(pcs); // Consumer might not exist after the consumption state is created, but before attaching the corresponding // consumer. - long offsetLagOptional = + long lagBasedOnMetrics = getPartitionOffsetLagBasedOnMetrics(sourceKafkaURL, currentLeaderTopic, pcs.getUserPartition()); - if (offsetLagOptional >= 0) { - return offsetLagOptional; + if (lagBasedOnMetrics >= 0) { + return lagBasedOnMetrics; } // Fall back to use the old way (latest VT offset in remote kafka - latest VT offset in local kafka) long localOffset = @@ -2625,10 +2625,10 @@ private long getLeaderOffsetLag(Predicate par final String kafkaSourceAddress = getSourceKafkaUrlForOffsetLagMeasurement(pcs); // Consumer might not exist after the consumption state is created, but before attaching the corresponding // consumer. - long offsetLagOptional = + long lagBasedOnMetrics = getPartitionOffsetLagBasedOnMetrics(kafkaSourceAddress, currentLeaderTopic, pcs.getPartition()); - if (offsetLagOptional >= 0) { - return offsetLagOptional; + if (lagBasedOnMetrics >= 0) { + return lagBasedOnMetrics; } // Fall back to calculate offset lag in the original approach @@ -2700,12 +2700,12 @@ private long getFollowerOffsetLag(Predicate p .filter(partitionConsumptionStateFilter) // the lag is (latest VT offset - consumed VT offset) .mapToLong((pcs) -> { - // Consumer might not existed after the consumption state is created, but before attaching the corresponding + // Consumer might not exist after the consumption state is created, but before attaching the corresponding // consumer. - long offsetLagOptional = + long lagBasedOnMetrics = getPartitionOffsetLagBasedOnMetrics(localKafkaServer, versionTopic, pcs.getPartition()); - if (offsetLagOptional >= 0) { - return offsetLagOptional; + if (lagBasedOnMetrics >= 0) { + return lagBasedOnMetrics; } // Fall back to calculate offset lag in the old way return measureLagWithCallToPubSub( diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java index 2b02bc5023..91a364e830 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java @@ -710,6 +710,7 @@ public String getPubSubClusterAddress() { public void close() { if (isClosed.get()) { logger.warn("{} is already closed", this); + return; } CompletableFuture.runAsync(() -> { if (isClosed.compareAndSet(false, true)) { From 3bca98fea0e604f88571de3c0037a2943c01b42a Mon Sep 17 00:00:00 2001 From: Sushant Mane Date: Wed, 14 Feb 2024 15:31:50 -0800 Subject: [PATCH 6/8] use micro second as unit for pubsub admin latencies --- .../consumer/AggKafkaConsumerService.java | 7 ++++- .../venice/pubsub/manager/TopicManager.java | 31 +++++++++---------- .../pubsub/manager/TopicManagerStats.java | 5 +-- .../pubsub/manager/TopicMetadataFetcher.java | 18 +++++------ 4 files changed, 33 insertions(+), 28 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/AggKafkaConsumerService.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/AggKafkaConsumerService.java index 2bea3df298..94dd53f448 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/AggKafkaConsumerService.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/AggKafkaConsumerService.java @@ -393,8 +393,13 @@ public ConsumedDataReceiver minIsr, boolean useFastPubSubOperationTimeout) { - - long startTime = System.currentTimeMillis(); - long deadlineMs = startTime + (useFastPubSubOperationTimeout + long startTimeMs = System.currentTimeMillis(); + long deadlineMs = startTimeMs + (useFastPubSubOperationTimeout ? PUBSUB_FAST_OPERATION_TIMEOUT_MS : topicManagerContext.getPubSubOperationTimeoutMs()); PubSubTopicConfiguration pubSubTopicConfiguration = new PubSubTopicConfiguration( @@ -213,7 +212,7 @@ public void createTopic( } else { throw new PubSubOpTimeoutException( "Timeout while creating topic: " + topicName + ". Topic still does not exist after " - + (deadlineMs - startTime) + "ms.", + + (deadlineMs - startTimeMs) + "ms.", e); } } @@ -223,12 +222,12 @@ public void createTopic( } protected void waitUntilTopicCreated(PubSubTopic topicName, int partitionCount, long deadlineMs) { - long startTime = System.currentTimeMillis(); + long startTimeMs = System.nanoTime(); while (!containsTopicAndAllPartitionsAreOnline(topicName, partitionCount)) { if (System.currentTimeMillis() > deadlineMs) { throw new PubSubOpTimeoutException( "Timeout while creating topic: " + topicName + ". Topic still did not pass all the checks after " - + (deadlineMs - startTime) + "ms."); + + (deadlineMs - startTimeMs) + "ms."); } Utils.sleep(200); } @@ -369,7 +368,7 @@ public boolean updateTopicMinInSyncReplica(PubSubTopic topicName, int minISR) * @return a map of topic name to retention time in MS. */ public Map getAllTopicRetentions() { - long startTime = System.currentTimeMillis(); + long startTime = System.nanoTime(); Map topicRetentions = pubSubAdminAdapter.getAllTopicRetentions(); stats.recordLatency(GET_ALL_TOPIC_RETENTIONS, startTime); return topicRetentions; @@ -414,13 +413,13 @@ private void createTopic( int numPartitions, int replicationFactor, PubSubTopicConfiguration topicConfiguration) { - long startTime = System.currentTimeMillis(); + long startTime = System.nanoTime(); pubSubAdminAdapter.createTopic(pubSubTopic, numPartitions, replicationFactor, topicConfiguration); stats.recordLatency(CREATE_TOPIC, startTime); } private void setTopicConfig(PubSubTopic pubSubTopic, PubSubTopicConfiguration pubSubTopicConfiguration) { - long startTime = System.currentTimeMillis(); + long startTime = System.nanoTime(); pubSubAdminAdapter.setTopicConfig(pubSubTopic, pubSubTopicConfiguration); stats.recordLatency(SET_TOPIC_CONFIG, startTime); } @@ -429,7 +428,7 @@ private void setTopicConfig(PubSubTopic pubSubTopic, PubSubTopicConfiguration pu * This operation is a little heavy, since it will pull the configs for all the topics. */ public PubSubTopicConfiguration getTopicConfig(PubSubTopic pubSubTopic) { - long startTime = System.currentTimeMillis(); + long startTime = System.nanoTime(); PubSubTopicConfiguration pubSubTopicConfiguration = pubSubAdminAdapter.getTopicConfig(pubSubTopic); topicConfigCache.put(pubSubTopic, pubSubTopicConfiguration); stats.recordLatency(GET_TOPIC_CONFIG, startTime); @@ -437,7 +436,7 @@ public PubSubTopicConfiguration getTopicConfig(PubSubTopic pubSubTopic) { } public PubSubTopicConfiguration getTopicConfigWithRetry(PubSubTopic topicName) { - long startTime = System.currentTimeMillis(); + long startTime = System.nanoTime(); PubSubTopicConfiguration pubSubTopicConfiguration = pubSubAdminAdapter.getTopicConfigWithRetry(topicName); topicConfigCache.put(topicName, pubSubTopicConfiguration); stats.recordLatency(GET_TOPIC_CONFIG_WITH_RETRY, startTime); @@ -457,7 +456,7 @@ public PubSubTopicConfiguration getCachedTopicConfig(PubSubTopic topicName) { } public Map getSomeTopicConfigs(Set topicNames) { - long startTime = System.currentTimeMillis(); + long startTime = System.nanoTime(); Map topicConfigs = pubSubAdminAdapter.getSomeTopicConfigs(topicNames); for (Map.Entry topicConfig: topicConfigs.entrySet()) { topicConfigCache.put(topicConfig.getKey(), topicConfig.getValue()); @@ -478,7 +477,7 @@ public void ensureTopicIsDeletedAndBlock(PubSubTopic pubSubTopic) { logger.info("Deleting topic: {}", pubSubTopic); try { - long startTime = System.currentTimeMillis(); + long startTime = System.nanoTime(); pubSubAdminAdapter.deleteTopic(pubSubTopic, Duration.ofMillis(topicManagerContext.getPubSubOperationTimeoutMs())); stats.recordLatency(DELETE_TOPIC, startTime); logger.info("Topic: {} has been deleted", pubSubTopic); @@ -531,7 +530,7 @@ public void ensureTopicIsDeletedAndBlockWithRetry(PubSubTopic pubSubTopic) { // TODO: Evaluate if we need synchronized here public synchronized Set listTopics() { - long startTime = System.currentTimeMillis(); + long startTime = System.nanoTime(); Set topics = pubSubAdminAdapter.listAllTopics(); stats.recordLatency(LIST_ALL_TOPICS, startTime); return topics; @@ -545,7 +544,7 @@ public boolean containsTopicWithExpectationAndRetry( PubSubTopic topic, int maxAttempts, final boolean expectedResult) { - long startTime = System.currentTimeMillis(); + long startTime = System.nanoTime(); boolean containsTopic = pubSubAdminAdapter.containsTopicWithExpectationAndRetry(topic, maxAttempts, expectedResult); stats.recordLatency(CONTAINS_TOPIC_WITH_RETRY, startTime); return containsTopic; @@ -558,7 +557,7 @@ public boolean containsTopicWithExpectationAndRetry( Duration initialBackoff, Duration maxBackoff, Duration maxDuration) { - long startTime = System.currentTimeMillis(); + long startTime = System.nanoTime(); boolean containsTopic = pubSubAdminAdapter.containsTopicWithExpectationAndRetry( topic, maxAttempts, diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerStats.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerStats.java index ebfb797711..fb21373b94 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerStats.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerStats.java @@ -49,10 +49,11 @@ EnumMap getSensorsByTypes() { return sensorsByTypes; } - void recordLatency(SENSOR_TYPE sensorType, long startTimeInMs) { + void recordLatency(SENSOR_TYPE sensorType, long startTimeInNs) { if (sensorsByTypes == null || sensorType == null) { return; } - sensorsByTypes.get(sensorType).record(System.currentTimeMillis() - startTimeInMs); + // convert ns to us and record + sensorsByTypes.get(sensorType).record((System.nanoTime() - startTimeInNs) >> 10); } } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java index 0f621ed892..44ab610450 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java @@ -240,7 +240,7 @@ public void close() throws IOException { * @return true if the topic exists, false otherwise */ boolean containsTopic(PubSubTopic topic) { - long startTime = System.currentTimeMillis(); + long startTime = System.nanoTime(); boolean containsTopic = pubSubAdminAdapter.containsTopic(topic); stats.recordLatency(CONTAINS_TOPIC, startTime); return containsTopic; @@ -265,7 +265,7 @@ boolean containsTopicCached(PubSubTopic topic) { Int2LongMap getTopicLatestOffsets(PubSubTopic topic) { PubSubConsumerAdapter pubSubConsumerAdapter = acquireConsumer(); try { - long startTime = System.currentTimeMillis(); + long startTime = System.nanoTime(); List partitionInfoList = pubSubConsumerAdapter.partitionsFor(topic); stats.recordLatency(PARTITIONS_FOR, startTime); @@ -279,7 +279,7 @@ Int2LongMap getTopicLatestOffsets(PubSubTopic topic) { topicPartitions.add(partitionInfo.getTopicPartition()); } - startTime = System.currentTimeMillis(); + startTime = System.nanoTime(); Map offsetsMap = pubSubConsumerAdapter.endOffsets(topicPartitions, PUBSUB_OFFSET_API_TIMEOUT_DURATION_DEFAULT_VALUE); stats.recordLatency(GET_TOPIC_LATEST_OFFSETS, startTime); @@ -301,7 +301,7 @@ Int2LongMap getTopicLatestOffsets(PubSubTopic topic) { List getTopicPartitionInfo(PubSubTopic topic) { PubSubConsumerAdapter pubSubConsumerAdapter = acquireConsumer(); try { - long startTime = System.currentTimeMillis(); + long startTime = System.nanoTime(); List res = pubSubConsumerAdapter.partitionsFor(topic); stats.recordLatency(PARTITIONS_FOR, startTime); return res; @@ -325,7 +325,7 @@ List getTopicPartitionInfo(PubSubTopic topic) { long getLatestOffset(PubSubTopicPartition pubSubTopicPartition) { PubSubConsumerAdapter pubSubConsumerAdapter = acquireConsumer(); try { - long startTime = System.currentTimeMillis(); + long startTime = System.nanoTime(); Map offsetMap = pubSubConsumerAdapter .endOffsets(Collections.singleton(pubSubTopicPartition), PUBSUB_OFFSET_API_TIMEOUT_DURATION_DEFAULT_VALUE); stats.recordLatency(GET_PARTITION_LATEST_OFFSETS, startTime); @@ -398,7 +398,7 @@ long getOffsetForTime(PubSubTopicPartition pubSubTopicPartition, long timestamp) PubSubConsumerAdapter pubSubConsumerAdapter = acquireConsumer(); try { - long startTime = System.currentTimeMillis(); + long startTime = System.nanoTime(); Long result = pubSubConsumerAdapter .offsetForTime(pubSubTopicPartition, timestamp, PUBSUB_OFFSET_API_TIMEOUT_DURATION_DEFAULT_VALUE); stats.recordLatency(GET_OFFSET_FOR_TIME, startTime); @@ -447,7 +447,7 @@ long getProducerTimestampOfLastDataMessage(PubSubTopicPartition pubSubTopicParti int fetchSize = 10; int totalAttempts = 3; int fetchedRecordsCount; - long startTime = System.currentTimeMillis(); + long startTime = System.nanoTime(); do { List> lastConsumedRecords = consumeLatestRecords(pubSubTopicPartition, fetchSize); @@ -618,7 +618,7 @@ CompletableFuture invalidateKeyAsync(PubSubTopic pubSubTopic) { } void invalidateKey(PubSubTopic pubSubTopic) { - long startTime = System.currentTimeMillis(); + long startTime = System.nanoTime(); LOGGER.info("Invalidating cache for topic: {}", pubSubTopic); topicExistenceCache.remove(pubSubTopic); Set topicPartitions = new HashSet<>(); @@ -638,7 +638,7 @@ void invalidateKey(PubSubTopic pubSubTopic) { invalidateKey(pubSubTopicPartition); } - LOGGER.info("Invalidated cache for topic: {} in {} ms", pubSubTopic, System.currentTimeMillis() - startTime); + LOGGER.info("Invalidated cache for topic: {} in {} ns", pubSubTopic, System.nanoTime() - startTime); } /** From 8c149b907f6e31df18b152210c815648527b2119 Mon Sep 17 00:00:00 2001 From: Sushant Mane Date: Wed, 14 Feb 2024 16:09:07 -0800 Subject: [PATCH 7/8] add defensive check in md consumer to ensure there are no subscriptions --- .../pubsub/manager/TopicMetadataFetcher.java | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java index 44ab610450..7b838d8a05 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java @@ -563,6 +563,7 @@ List> consumeLatestRecords( } // consume latest records + ensureConsumerHasNoSubscriptions(pubSubConsumerAdapter); long consumePastOffset = Math.max(Math.max(latestOffset - lastRecordsCount, earliestOffset) - 1, -1); pubSubConsumerAdapter.subscribe(pubSubTopicPartition, consumePastOffset); subscribed = true; @@ -607,6 +608,19 @@ List> consumeLatestRecords( } } + void ensureConsumerHasNoSubscriptions(PubSubConsumerAdapter pubSubConsumerAdapter) { + Set assignedPartitions = pubSubConsumerAdapter.getAssignment(); + if (assignedPartitions.isEmpty()) { + return; + } + LOGGER.warn( + "Consumer: {} of has lingering subscriptions: {}. Unsubscribing from all of them." + " Consumer belongs to {}", + pubSubConsumerAdapter, + assignedPartitions, + this); + pubSubConsumerAdapter.batchUnsubscribe(assignedPartitions); + } + void invalidateKey(PubSubTopicPartition pubSubTopicPartition) { latestOffsetCache.remove(pubSubTopicPartition); lastProducerTimestampCache.remove(pubSubTopicPartition); From f98e164b219529b12cb9b311e5656239daedd128 Mon Sep 17 00:00:00 2001 From: Sushant Mane Date: Wed, 14 Feb 2024 18:28:08 -0800 Subject: [PATCH 8/8] use LatencyUtils.getLatencyInMS --- .../com/linkedin/venice/pubsub/manager/TopicManagerStats.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerStats.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerStats.java index fb21373b94..bd9eb1ff4f 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerStats.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManagerStats.java @@ -2,6 +2,7 @@ import com.linkedin.venice.stats.AbstractVeniceStats; import com.linkedin.venice.stats.TehutiUtils; +import com.linkedin.venice.utils.LatencyUtils; import io.tehuti.metrics.MetricsRepository; import io.tehuti.metrics.Sensor; import io.tehuti.metrics.stats.Avg; @@ -54,6 +55,6 @@ void recordLatency(SENSOR_TYPE sensorType, long startTimeInNs) { return; } // convert ns to us and record - sensorsByTypes.get(sensorType).record((System.nanoTime() - startTimeInNs) >> 10); + sensorsByTypes.get(sensorType).record(LatencyUtils.getLatencyInMS(startTimeInNs)); } }