diff --git a/.gitignore b/.gitignore index ce7b4b56264..77dc929bbc5 100644 --- a/.gitignore +++ b/.gitignore @@ -17,6 +17,7 @@ build/ internal/venice-test-common/tmp/ internal/venice-test-common/dump.complete.hprof internal/venice-test-common/src/jmh/generated +internal/venice-test-common/file:/ .jekyll-cache/ _site/ Gemfile.lock diff --git a/build.gradle b/build.gradle index 23ef82d2ab2..e6ad01275bc 100644 --- a/build.gradle +++ b/build.gradle @@ -125,7 +125,7 @@ ext.libraries = [ snappy: 'org.iq80.snappy:snappy:0.4', spark: 'com.sparkjava:spark-core:2.9.4', // Spark-Java Rest framework spotbugs: 'com.github.spotbugs:spotbugs:4.5.2', - tehuti: 'io.tehuti:tehuti:0.11.4', + tehuti: 'io.tehuti:tehuti:0.12.2', testcontainers: 'org.testcontainers:testcontainers:1.18.0', testng: 'org.testng:testng:6.14.3', tomcatAnnotations: 'org.apache.tomcat:annotations-api:6.0.53', @@ -805,7 +805,9 @@ task verifyJdkVersion { def currentJdkVersion = JavaVersion.current() def isSupported = supportedJdkVersions.any {version -> currentJdkVersion.equals(version) } if (!isSupported) { - throw new GradleException("Invalid JDK version: ${currentJdkVersion}. Supported versions: ${supportedJdkVersions.join(', ')}.") + throw new GradleException("Invalid JDK version: ${currentJdkVersion}.\n" + \ + "Supported versions: ${supportedJdkVersions.join(', ')}.\n" + \ + "Please set the JAVA_HOME environment variable to a supported version either locally or globally.") } println "JDK version ${currentJdkVersion} is valid." } 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 096adeac537..43c44ac08cb 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 @@ -50,6 +50,8 @@ import static com.linkedin.venice.ConfigKeys.PUBSUB_TOPIC_MANAGER_METADATA_FETCHER_THREAD_POOL_SIZE; import static com.linkedin.venice.ConfigKeys.ROUTER_PRINCIPAL_NAME; import static com.linkedin.venice.ConfigKeys.SERVER_AA_WC_LEADER_QUOTA_RECORDS_PER_SECOND; +import static com.linkedin.venice.ConfigKeys.SERVER_AA_WC_WORKLOAD_PARALLEL_PROCESSING_ENABLED; +import static com.linkedin.venice.ConfigKeys.SERVER_AA_WC_WORKLOAD_PARALLEL_PROCESSING_THREAD_POOL_SIZE; import static com.linkedin.venice.ConfigKeys.SERVER_BATCH_REPORT_END_OF_INCREMENTAL_PUSH_STATUS_ENABLED; import static com.linkedin.venice.ConfigKeys.SERVER_BLOCKING_QUEUE_TYPE; import static com.linkedin.venice.ConfigKeys.SERVER_CHANNEL_OPTION_WRITE_BUFFER_WATERMARK_HIGH_BYTES; @@ -538,6 +540,8 @@ public class VeniceServerConfig extends VeniceClusterConfig { private final int nonCurrentVersionAAWCLeaderQuotaRecordsPerSecond; private final int nonCurrentVersionNonAAWCLeaderQuotaRecordsPerSecond; private final int channelOptionWriteBufferHighBytes; + private final boolean aaWCWorkloadParallelProcessingEnabled; + private final int aaWCWorkloadParallelProcessingThreadPoolSize; public VeniceServerConfig(VeniceProperties serverProperties) throws ConfigurationException { this(serverProperties, Collections.emptyMap()); @@ -899,6 +903,10 @@ public VeniceServerConfig(VeniceProperties serverProperties, Map keyLevelLocksManager; private final AggVersionedIngestionStats aggVersionedIngestionStats; private final RemoteIngestionRepairService remoteIngestionRepairService; + private final Lazy ingestionBatchProcessorLazy; private static class ReusableObjects { // reuse buffer for rocksDB value object @@ -128,15 +130,13 @@ public ActiveActiveStoreIngestionTask( this.aggVersionedIngestionStats = versionedIngestionStats; int knownKafkaClusterNumber = serverConfig.getKafkaClusterIdToUrlMap().size(); - int consumerPoolSizePerKafkaCluster = serverConfig.getConsumerPoolSizePerKafkaCluster(); + int initialPoolSize = knownKafkaClusterNumber + 1; - /** - * In theory, the maximum # of keys each ingestion task can process is the # of consumers allocated for it. - */ - int maxKeyLevelLocksPoolSize = - Math.min(storeVersionPartitionCount, consumerPoolSizePerKafkaCluster) * knownKafkaClusterNumber + 1; - this.keyLevelLocksManager = - Lazy.of(() -> new KeyLevelLocksManager(getVersionTopic().getName(), initialPoolSize, maxKeyLevelLocksPoolSize)); + this.keyLevelLocksManager = Lazy.of( + () -> new KeyLevelLocksManager( + getVersionTopic().getName(), + initialPoolSize, + getKeyLevelLockMaxPoolSizeBasedOnServerConfig(serverConfig, storeVersionPartitionCount))); StringAnnotatedStoreSchemaCache annotatedReadOnlySchemaRepository = new StringAnnotatedStoreSchemaCache(storeName, schemaRepository); @@ -152,22 +152,55 @@ public ActiveActiveStoreIngestionTask( isWriteComputationEnabled, getServerConfig().isComputeFastAvroEnabled()); this.remoteIngestionRepairService = builder.getRemoteIngestionRepairService(); + this.ingestionBatchProcessorLazy = Lazy.of(() -> { + if (!serverConfig.isAAWCWorkloadParallelProcessingEnabled()) { + LOGGER.info("AA/WC workload parallel processing enabled is false"); + return null; + } + LOGGER.info("AA/WC workload parallel processing enabled is true"); + return new IngestionBatchProcessor( + kafkaVersionTopic, + parallelProcessingThreadPool, + null, + this::processActiveActiveMessage, + isWriteComputationEnabled, + isActiveActiveReplicationEnabled()); + }); + } + + public static int getKeyLevelLockMaxPoolSizeBasedOnServerConfig(VeniceServerConfig serverConfig, int partitionCount) { + int consumerPoolSizeForLeaderConsumption = 0; + if (serverConfig.isDedicatedConsumerPoolForAAWCLeaderEnabled()) { + consumerPoolSizeForLeaderConsumption = serverConfig.getDedicatedConsumerPoolSizeForAAWCLeader(); + } else if (serverConfig.getConsumerPoolStrategyType() + .equals(KafkaConsumerServiceDelegator.ConsumerPoolStrategyType.CURRENT_VERSION_PRIORITIZATION)) { + consumerPoolSizeForLeaderConsumption = serverConfig.getConsumerPoolSizeForCurrentVersionAAWCLeader() + + serverConfig.getConsumerPoolSizeForNonCurrentVersionAAWCLeader(); + } else { + consumerPoolSizeForLeaderConsumption = serverConfig.getConsumerPoolSizePerKafkaCluster(); + } + int multiplier = 1; + if (serverConfig.isAAWCWorkloadParallelProcessingEnabled()) { + multiplier = serverConfig.getAAWCWorkloadParallelProcessingThreadPoolSize(); + } + return Math.min(partitionCount, consumerPoolSizeForLeaderConsumption) + * serverConfig.getKafkaClusterIdToUrlMap().size() * multiplier + 1; } @Override protected DelegateConsumerRecordResult delegateConsumerRecord( - PubSubMessage consumerRecord, + PubSubMessageProcessedResultWrapper consumerRecordWrapper, int partition, String kafkaUrl, int kafkaClusterId, long beforeProcessingPerRecordTimestampNs, long beforeProcessingBatchRecordsTimestampMs) { - if (!consumerRecord.getTopicPartition().getPubSubTopic().isRealTime()) { + if (!consumerRecordWrapper.getMessage().getTopicPartition().getPubSubTopic().isRealTime()) { /** * We don't need to lock the partition here because during VT consumption there is only one consumption source. */ return super.delegateConsumerRecord( - consumerRecord, + consumerRecordWrapper, partition, kafkaUrl, kafkaClusterId, @@ -185,12 +218,12 @@ protected DelegateConsumerRecordResult delegateConsumerRecord( * -> [fabric B thread]update transient record cache -> [fabric B thread]produce to VT -> [fabric A thread]update transient record cache * -> [fabric A thread]produce to VT */ - final ByteArrayKey byteArrayKey = ByteArrayKey.wrap(consumerRecord.getKey().getKey()); + final ByteArrayKey byteArrayKey = ByteArrayKey.wrap(consumerRecordWrapper.getMessage().getKey().getKey()); ReentrantLock keyLevelLock = this.keyLevelLocksManager.get().acquireLockByKey(byteArrayKey); keyLevelLock.lock(); try { return super.delegateConsumerRecord( - consumerRecord, + consumerRecordWrapper, partition, kafkaUrl, kafkaClusterId, @@ -206,6 +239,7 @@ protected DelegateConsumerRecordResult delegateConsumerRecord( @Override protected void putInStorageEngine(int partition, byte[] keyBytes, Put put) { try { + // TODO: Honor BatchConflictResolutionPolicy and maybe persist RMD for batch push records. StorageOperationType storageOperationType = getStorageOperationType(partition, put.putValue, put.replicationMetadataPayload); @@ -360,9 +394,12 @@ byte[] getRmdWithValueSchemaByteBufferFromStorage( return result.serialize(); } - // This function may modify the original record in KME, it is unsafe to use the payload from KME directly after - // this function. - protected void processMessageAndMaybeProduceToKafka( + @Override + protected IngestionBatchProcessor getIngestionBatchProcessor() { + return ingestionBatchProcessorLazy.get(); + } + + private PubSubMessageProcessedResult processActiveActiveMessage( PubSubMessage consumerRecord, PartitionConsumptionState partitionConsumptionState, int partition, @@ -370,24 +407,6 @@ protected void processMessageAndMaybeProduceToKafka( int kafkaClusterId, long beforeProcessingRecordTimestampNs, long beforeProcessingBatchRecordsTimestampMs) { - /** - * With {@link BatchConflictResolutionPolicy.BATCH_WRITE_LOSES} there is no need - * to perform DCR before EOP and L/F DIV passthrough mode should be used. If the version is going through data - * recovery then there is no need to perform DCR until we completed data recovery and switched to consume from RT. - * TODO. We need to refactor this logic when we support other batch conflict resolution policy. - */ - if (!partitionConsumptionState.isEndOfPushReceived() - || isDataRecovery && partitionConsumptionState.getTopicSwitch() != null) { - super.processMessageAndMaybeProduceToKafka( - consumerRecord, - partitionConsumptionState, - partition, - kafkaUrl, - kafkaClusterId, - beforeProcessingRecordTimestampNs, - beforeProcessingBatchRecordsTimestampMs); - return; - } KafkaKey kafkaKey = consumerRecord.getKey(); KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); byte[] keyBytes = kafkaKey.getKey(); @@ -421,6 +440,14 @@ protected void processMessageAndMaybeProduceToKafka( consumerRecord.getTopicPartition(), valueManifestContainer, beforeProcessingBatchRecordsTimestampMs)); + if (hasChangeCaptureView) { + /** + * Since this function will update the transient cache before writing the view, and if there is + * a change capture view writer, we need to lookup first, otherwise the transient cache will be populated + * when writing to the view after this function. + */ + oldValueProvider.get(); + } final RmdWithValueSchemaId rmdWithValueSchemaID = getReplicationMetadataAndSchemaId( partitionConsumptionState, @@ -447,7 +474,7 @@ protected void processMessageAndMaybeProduceToKafka( switch (msgType) { case PUT: mergeConflictResult = mergeConflictResolver.put( - unwrapByteBufferFromOldValueProvider(oldValueProvider), + oldValueByteBufferProvider, rmdWithValueSchemaID, ((Put) kafkaValue.payloadUnion).putValue, writeTimestamp, @@ -455,8 +482,8 @@ protected void processMessageAndMaybeProduceToKafka( sourceOffset, kafkaClusterId, kafkaClusterId // Use the kafka cluster ID as the colo ID for now because one colo/fabric has only one - // Kafka cluster. TODO: evaluate whether it is enough this way, or we need to add a new - // config to represent the mapping from Kafka server URLs to colo ID. + // Kafka cluster. TODO: evaluate whether it is enough this way, or we need to add a new + // config to represent the mapping from Kafka server URLs to colo ID. ); getHostLevelIngestionStats() .recordIngestionActiveActivePutLatency(LatencyUtils.getElapsedTimeFromNSToMS(beforeDCRTimestampInNs)); @@ -499,9 +526,107 @@ protected void processMessageAndMaybeProduceToKafka( // Record the last ignored offset partitionConsumptionState .updateLatestIgnoredUpstreamRTOffset(kafkaClusterIdToUrlMap.get(kafkaClusterId), sourceOffset); + return new PubSubMessageProcessedResult( + new MergeConflictResultWrapper( + mergeConflictResult, + oldValueProvider, + oldValueByteBufferProvider, + rmdWithValueSchemaID, + valueManifestContainer, + null, + null)); } else { validatePostOperationResultsAndRecord(mergeConflictResult, offsetSumPreOperation, recordTimestampsPreOperation); + final ByteBuffer updatedValueBytes = maybeCompressData( + consumerRecord.getTopicPartition().getPartitionNumber(), + mergeConflictResult.getNewValue(), + partitionConsumptionState); + + final int valueSchemaId = mergeConflictResult.getValueSchemaId(); + + GenericRecord rmdRecord = mergeConflictResult.getRmdRecord(); + final ByteBuffer updatedRmdBytes = + rmdSerDe.serializeRmdRecord(mergeConflictResult.getValueSchemaId(), mergeConflictResult.getRmdRecord()); + + if (updatedValueBytes == null) { + hostLevelIngestionStats.recordTombstoneCreatedDCR(); + aggVersionedIngestionStats.recordTombStoneCreationDCR(storeName, versionNumber); + partitionConsumptionState + .setTransientRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, valueSchemaId, rmdRecord); + } else { + int valueLen = updatedValueBytes.remaining(); + partitionConsumptionState.setTransientRecord( + kafkaClusterId, + consumerRecord.getOffset(), + keyBytes, + updatedValueBytes.array(), + updatedValueBytes.position(), + valueLen, + valueSchemaId, + rmdRecord); + } + return new PubSubMessageProcessedResult( + new MergeConflictResultWrapper( + mergeConflictResult, + oldValueProvider, + oldValueByteBufferProvider, + rmdWithValueSchemaID, + valueManifestContainer, + updatedValueBytes, + updatedRmdBytes)); + } + } + + // This function may modify the original record in KME, it is unsafe to use the payload from KME directly after + // this function. + protected void processMessageAndMaybeProduceToKafka( + PubSubMessageProcessedResultWrapper consumerRecordWrapper, + PartitionConsumptionState partitionConsumptionState, + int partition, + String kafkaUrl, + int kafkaClusterId, + long beforeProcessingRecordTimestampNs, + long beforeProcessingBatchRecordsTimestampMs) { + /** + * With {@link BatchConflictResolutionPolicy.BATCH_WRITE_LOSES} there is no need + * to perform DCR before EOP and L/F DIV passthrough mode should be used. If the version is going through data + * recovery then there is no need to perform DCR until we completed data recovery and switched to consume from RT. + * TODO. We need to refactor this logic when we support other batch conflict resolution policy. + */ + if (!partitionConsumptionState.isEndOfPushReceived() + || isDataRecovery && partitionConsumptionState.getTopicSwitch() != null) { + super.processMessageAndMaybeProduceToKafka( + consumerRecordWrapper, + partitionConsumptionState, + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingRecordTimestampNs, + beforeProcessingBatchRecordsTimestampMs); + return; + } + PubSubMessage consumerRecord = consumerRecordWrapper.getMessage(); + KafkaKey kafkaKey = consumerRecord.getKey(); + // KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); + byte[] keyBytes = kafkaKey.getKey(); + final MergeConflictResultWrapper mergeConflictResultWrapper; + if (consumerRecordWrapper.getProcessedResult() != null + && consumerRecordWrapper.getProcessedResult().getMergeConflictResultWrapper() != null) { + mergeConflictResultWrapper = consumerRecordWrapper.getProcessedResult().getMergeConflictResultWrapper(); + } else { + mergeConflictResultWrapper = processActiveActiveMessage( + consumerRecord, + partitionConsumptionState, + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingRecordTimestampNs, + beforeProcessingBatchRecordsTimestampMs).getMergeConflictResultWrapper(); + } + + MergeConflictResult mergeConflictResult = mergeConflictResultWrapper.getMergeConflictResult(); + if (!mergeConflictResult.isUpdateIgnored()) { // Apply this update to any views for this store // TODO: It'd be good to be able to do this in LeaderFollowerStoreIngestionTask instead, however, AA currently is // the @@ -524,8 +649,9 @@ protected void processMessageAndMaybeProduceToKafka( int index = 0; // The first future is for the previous write to VT viewWriterFutures[index++] = partitionConsumptionState.getLastVTProduceCallFuture(); - ByteBuffer oldValueBB = oldValueByteBufferProvider.get(); - int oldValueSchemaId = oldValueBB == null ? -1 : oldValueProvider.get().writerSchemaId(); + ByteBuffer oldValueBB = mergeConflictResultWrapper.getOldValueByteBufferProvider().get(); + int oldValueSchemaId = + oldValueBB == null ? -1 : mergeConflictResultWrapper.getOldValueProvider().get().writerSchemaId(); for (VeniceViewWriter writer: viewWriters.values()) { viewWriterFutures[index++] = writer.processRecord( mergeConflictResult.getNewValue(), @@ -540,16 +666,14 @@ protected void processMessageAndMaybeProduceToKafka( hostLevelIngestionStats.recordViewProducerLatency(LatencyUtils.getElapsedTimeFromMsToMs(preprocessingTime)); if (exception == null) { producePutOrDeleteToKafka( - mergeConflictResult, + mergeConflictResultWrapper, partitionConsumptionState, keyBytes, consumerRecord, partition, kafkaUrl, kafkaClusterId, - beforeProcessingRecordTimestampNs, - valueManifestContainer.getManifest(), - rmdWithValueSchemaID == null ? null : rmdWithValueSchemaID.getRmdManifest()); + beforeProcessingRecordTimestampNs); currentVersionTopicWrite.complete(null); } else { VeniceException veniceException = new VeniceException(exception); @@ -563,16 +687,14 @@ protected void processMessageAndMaybeProduceToKafka( // after // this call. producePutOrDeleteToKafka( - mergeConflictResult, + mergeConflictResultWrapper, partitionConsumptionState, keyBytes, consumerRecord, partition, kafkaUrl, kafkaClusterId, - beforeProcessingRecordTimestampNs, - valueManifestContainer.getManifest(), - rmdWithValueSchemaID == null ? null : rmdWithValueSchemaID.getRmdManifest()); + beforeProcessingRecordTimestampNs); } } } @@ -702,7 +824,7 @@ ByteBuffer getCurrentValueFromTransientRecord(PartitionConsumptionState.Transien * This function may modify the original record in KME and it is unsafe to use the payload from KME directly after * this function. * - * @param mergeConflictResult The result of conflict resolution. + * @param mergeConflictResultWrapper The result of conflict resolution. * @param partitionConsumptionState The {@link PartitionConsumptionState} of the current partition * @param key The key bytes of the incoming record. * @param consumerRecord The {@link PubSubMessage} for the current record. @@ -710,37 +832,31 @@ ByteBuffer getCurrentValueFromTransientRecord(PartitionConsumptionState.Transien * @param kafkaUrl */ private void producePutOrDeleteToKafka( - MergeConflictResult mergeConflictResult, + MergeConflictResultWrapper mergeConflictResultWrapper, PartitionConsumptionState partitionConsumptionState, byte[] key, PubSubMessage consumerRecord, int partition, String kafkaUrl, int kafkaClusterId, - long beforeProcessingRecordTimestampNs, - ChunkedValueManifest oldValueManifest, - ChunkedValueManifest oldRmdManifest) { - - final ByteBuffer updatedValueBytes = maybeCompressData( - consumerRecord.getTopicPartition().getPartitionNumber(), - mergeConflictResult.getNewValue(), - partitionConsumptionState); - + long beforeProcessingRecordTimestampNs) { + MergeConflictResult mergeConflictResult = mergeConflictResultWrapper.getMergeConflictResult(); + ByteBuffer updatedValueBytes = mergeConflictResultWrapper.getUpdatedValueBytes(); + ByteBuffer updatedRmdBytes = mergeConflictResultWrapper.getUpdatedRmdBytes(); final int valueSchemaId = mergeConflictResult.getValueSchemaId(); - GenericRecord rmdRecord = mergeConflictResult.getRmdRecord(); - final ByteBuffer updatedRmdBytes = - rmdSerDe.serializeRmdRecord(mergeConflictResult.getValueSchemaId(), mergeConflictResult.getRmdRecord()); - // finally produce and update the transient record map. - if (updatedValueBytes == null) { + ChunkedValueManifest oldValueManifest = mergeConflictResultWrapper.getOldValueManifestContainer().getManifest(); + ChunkedValueManifest oldRmdManifest = mergeConflictResultWrapper.getOldRmdWithValueSchemaId() == null + ? null + : mergeConflictResultWrapper.getOldRmdWithValueSchemaId().getRmdManifest(); + // finally produce + if (mergeConflictResultWrapper.getUpdatedValueBytes() == null) { hostLevelIngestionStats.recordTombstoneCreatedDCR(); aggVersionedIngestionStats.recordTombStoneCreationDCR(storeName, versionNumber); - partitionConsumptionState - .setTransientRecord(kafkaClusterId, consumerRecord.getOffset(), key, valueSchemaId, rmdRecord); Delete deletePayload = new Delete(); deletePayload.schemaId = valueSchemaId; deletePayload.replicationMetadataVersionId = rmdProtocolVersionId; - deletePayload.replicationMetadataPayload = updatedRmdBytes; + deletePayload.replicationMetadataPayload = mergeConflictResultWrapper.getUpdatedRmdBytes(); BiConsumer produceToTopicFunction = (callback, sourceTopicOffset) -> veniceWriter.get() .delete( @@ -763,17 +879,6 @@ private void producePutOrDeleteToKafka( kafkaClusterId, beforeProcessingRecordTimestampNs); } else { - int valueLen = updatedValueBytes.remaining(); - partitionConsumptionState.setTransientRecord( - kafkaClusterId, - consumerRecord.getOffset(), - key, - updatedValueBytes.array(), - updatedValueBytes.position(), - valueLen, - valueSchemaId, - rmdRecord); - Put updatedPut = new Put(); updatedPut.putValue = ByteUtils .prependIntHeaderToByteBuffer(updatedValueBytes, valueSchemaId, mergeConflictResult.doesResultReuseInput()); diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/IngestionBatchProcessor.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/IngestionBatchProcessor.java new file mode 100644 index 00000000000..3e1a8202fdc --- /dev/null +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/IngestionBatchProcessor.java @@ -0,0 +1,165 @@ +package com.linkedin.davinci.kafka.consumer; + +import com.linkedin.davinci.utils.ByteArrayKey; +import com.linkedin.venice.exceptions.VeniceException; +import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; +import com.linkedin.venice.message.KafkaKey; +import com.linkedin.venice.pubsub.api.PubSubMessage; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.ReentrantLock; + + +/** + * This class is used to process the AA/WC messages in parallel to address the long-tail partition lagging issues. + * For the AA/WC message handling, the consumption is not the bottleneck, but the processing overhead, and + * even with a single consumer, with {@link IngestionBatchProcessor}, we hope we can utilize the full node's + * resources to speed up the leader ingestion. + */ +public class IngestionBatchProcessor { + interface ProcessingFunction { + PubSubMessageProcessedResult apply( + PubSubMessage consumerRecord, + PartitionConsumptionState partitionConsumptionState, + int partition, + String kafkaUrl, + int kafkaClusterId, + long beforeProcessingRecordTimestampNs, + long beforeProcessingBatchRecordsTimestampMs); + } + + private final String storeVersionName; + private final ExecutorService batchProcessingThreadPool; + private final KeyLevelLocksManager lockManager; + private final boolean isWriteComputationEnabled; + private final boolean isActiveActiveReplicationEnabled; + private final ProcessingFunction processingFunction; + + public IngestionBatchProcessor( + String storeVersionName, + ExecutorService batchProcessingThreadPool, + KeyLevelLocksManager lockManager, + ProcessingFunction processingFunction, + boolean isWriteComputationEnabled, + boolean isActiveActiveReplicationEnabled) { + this.storeVersionName = storeVersionName; + this.batchProcessingThreadPool = batchProcessingThreadPool; + this.lockManager = lockManager; + this.processingFunction = processingFunction; + this.isWriteComputationEnabled = isWriteComputationEnabled; + this.isActiveActiveReplicationEnabled = isActiveActiveReplicationEnabled; + } + + /** + * When {@link #lockManager} is not null, this function will try to lock all the keys + * (except Control Messages) passed by the params. + */ + public List lockKeys(List> records) { + if (lockManager != null) { + List locks = new ArrayList<>(records.size()); + records.forEach(r -> { + if (!r.getKey().isControlMessage()) { + ReentrantLock lock = lockManager.acquireLockByKey(ByteArrayKey.wrap(r.getKey().getKey())); + locks.add(lock); + lock.lock(); + } + }); + return locks; + } + return Collections.emptyList(); + } + + public void unlockKeys(List> records, List locks) { + if (lockManager != null) { + locks.forEach(lock -> lock.unlock()); + records.forEach(r -> { + if (!r.getKey().isControlMessage()) { + lockManager.releaseLock(ByteArrayKey.wrap(r.getKey().getKey())); + } + }); + } + } + + public static boolean isAllMessagesFromRTTopic( + Iterable> records) { + for (PubSubMessage record: records) { + if (!record.getTopicPartition().getPubSubTopic().isRealTime()) { + return false; + } + } + return true; + } + + public List> process( + List> records, + PartitionConsumptionState partitionConsumptionState, + int partition, + String kafkaUrl, + int kafkaClusterId, + long beforeProcessingRecordTimestampNs, + long beforeProcessingBatchRecordsTimestampMs) { + if (records.isEmpty()) { + return Collections.emptyList(); + } + AtomicBoolean isAllMessagesFromRTTopic = new AtomicBoolean(true); + List> resultList = + new ArrayList<>(records.size()); + records.forEach(r -> { + resultList.add(new PubSubMessageProcessedResultWrapper<>(r)); + if (!r.getTopicPartition().getPubSubTopic().isRealTime()) { + isAllMessagesFromRTTopic.set(false); + } + }); + if (!isWriteComputationEnabled && !isActiveActiveReplicationEnabled) { + return resultList; + } + // Only handle records from the real-time topic + if (!isAllMessagesFromRTTopic.get()) { + return resultList; + } + /** + * We would like to process the messages belonging to the same key sequentially to avoid race conditions. + */ + Map>> keyGroupMap = + new HashMap<>(records.size()); + resultList.forEach(r -> { + PubSubMessage message = r.getMessage(); + if (!message.getKey().isControlMessage()) { + ByteArrayKey byteArrayKey = ByteArrayKey.wrap(message.getKey().getKey()); + keyGroupMap.computeIfAbsent(byteArrayKey, (ignored) -> new ArrayList<>()).add(r); + } + }); + List> futureList = new ArrayList<>(keyGroupMap.size()); + keyGroupMap.forEach((ignored, recordsWithTheSameKey) -> { + futureList.add(CompletableFuture.runAsync(() -> { + recordsWithTheSameKey.forEach(recordWithTheSameKey -> { + recordWithTheSameKey.setProcessedResult( + processingFunction.apply( + recordWithTheSameKey.getMessage(), + partitionConsumptionState, + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingRecordTimestampNs, + beforeProcessingBatchRecordsTimestampMs)); + }); + }, batchProcessingThreadPool)); + }); + try { + CompletableFuture.allOf(futureList.toArray(new CompletableFuture[0])).get(); + } catch (Exception e) { + throw new VeniceException( + "Failed to execute the batch processing for " + storeVersionName + " partition: " + + partitionConsumptionState.getPartition(), + e); + } + + return resultList; + } +} 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 28c21390eb6..e073b20b72b 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 @@ -188,6 +188,7 @@ public class KafkaStoreIngestionService extends AbstractVeniceService implements private final PubSubTopicRepository pubSubTopicRepository = new PubSubTopicRepository(); private KafkaValueSerializer kafkaValueSerializer; private final IngestionThrottler ingestionThrottler; + private final ExecutorService aaWCWorkLoadProcessingThreadPool; public KafkaStoreIngestionService( StorageEngineRepository storageEngineRepository, @@ -438,6 +439,14 @@ public void handleStoreDeleted(Store store) { VeniceViewWriterFactory viewWriterFactory = new VeniceViewWriterFactory(veniceConfigLoader); + if (serverConfig.isAAWCWorkloadParallelProcessingEnabled()) { + this.aaWCWorkLoadProcessingThreadPool = Executors.newFixedThreadPool( + serverConfig.getAAWCWorkloadParallelProcessingThreadPoolSize(), + new DaemonThreadFactory("AA_WC_PARALLEL_PROCESSING")); + } else { + this.aaWCWorkLoadProcessingThreadPool = null; + } + ingestionTaskFactory = StoreIngestionTaskFactory.builder() .setVeniceWriterFactory(veniceWriterFactory) .setStorageEngineRepository(storageEngineRepository) @@ -463,6 +472,7 @@ public void handleStoreDeleted(Store store) { .setRunnableForKillIngestionTasksForNonCurrentVersions( serverConfig.getIngestionMemoryLimit() > 0 ? () -> killConsumptionTaskForNonCurrentVersions() : null) .setHeartbeatMonitoringService(heartbeatMonitoringService) + .setAAWCWorkLoadProcessingThreadPool(aaWCWorkLoadProcessingThreadPool) .build(); } @@ -569,6 +579,8 @@ public void stopInner() { leaderFollowerNotifiers.forEach(VeniceNotifier::close); Utils.closeQuietlyWithErrorLogged(metaStoreWriter); + shutdownExecutorService(aaWCWorkLoadProcessingThreadPool, "aaWCWorkLoadProcessingThreadPool", true); + kafkaMessageEnvelopeSchemaReader.ifPresent(Utils::closeQuietlyWithErrorLogged); // close it the very end to make sure all ingestion task have released the shared producers. 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 b45066deba6..e5843714d78 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 @@ -24,11 +24,11 @@ import com.linkedin.davinci.schema.merge.MergeRecordHelper; import com.linkedin.davinci.stats.ingestion.heartbeat.HeartbeatMonitoringService; import com.linkedin.davinci.storage.chunking.ChunkedValueManifestContainer; -import com.linkedin.davinci.storage.chunking.ChunkingAdapter; import com.linkedin.davinci.storage.chunking.GenericRecordChunkingAdapter; import com.linkedin.davinci.store.AbstractStorageEngine; import com.linkedin.davinci.store.cache.backend.ObjectCacheBackend; import com.linkedin.davinci.store.record.ValueRecord; +import com.linkedin.davinci.store.view.ChangeCaptureViewWriter; import com.linkedin.davinci.store.view.VeniceViewWriter; import com.linkedin.davinci.validation.KafkaDataIntegrityValidator; import com.linkedin.venice.common.VeniceSystemStoreUtils; @@ -85,6 +85,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; @@ -184,11 +185,14 @@ public class LeaderFollowerStoreIngestionTask extends StoreIngestionTask { private long dataRecoveryCompletionTimeLagThresholdInMs = 0; protected final Map viewWriters; + protected final boolean hasChangeCaptureView; protected final AvroStoreDeserializerCache storeDeserializerCache; private final AtomicLong lastSendIngestionHeartbeatTimestamp = new AtomicLong(0); + private final Lazy ingestionBatchProcessingLazy; + public LeaderFollowerStoreIngestionTask( StoreIngestionTaskFactory.Builder builder, Store store, @@ -293,13 +297,36 @@ public LeaderFollowerStoreIngestionTask( store, version.getNumber(), schemaRepository.getKeySchema(store.getName()).getSchema()); + boolean tmpValueForHasChangeCaptureViewWriter = false; + for (Map.Entry viewWriter: viewWriters.entrySet()) { + if (viewWriter.getValue() instanceof ChangeCaptureViewWriter) { + tmpValueForHasChangeCaptureViewWriter = true; + break; + } + } + hasChangeCaptureView = tmpValueForHasChangeCaptureViewWriter; } else { viewWriters = Collections.emptyMap(); + hasChangeCaptureView = false; } this.storeDeserializerCache = new AvroStoreDeserializerCache( builder.getSchemaRepo(), getStoreName(), serverConfig.isComputeFastAvroEnabled()); + this.ingestionBatchProcessingLazy = Lazy.of(() -> { + if (!serverConfig.isAAWCWorkloadParallelProcessingEnabled()) { + LOGGER.info("AA/WC workload parallel processing enabled is false"); + return null; + } + LOGGER.info("AA/WC workload parallel processing enabled is true"); + return new IngestionBatchProcessor( + kafkaVersionTopic, + parallelProcessingThreadPool, + null, + this::processMessage, + isWriteComputationEnabled, + isActiveActiveReplicationEnabled()); + }); } @Override @@ -316,6 +343,11 @@ protected void closeVeniceViewWriters() { } } + @Override + protected IngestionBatchProcessor getIngestionBatchProcessor() { + return ingestionBatchProcessingLazy.get(); + } + /** * Close a DIV segment for a version topic partition. */ @@ -2121,6 +2153,63 @@ protected void recordHeartbeatReceived( } } + @Override + protected Iterable> validateAndFilterOutDuplicateMessagesFromLeaderTopic( + Iterable> records, + PubSubTopicPartition topicPartition) { + PartitionConsumptionState partitionConsumptionState = + partitionConsumptionStateMap.get(topicPartition.getPartitionNumber()); + if (partitionConsumptionState == null) { + // The partition is likely unsubscribed, will skip these messages. + LOGGER.warn( + "No partition consumption state for store version: {}, partition:{}, will filter out all the messages", + kafkaVersionTopic, + topicPartition.getPartitionNumber()); + return Collections.emptyList(); + } + boolean isEndOfPushReceived = partitionConsumptionState.isEndOfPushReceived(); + if (!shouldProduceToVersionTopic(partitionConsumptionState)) { + return records; + } + /** + * Just to note this code is getting executed in Leader only. Leader DIV check progress is always ahead of the + * actual data persisted on disk. Leader DIV check results will not be persisted on disk. + */ + Iterator> iter = records.iterator(); + while (iter.hasNext()) { + PubSubMessage record = iter.next(); + try { + /** + * TODO: An improvement can be made to fail all future versions for fatal DIV exceptions after EOP. + */ + validateMessage( + this.kafkaDataIntegrityValidatorForLeaders, + record, + isEndOfPushReceived, + partitionConsumptionState); + versionedDIVStats.recordSuccessMsg(storeName, versionNumber); + } catch (FatalDataValidationException e) { + if (!isEndOfPushReceived) { + throw e; + } + } catch (DuplicateDataException e) { + /** + * Skip duplicated messages; leader must not produce duplicated messages from RT to VT, because leader will + * override the DIV info for messages from RT; as a result, both leaders and followers will persisted duplicated + * messages to disk, and potentially rewind a k/v pair to an old value. + */ + divErrorMetricCallback.accept(e); + LOGGER.info( + "Skipping a duplicate record from: {} offset: {} for replica: {}", + record.getTopicPartition(), + record.getOffset(), + partitionConsumptionState.getReplicaId()); + iter.remove(); + } + } + return records; + } + /** * The goal of this function is to possibly produce the incoming kafka message consumed from local VT, remote VT, RT or SR topic to * local VT if needed. It's decided based on the function output of {@link #shouldProduceToVersionTopic} and message type. @@ -2146,12 +2235,13 @@ protected void recordHeartbeatReceived( */ @Override protected DelegateConsumerRecordResult delegateConsumerRecord( - PubSubMessage consumerRecord, + PubSubMessageProcessedResultWrapper consumerRecordWrapper, int partition, String kafkaUrl, int kafkaClusterId, long beforeProcessingPerRecordTimestampNs, long beforeProcessingBatchRecordsTimestampMs) { + PubSubMessage consumerRecord = consumerRecordWrapper.getMessage(); try { KafkaKey kafkaKey = consumerRecord.getKey(); KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); @@ -2208,40 +2298,6 @@ protected DelegateConsumerRecordResult delegateConsumerRecord( updateLatestInMemoryLeaderConsumedRTOffset(partitionConsumptionState, kafkaUrl, consumerRecord.getOffset()); } - /** - * Just to note this code is getting executed in Leader only. Leader DIV check progress is always ahead of the - * actual data persisted on disk. Leader DIV check results will not be persisted on disk. - */ - boolean isEndOfPushReceived = partitionConsumptionState.isEndOfPushReceived(); - try { - /** - * TODO: An improvement can be made to fail all future versions for fatal DIV exceptions after EOP. - */ - validateMessage( - this.kafkaDataIntegrityValidatorForLeaders, - consumerRecord, - isEndOfPushReceived, - partitionConsumptionState); - versionedDIVStats.recordSuccessMsg(storeName, versionNumber); - } catch (FatalDataValidationException e) { - if (!isEndOfPushReceived) { - throw e; - } - } catch (DuplicateDataException e) { - /** - * Skip duplicated messages; leader must not produce duplicated messages from RT to VT, because leader will - * override the DIV info for messages from RT; as a result, both leaders and followers will persisted duplicated - * messages to disk, and potentially rewind a k/v pair to an old value. - */ - divErrorMetricCallback.accept(e); - LOGGER.debug( - "Skipping a duplicate record from: {} offset: {} for replica: {}", - consumerRecord.getTopicPartition(), - consumerRecord.getOffset(), - partitionConsumptionState.getReplicaId()); - return DelegateConsumerRecordResult.DUPLICATE_MESSAGE; - } - // heavy leader processing starts here versionedIngestionStats.recordLeaderPreprocessingLatency( storeName, @@ -2444,7 +2500,7 @@ protected DelegateConsumerRecordResult delegateConsumerRecord( // This function may modify the original record in KME and it is unsafe to use the payload from KME directly // after this call. processMessageAndMaybeProduceToKafka( - consumerRecord, + consumerRecordWrapper, partitionConsumptionState, partition, kafkaUrl, @@ -2926,7 +2982,7 @@ protected boolean shouldCompressData(PartitionConsumptionState partitionConsumpt return !compressionStrategy.equals(CompressionStrategy.NO_OP); } - protected void processMessageAndMaybeProduceToKafka( + private PubSubMessageProcessedResult processMessage( PubSubMessage consumerRecord, PartitionConsumptionState partitionConsumptionState, int partition, @@ -2963,8 +3019,153 @@ protected void processMessageAndMaybeProduceToKafka( null); } - LeaderProducedRecordContext leaderProducedRecordContext = - LeaderProducedRecordContext.newPutRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, put); + return new PubSubMessageProcessedResult(new WriteComputeResultWrapper(put, null, false)); + + case UPDATE: + /** + * 1. Currently, we support chunking only for messages produced on VT topic during batch part of the ingestion + * for hybrid stores. Chunking is NOT supported for messages produced to RT topics during streaming ingestion. + * + * So the assumption here is that the PUT/UPDATE messages stored in transientRecord should always be a full value + * (non chunked). Decoding should succeed using the simplified API + * {@link ChunkingAdapter#constructValue} + * + * 2. We always use the latest value schema to deserialize stored value bytes. + * 3. We always use the partial update schema with an ID combination of the latest value schema ID + update schema ID + * to deserialize the incoming Update request payload bytes. + * + * The reason for 2 and 3 is that we depend on the fact that the latest value schema must be a superset schema + * that contains all value fields that ever existed in a store value schema. So, always using a superset schema + * as the reader schema avoids data loss where the serialized bytes contain data for a field, however, the + * deserialized record does not contain that field because the reader schema does not contain that field. + */ + Update update = (Update) kafkaValue.payloadUnion; + final int readerValueSchemaId; + final int readerUpdateProtocolVersion; + if (isIngestingSystemStore()) { + DerivedSchemaEntry latestDerivedSchemaEntry = schemaRepository.getLatestDerivedSchema(storeName); + readerValueSchemaId = latestDerivedSchemaEntry.getValueSchemaID(); + readerUpdateProtocolVersion = latestDerivedSchemaEntry.getId(); + } else { + SchemaEntry supersetSchemaEntry = schemaRepository.getSupersetSchema(storeName); + if (supersetSchemaEntry == null) { + throw new IllegalStateException("Cannot find superset schema for store: " + storeName); + } + readerValueSchemaId = supersetSchemaEntry.getId(); + readerUpdateProtocolVersion = update.updateSchemaId; + } + ChunkedValueManifestContainer valueManifestContainer = new ChunkedValueManifestContainer(); + final GenericRecord currValue = readStoredValueRecord( + partitionConsumptionState, + keyBytes, + readerValueSchemaId, + consumerRecord.getTopicPartition(), + valueManifestContainer); + + final byte[] updatedValueBytes; + final ChunkedValueManifest oldValueManifest = valueManifestContainer.getManifest(); + + try { + long writeComputeStartTimeInNS = System.nanoTime(); + // Leader nodes are the only ones which process UPDATES, so it's valid to always compress and not call + // 'maybeCompress'. + updatedValueBytes = compressor.get() + .compress( + storeWriteComputeHandler.applyWriteCompute( + currValue, + update.schemaId, + readerValueSchemaId, + update.updateValue, + update.updateSchemaId, + readerUpdateProtocolVersion)); + hostLevelIngestionStats + .recordWriteComputeUpdateLatency(LatencyUtils.getElapsedTimeFromNSToMS(writeComputeStartTimeInNS)); + } catch (Exception e) { + writeComputeFailureCode = StatsErrorCode.WRITE_COMPUTE_UPDATE_FAILURE.code; + throw new RuntimeException(e); + } + + if (updatedValueBytes == null) { + if (currValue != null) { + throw new IllegalStateException( + "Detect a situation where the current value exists and the Write Compute request" + + "deletes the current value. It is unexpected because Write Compute only supports partial update and does " + + "not support record value deletion."); + } else { + // No-op. The fact that currValue does not exist on the leader means currValue does not exist on the + // follower + // either. So, there is no need to tell the follower replica to do anything. + return new PubSubMessageProcessedResult(new WriteComputeResultWrapper(null, null, true)); + } + } else { + partitionConsumptionState.setTransientRecord( + kafkaClusterId, + consumerRecord.getOffset(), + keyBytes, + updatedValueBytes, + 0, + updatedValueBytes.length, + readerValueSchemaId, + null); + + ByteBuffer updateValueWithSchemaId = + ByteUtils.prependIntHeaderToByteBuffer(ByteBuffer.wrap(updatedValueBytes), readerValueSchemaId, false); + + Put updatedPut = new Put(); + updatedPut.putValue = updateValueWithSchemaId; + updatedPut.schemaId = readerValueSchemaId; + return new PubSubMessageProcessedResult(new WriteComputeResultWrapper(updatedPut, oldValueManifest, false)); + } + case DELETE: + /** + * For WC enabled stores update the transient record map with the latest {key,null} for similar reason as mentioned in PUT above. + */ + if (isWriteComputationEnabled && partitionConsumptionState.isEndOfPushReceived()) { + partitionConsumptionState.setTransientRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, -1, null); + } + return new PubSubMessageProcessedResult(new WriteComputeResultWrapper(null, null, false)); + + default: + throw new VeniceMessageException( + ingestionTaskName + " : Invalid/Unrecognized operation type submitted: " + kafkaValue.messageType); + } + } + + protected void processMessageAndMaybeProduceToKafka( + PubSubMessageProcessedResultWrapper consumerRecordWrapper, + PartitionConsumptionState partitionConsumptionState, + int partition, + String kafkaUrl, + int kafkaClusterId, + long beforeProcessingRecordTimestampNs, + long beforeProcessingBatchRecordsTimestampMs) { + PubSubMessage consumerRecord = consumerRecordWrapper.getMessage(); + KafkaKey kafkaKey = consumerRecord.getKey(); + KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); + byte[] keyBytes = kafkaKey.getKey(); + MessageType msgType = MessageType.valueOf(kafkaValue.messageType); + + WriteComputeResultWrapper writeComputeResultWrapper; + if (consumerRecordWrapper.getProcessedResult() != null + && consumerRecordWrapper.getProcessedResult().getWriteComputeResultWrapper() != null) { + writeComputeResultWrapper = consumerRecordWrapper.getProcessedResult().getWriteComputeResultWrapper(); + } else { + writeComputeResultWrapper = processMessage( + consumerRecord, + partitionConsumptionState, + partition, + kafkaUrl, + kafkaClusterId, + beforeProcessingRecordTimestampNs, + beforeProcessingBatchRecordsTimestampMs).getWriteComputeResultWrapper(); + } + + Put newPut = writeComputeResultWrapper.getNewPut(); + LeaderProducedRecordContext leaderProducedRecordContext; + switch (msgType) { + case PUT: + leaderProducedRecordContext = + LeaderProducedRecordContext.newPutRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, newPut); produceToLocalKafka( consumerRecord, partitionConsumptionState, @@ -2997,7 +3198,12 @@ protected void processMessageAndMaybeProduceToKafka( leaderMetadataWrapper); } else { veniceWriter.get() - .put(keyBytes, ByteUtils.extractByteArray(putValue), put.schemaId, callback, leaderMetadataWrapper); + .put( + keyBytes, + ByteUtils.extractByteArray(newPut.putValue), + newPut.schemaId, + callback, + leaderMetadataWrapper); } }, partition, @@ -3007,23 +3213,37 @@ protected void processMessageAndMaybeProduceToKafka( break; case UPDATE: - handleUpdateRequest( - (Update) kafkaValue.payloadUnion, - keyBytes, + if (writeComputeResultWrapper.isSkipProduce()) { + return; + } + + leaderProducedRecordContext = + LeaderProducedRecordContext.newPutRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, newPut); + BiConsumer produceFunction = + (callback, leaderMetadataWrapper) -> veniceWriter.get() + .put( + keyBytes, + ByteUtils.extractByteArray(newPut.getPutValue()), + newPut.getSchemaId(), + callback, + leaderMetadataWrapper, + APP_DEFAULT_LOGICAL_TS, + null, + writeComputeResultWrapper.getOldValueManifest(), + null); + + produceToLocalKafka( consumerRecord, + partitionConsumptionState, + leaderProducedRecordContext, + produceFunction, + partitionConsumptionState.getPartition(), kafkaUrl, kafkaClusterId, - partitionConsumptionState, beforeProcessingRecordTimestampNs); break; case DELETE: - /** - * For WC enabled stores update the transient record map with the latest {key,null} for similar reason as mentioned in PUT above. - */ - if (isWriteComputationEnabled && partitionConsumptionState.isEndOfPushReceived()) { - partitionConsumptionState.setTransientRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, -1, null); - } leaderProducedRecordContext = LeaderProducedRecordContext .newDeleteRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, (Delete) kafkaValue.payloadUnion); produceToLocalKafka( @@ -3058,133 +3278,6 @@ protected void processMessageAndMaybeProduceToKafka( } } - /** - * 1. Currently, we support chunking only for messages produced on VT topic during batch part of the ingestion - * for hybrid stores. Chunking is NOT supported for messages produced to RT topics during streaming ingestion. - * - * So the assumption here is that the PUT/UPDATE messages stored in transientRecord should always be a full value - * (non chunked). Decoding should succeed using the simplified API - * {@link ChunkingAdapter#constructValue} - * - * 2. We always use the latest value schema to deserialize stored value bytes. - * 3. We always use the partial update schema with an ID combination of the latest value schema ID + update schema ID - * to deserialize the incoming Update request payload bytes. - * - * The reason for 2 and 3 is that we depend on the fact that the latest value schema must be a superset schema - * that contains all value fields that ever existed in a store value schema. So, always using a superset schema - * as the reader schema avoids data loss where the serialized bytes contain data for a field, however, the - * deserialized record does not contain that field because the reader schema does not contain that field. - */ - private void handleUpdateRequest( - Update update, - byte[] keyBytes, - PubSubMessage consumerRecord, - String kafkaUrl, - int kafkaClusterId, - PartitionConsumptionState partitionConsumptionState, - long beforeProcessingRecordTimestampNs) { - - final int readerValueSchemaId; - final int readerUpdateProtocolVersion; - if (isIngestingSystemStore()) { - DerivedSchemaEntry latestDerivedSchemaEntry = schemaRepository.getLatestDerivedSchema(storeName); - readerValueSchemaId = latestDerivedSchemaEntry.getValueSchemaID(); - readerUpdateProtocolVersion = latestDerivedSchemaEntry.getId(); - } else { - SchemaEntry supersetSchemaEntry = schemaRepository.getSupersetSchema(storeName); - if (supersetSchemaEntry == null) { - throw new IllegalStateException("Cannot find superset schema for store: " + storeName); - } - readerValueSchemaId = supersetSchemaEntry.getId(); - readerUpdateProtocolVersion = update.updateSchemaId; - } - ChunkedValueManifestContainer valueManifestContainer = new ChunkedValueManifestContainer(); - final GenericRecord currValue = readStoredValueRecord( - partitionConsumptionState, - keyBytes, - readerValueSchemaId, - consumerRecord.getTopicPartition(), - valueManifestContainer); - - final byte[] updatedValueBytes; - final ChunkedValueManifest oldValueManifest = valueManifestContainer.getManifest(); - - try { - long writeComputeStartTimeInNS = System.nanoTime(); - // Leader nodes are the only ones which process UPDATES, so it's valid to always compress and not call - // 'maybeCompress'. - updatedValueBytes = compressor.get() - .compress( - storeWriteComputeHandler.applyWriteCompute( - currValue, - update.schemaId, - readerValueSchemaId, - update.updateValue, - update.updateSchemaId, - readerUpdateProtocolVersion)); - hostLevelIngestionStats - .recordWriteComputeUpdateLatency(LatencyUtils.getElapsedTimeFromNSToMS(writeComputeStartTimeInNS)); - } catch (Exception e) { - writeComputeFailureCode = StatsErrorCode.WRITE_COMPUTE_UPDATE_FAILURE.code; - throw new RuntimeException(e); - } - - if (updatedValueBytes == null) { - if (currValue != null) { - throw new IllegalStateException( - "Detect a situation where the current value exists and the Write Compute request" - + "deletes the current value. It is unexpected because Write Compute only supports partial update and does " - + "not support record value deletion."); - } else { - // No-op. The fact that currValue does not exist on the leader means currValue does not exist on the follower - // either. So, there is no need to tell the follower replica to do anything. - } - } else { - partitionConsumptionState.setTransientRecord( - kafkaClusterId, - consumerRecord.getOffset(), - keyBytes, - updatedValueBytes, - 0, - updatedValueBytes.length, - readerValueSchemaId, - null); - - ByteBuffer updateValueWithSchemaId = - ByteUtils.prependIntHeaderToByteBuffer(ByteBuffer.wrap(updatedValueBytes), readerValueSchemaId, false); - - Put updatedPut = new Put(); - updatedPut.putValue = updateValueWithSchemaId; - updatedPut.schemaId = readerValueSchemaId; - - LeaderProducedRecordContext leaderProducedRecordContext = - LeaderProducedRecordContext.newPutRecord(kafkaClusterId, consumerRecord.getOffset(), keyBytes, updatedPut); - - BiConsumer produceFunction = - (callback, leaderMetadataWrapper) -> veniceWriter.get() - .put( - keyBytes, - updatedValueBytes, - readerValueSchemaId, - callback, - leaderMetadataWrapper, - APP_DEFAULT_LOGICAL_TS, - null, - oldValueManifest, - null); - - produceToLocalKafka( - consumerRecord, - partitionConsumptionState, - leaderProducedRecordContext, - produceFunction, - partitionConsumptionState.getPartition(), - kafkaUrl, - kafkaClusterId, - beforeProcessingRecordTimestampNs); - } - } - /** * Read the existing value. If a value for this key is found from the transient map then use that value, otherwise read * it from the storage engine. diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/MergeConflictResultWrapper.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/MergeConflictResultWrapper.java new file mode 100644 index 00000000000..9d2bcd22ac1 --- /dev/null +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/MergeConflictResultWrapper.java @@ -0,0 +1,67 @@ +package com.linkedin.davinci.kafka.consumer; + +import com.linkedin.davinci.replication.RmdWithValueSchemaId; +import com.linkedin.davinci.replication.merge.MergeConflictResult; +import com.linkedin.davinci.storage.chunking.ChunkedValueManifestContainer; +import com.linkedin.davinci.store.record.ByteBufferValueRecord; +import com.linkedin.venice.utils.lazy.Lazy; +import java.nio.ByteBuffer; + + +/** + * This wrapper is used to keep the context after handling Active/Active messages. + */ +public class MergeConflictResultWrapper { + private final MergeConflictResult mergeConflictResult; + private final Lazy> oldValueProvider; + private final Lazy oldValueByteBufferProvider; + private final RmdWithValueSchemaId oldRmdWithValueSchemaId; + private final ChunkedValueManifestContainer oldValueManifestContainer; + private final ByteBuffer updatedValueBytes; + private final ByteBuffer updatedRmdBytes; + + public MergeConflictResultWrapper( + MergeConflictResult mergeConflictResult, + Lazy> oldValueProvider, + Lazy oldValueByteBufferProvider, + RmdWithValueSchemaId oldRmdWithValueSchemaId, + ChunkedValueManifestContainer oldValueManifestContainer, + ByteBuffer updatedValueBytes, + ByteBuffer updatedRmdBytes) { + this.mergeConflictResult = mergeConflictResult; + this.oldValueProvider = oldValueProvider; + this.oldValueByteBufferProvider = oldValueByteBufferProvider; + this.oldRmdWithValueSchemaId = oldRmdWithValueSchemaId; + this.oldValueManifestContainer = oldValueManifestContainer; + this.updatedValueBytes = updatedValueBytes; + this.updatedRmdBytes = updatedRmdBytes; + } + + public MergeConflictResult getMergeConflictResult() { + return mergeConflictResult; + } + + public Lazy getOldValueByteBufferProvider() { + return oldValueByteBufferProvider; + } + + public RmdWithValueSchemaId getOldRmdWithValueSchemaId() { + return oldRmdWithValueSchemaId; + } + + public ChunkedValueManifestContainer getOldValueManifestContainer() { + return oldValueManifestContainer; + } + + public Lazy> getOldValueProvider() { + return oldValueProvider; + } + + public ByteBuffer getUpdatedValueBytes() { + return updatedValueBytes; + } + + public ByteBuffer getUpdatedRmdBytes() { + return updatedRmdBytes; + } +} diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/PubSubMessageProcessedResult.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/PubSubMessageProcessedResult.java new file mode 100644 index 00000000000..ea93018427e --- /dev/null +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/PubSubMessageProcessedResult.java @@ -0,0 +1,24 @@ +package com.linkedin.davinci.kafka.consumer; + +public class PubSubMessageProcessedResult { + private final MergeConflictResultWrapper mergeConflictResultWrapper; + private final WriteComputeResultWrapper writeComputeResultWrapper; + + public PubSubMessageProcessedResult(MergeConflictResultWrapper mergeConflictResultWrapper) { + this.mergeConflictResultWrapper = mergeConflictResultWrapper; + this.writeComputeResultWrapper = null; + } + + public PubSubMessageProcessedResult(WriteComputeResultWrapper writeComputeResultWrapper) { + this.writeComputeResultWrapper = writeComputeResultWrapper; + this.mergeConflictResultWrapper = null; + } + + public MergeConflictResultWrapper getMergeConflictResultWrapper() { + return mergeConflictResultWrapper; + } + + public WriteComputeResultWrapper getWriteComputeResultWrapper() { + return writeComputeResultWrapper; + } +} diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/PubSubMessageProcessedResultWrapper.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/PubSubMessageProcessedResultWrapper.java new file mode 100644 index 00000000000..98a9ca3feb3 --- /dev/null +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/PubSubMessageProcessedResultWrapper.java @@ -0,0 +1,25 @@ +package com.linkedin.davinci.kafka.consumer; + +import com.linkedin.venice.pubsub.api.PubSubMessage; + + +public class PubSubMessageProcessedResultWrapper { + private final PubSubMessage message; + private PubSubMessageProcessedResult processedResult; + + public PubSubMessageProcessedResultWrapper(PubSubMessage message) { + this.message = message; + } + + public PubSubMessage getMessage() { + return message; + } + + public PubSubMessageProcessedResult getProcessedResult() { + return processedResult; + } + + public void setProcessedResult(PubSubMessageProcessedResult transformedResult) { + this.processedResult = transformedResult; + } +} 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 6f9e0407125..bf3e35fb229 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 @@ -106,6 +106,7 @@ import com.linkedin.venice.utils.Time; import com.linkedin.venice.utils.Timer; import com.linkedin.venice.utils.Utils; +import com.linkedin.venice.utils.ValueHolder; import com.linkedin.venice.utils.VeniceProperties; import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap; import com.linkedin.venice.utils.lazy.Lazy; @@ -120,6 +121,7 @@ import java.util.Collection; import java.util.Collections; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; @@ -138,6 +140,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.BooleanSupplier; import java.util.function.Consumer; import java.util.function.Function; @@ -337,6 +340,8 @@ public abstract class StoreIngestionTask implements Runnable, Closeable { protected volatile PartitionReplicaIngestionContext.WorkloadType workloadType; protected final boolean batchReportIncPushStatusEnabled; + protected final ExecutorService parallelProcessingThreadPool; + public StoreIngestionTask( StoreIngestionTaskFactory.Builder builder, Store store, @@ -505,6 +510,7 @@ public StoreIngestionTask( LOGGER.info("Disabled record-level metric when ingesting current version: {}", kafkaVersionTopic); } this.batchReportIncPushStatusEnabled = !isDaVinciClient && serverConfig.getBatchReportEOIPEnabled(); + this.parallelProcessingThreadPool = builder.getAAWCWorkLoadProcessingThreadPool(); } /** Package-private on purpose, only intended for tests. Do not use for production use cases. */ @@ -512,6 +518,8 @@ void setPurgeTransientRecordBuffer(boolean purgeTransientRecordBuffer) { this.purgeTransientRecordBuffer = purgeTransientRecordBuffer; } + protected abstract IngestionBatchProcessor getIngestionBatchProcessor(); + public AbstractStorageEngine getStorageEngine() { return storageEngine; } @@ -1030,6 +1038,82 @@ protected void produceToStoreBufferService( } } + protected abstract Iterable> validateAndFilterOutDuplicateMessagesFromLeaderTopic( + Iterable> records, + PubSubTopicPartition topicPartition); + + private int handleSingleMessage( + PubSubMessageProcessedResultWrapper consumerRecordWrapper, + PubSubTopicPartition topicPartition, + PartitionConsumptionState partitionConsumptionState, + String kafkaUrl, + int kafkaClusterId, + long beforeProcessingPerRecordTimestampNs, + long beforeProcessingBatchRecordsTimestampMs, + boolean metricsEnabled, + ValueHolder elapsedTimeForPuttingIntoQueue) throws InterruptedException { + PubSubMessage record = consumerRecordWrapper.getMessage(); + if (record.getKey().isControlMessage()) { + ControlMessage controlMessage = (ControlMessage) record.getValue().payloadUnion; + if (ControlMessageType.valueOf(controlMessage.controlMessageType) == ControlMessageType.START_OF_PUSH) { + /** + * N.B.: The rest of the {@link ControlMessage} types are handled by: + * {@link #processControlMessage(KafkaMessageEnvelope, ControlMessage, int, long, PartitionConsumptionState)} + * + * But for the SOP in particular, we want to process it here, at the start of the pipeline, to ensure that the + * {@link StoreVersionState} is properly primed, as other functions below this point, but prior to being + * enqueued into the {@link StoreBufferService} rely on this state to be there. + */ + processStartOfPush( + record.getValue(), + controlMessage, + record.getTopicPartition().getPartitionNumber(), + partitionConsumptionStateMap.get(topicPartition.getPartitionNumber())); + } + } + + // This function may modify the original record in KME and it is unsafe to use the payload from KME directly after + // this call. + DelegateConsumerRecordResult delegateConsumerRecordResult = delegateConsumerRecord( + consumerRecordWrapper, + topicPartition.getPartitionNumber(), + kafkaUrl, + kafkaClusterId, + beforeProcessingPerRecordTimestampNs, + beforeProcessingBatchRecordsTimestampMs); + + switch (delegateConsumerRecordResult) { + case QUEUED_TO_DRAINER: + long queuePutStartTimeInNS = metricsEnabled ? System.nanoTime() : 0; + + // blocking call + storeBufferService.putConsumerRecord( + record, + this, + null, + topicPartition.getPartitionNumber(), + kafkaUrl, + beforeProcessingPerRecordTimestampNs); + + if (metricsEnabled) { + elapsedTimeForPuttingIntoQueue.setValue( + elapsedTimeForPuttingIntoQueue.getValue() + LatencyUtils.getElapsedTimeFromNSToMS(queuePutStartTimeInNS)); + } + break; + case PRODUCED_TO_KAFKA: + case SKIPPED_MESSAGE: + break; + default: + throw new VeniceException( + ingestionTaskName + " received unknown DelegateConsumerRecordResult enum for " + + record.getTopicPartition()); + } + // Update the latest message consumed time + partitionConsumptionState.setLatestMessageConsumedTimestampInMs(beforeProcessingBatchRecordsTimestampMs); + + return record.getPayloadSize(); + } + /** * This function is in charge of producing the consumer records to the writer buffers maintained by {@link StoreBufferService}. * @@ -1044,84 +1128,156 @@ protected void produceToStoreBufferServiceOrKafka( PubSubTopicPartition topicPartition, String kafkaUrl, int kafkaClusterId) throws InterruptedException { + PartitionConsumptionState partitionConsumptionState = + partitionConsumptionStateMap.get(topicPartition.getPartitionNumber()); + if (partitionConsumptionState == null) { + throw new VeniceException( + "PartitionConsumptionState should present for store version: " + kafkaVersionTopic + ", partition: " + + topicPartition.getPartitionNumber()); + } + /** + * Validate and filter out duplicate messages from the real-time topic as early as possible, so that + * the following batch processing logic won't spend useless efforts on duplicate messages. + */ + records = validateAndFilterOutDuplicateMessagesFromLeaderTopic(records, topicPartition); + + if ((isActiveActiveReplicationEnabled || isWriteComputationEnabled) + && serverConfig.isAAWCWorkloadParallelProcessingEnabled() + && IngestionBatchProcessor.isAllMessagesFromRTTopic(records)) { + produceToStoreBufferServiceOrKafkaInBatch( + records, + topicPartition, + partitionConsumptionState, + kafkaUrl, + kafkaClusterId); + return; + } + long totalBytesRead = 0; - double elapsedTimeForPuttingIntoQueue = 0; + ValueHolder elapsedTimeForPuttingIntoQueue = new ValueHolder<>(0d); boolean metricsEnabled = emitMetrics.get(); long beforeProcessingBatchRecordsTimestampMs = System.currentTimeMillis(); + + partitionConsumptionState = partitionConsumptionStateMap.get(topicPartition.getPartitionNumber()); for (PubSubMessage record: records) { long beforeProcessingPerRecordTimestampNs = System.nanoTime(); - PartitionConsumptionState partitionConsumptionState = - partitionConsumptionStateMap.get(topicPartition.getPartitionNumber()); - if (partitionConsumptionState != null) { - partitionConsumptionState.setLatestPolledMessageTimestampInMs(beforeProcessingBatchRecordsTimestampMs); - } + partitionConsumptionState.setLatestPolledMessageTimestampInMs(beforeProcessingBatchRecordsTimestampMs); if (!shouldProcessRecord(record)) { - if (partitionConsumptionState != null) { - partitionConsumptionState.updateLatestIgnoredUpstreamRTOffset(kafkaUrl, record.getOffset()); - } + partitionConsumptionState.updateLatestIgnoredUpstreamRTOffset(kafkaUrl, record.getOffset()); continue; } - if (record.getKey().isControlMessage()) { - ControlMessage controlMessage = (ControlMessage) record.getValue().payloadUnion; - if (ControlMessageType.valueOf(controlMessage.controlMessageType) == ControlMessageType.START_OF_PUSH) { - /** - * N.B.: The rest of the {@link ControlMessage} types are handled by: - * {@link #processControlMessage(KafkaMessageEnvelope, ControlMessage, int, long, PartitionConsumptionState)} - * - * But for the SOP in particular, we want to process it here, at the start of the pipeline, to ensure that the - * {@link StoreVersionState} is properly primed, as other functions below this point, but prior to being - * enqueued into the {@link StoreBufferService} rely on this state to be there. - */ - processStartOfPush( - record.getValue(), - controlMessage, - record.getTopicPartition().getPartitionNumber(), - partitionConsumptionStateMap.get(topicPartition.getPartitionNumber())); - } - } - // Check schema id availability before putting consumer record to drainer queue waitReadyToProcessRecord(record); - // This function may modify the original record in KME and it is unsafe to use the payload from KME directly after - // this call. - DelegateConsumerRecordResult delegateConsumerRecordResult = delegateConsumerRecord( - record, - topicPartition.getPartitionNumber(), + + totalBytesRead += handleSingleMessage( + new PubSubMessageProcessedResultWrapper<>(record), + topicPartition, + partitionConsumptionState, kafkaUrl, kafkaClusterId, beforeProcessingPerRecordTimestampNs, - beforeProcessingBatchRecordsTimestampMs); - switch (delegateConsumerRecordResult) { - case QUEUED_TO_DRAINER: - long queuePutStartTimeInNS = metricsEnabled ? System.nanoTime() : 0; - - // blocking call - storeBufferService.putConsumerRecord( - record, - this, - null, - topicPartition.getPartitionNumber(), - kafkaUrl, - beforeProcessingPerRecordTimestampNs); + beforeProcessingBatchRecordsTimestampMs, + metricsEnabled, + elapsedTimeForPuttingIntoQueue); + } - if (metricsEnabled) { - elapsedTimeForPuttingIntoQueue += LatencyUtils.getElapsedTimeFromNSToMS(queuePutStartTimeInNS); - } - break; - case PRODUCED_TO_KAFKA: - case SKIPPED_MESSAGE: - case DUPLICATE_MESSAGE: - break; - default: - throw new VeniceException( - ingestionTaskName + " received unknown DelegateConsumerRecordResult enum for " - + record.getTopicPartition()); + /** + * Even if the records list is empty, we still need to check quota to potentially resume partition + */ + storageUtilizationManager.enforcePartitionQuota(topicPartition.getPartitionNumber(), totalBytesRead); + + if (metricsEnabled) { + if (totalBytesRead > 0) { + hostLevelIngestionStats.recordTotalBytesReadFromKafkaAsUncompressedSize(totalBytesRead); + } + if (elapsedTimeForPuttingIntoQueue.getValue() > 0) { + hostLevelIngestionStats.recordConsumerRecordsQueuePutLatency( + elapsedTimeForPuttingIntoQueue.getValue(), + beforeProcessingBatchRecordsTimestampMs); } - totalBytesRead += record.getPayloadSize(); - // Update the latest message consumed time + + hostLevelIngestionStats.recordStorageQuotaUsed(storageUtilizationManager.getDiskQuotaUsage()); + } + } + + protected void produceToStoreBufferServiceOrKafkaInBatch( + Iterable> records, + PubSubTopicPartition topicPartition, + PartitionConsumptionState partitionConsumptionState, + String kafkaUrl, + int kafkaClusterId) throws InterruptedException { + long totalBytesRead = 0; + ValueHolder elapsedTimeForPuttingIntoQueue = new ValueHolder<>(0d); + boolean metricsEnabled = emitMetrics.get(); + long beforeProcessingBatchRecordsTimestampMs = System.currentTimeMillis(); + /** + * Split the records into mini batches. + */ + int batchSize = serverConfig.getAAWCWorkloadParallelProcessingThreadPoolSize(); + List>> batches = new ArrayList<>(); + List> ongoingBatch = new ArrayList<>(batchSize); + Iterator> iter = records.iterator(); + while (iter.hasNext()) { + PubSubMessage record = iter.next(); if (partitionConsumptionState != null) { - partitionConsumptionState.setLatestMessageConsumedTimestampInMs(beforeProcessingBatchRecordsTimestampMs); + partitionConsumptionState.setLatestPolledMessageTimestampInMs(beforeProcessingBatchRecordsTimestampMs); + } + if (!shouldProcessRecord(record)) { + if (partitionConsumptionState != null) { + partitionConsumptionState.updateLatestIgnoredUpstreamRTOffset(kafkaUrl, record.getOffset()); + } + continue; + } + waitReadyToProcessRecord(record); + ongoingBatch.add(record); + if (ongoingBatch.size() == batchSize) { + batches.add(ongoingBatch); + ongoingBatch = new ArrayList<>(batchSize); + } + } + if (!ongoingBatch.isEmpty()) { + batches.add(ongoingBatch); + } + if (batches.isEmpty()) { + return; + } + IngestionBatchProcessor ingestionBatchProcessor = getIngestionBatchProcessor(); + if (ingestionBatchProcessor == null) { + throw new VeniceException( + "IngestionBatchProcessor object should present for store version: " + kafkaVersionTopic); + } + /** + * Process records batch by batch. + */ + for (List> batch: batches) { + List locks = ingestionBatchProcessor.lockKeys(batch); + try { + long beforeProcessingPerRecordTimestampNs = System.nanoTime(); + List> processedResults = + ingestionBatchProcessor.process( + batch, + partitionConsumptionState, + topicPartition.getPartitionNumber(), + kafkaUrl, + kafkaClusterId, + beforeProcessingPerRecordTimestampNs, + beforeProcessingBatchRecordsTimestampMs); + + for (PubSubMessageProcessedResultWrapper processedRecord: processedResults) { + totalBytesRead += handleSingleMessage( + processedRecord, + topicPartition, + partitionConsumptionState, + kafkaUrl, + kafkaClusterId, + beforeProcessingPerRecordTimestampNs, + beforeProcessingBatchRecordsTimestampMs, + metricsEnabled, + elapsedTimeForPuttingIntoQueue); + } + } finally { + ingestionBatchProcessor.unlockKeys(batch, locks); } } @@ -1134,9 +1290,9 @@ protected void produceToStoreBufferServiceOrKafka( if (totalBytesRead > 0) { hostLevelIngestionStats.recordTotalBytesReadFromKafkaAsUncompressedSize(totalBytesRead); } - if (elapsedTimeForPuttingIntoQueue > 0) { + if (elapsedTimeForPuttingIntoQueue.getValue() > 0) { hostLevelIngestionStats.recordConsumerRecordsQueuePutLatency( - elapsedTimeForPuttingIntoQueue, + elapsedTimeForPuttingIntoQueue.getValue(), beforeProcessingBatchRecordsTimestampMs); } @@ -3918,7 +4074,7 @@ protected void waitForAllMessageToBeProcessedFromTopicPartition( } protected abstract DelegateConsumerRecordResult delegateConsumerRecord( - PubSubMessage consumerRecordWrapper, + PubSubMessageProcessedResultWrapper consumerRecordWrapper, int partition, String kafkaUrl, int kafkaClusterId, @@ -3926,7 +4082,7 @@ protected abstract DelegateConsumerRecordResult delegateConsumerRecord( long beforeProcessingBatchRecordsTimestampMs); /** - * This enum represents all potential results after calling {@link #delegateConsumerRecord(PubSubMessage, int, String, int, long, long)}. + * This enum represents all potential results after calling {@link #delegateConsumerRecord(PubSubMessageProcessedResultWrapper, int, String, int, long, long)}. */ protected enum DelegateConsumerRecordResult { /** @@ -3940,10 +4096,6 @@ protected enum DelegateConsumerRecordResult { * 3. Leader is consuming from local version topics */ QUEUED_TO_DRAINER, - /** - * The consumer record is a duplicated message. - */ - DUPLICATE_MESSAGE, /** * The consumer record is skipped. e.g. remote VT's TS message during data recovery. */ 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 d6f09f290f6..21b480af85a 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 @@ -27,6 +27,7 @@ import java.util.Optional; import java.util.Properties; import java.util.Queue; +import java.util.concurrent.ExecutorService; import java.util.function.BooleanSupplier; import java.util.function.Function; @@ -116,6 +117,7 @@ public static class Builder { private StorageEngineBackedCompressorFactory compressorFactory; private PubSubTopicRepository pubSubTopicRepository; private Runnable runnableForKillIngestionTasksForNonCurrentVersions; + private ExecutorService aaWCWorkLoadProcessingThreadPool; private interface Setter { void apply(); @@ -318,5 +320,13 @@ public Runnable getRunnableForKillIngestionTasksForNonCurrentVersions() { public Builder setRunnableForKillIngestionTasksForNonCurrentVersions(Runnable runnable) { return set(() -> this.runnableForKillIngestionTasksForNonCurrentVersions = runnable); } + + public Builder setAAWCWorkLoadProcessingThreadPool(ExecutorService executorService) { + return set(() -> this.aaWCWorkLoadProcessingThreadPool = executorService); + } + + public ExecutorService getAAWCWorkLoadProcessingThreadPool() { + return this.aaWCWorkLoadProcessingThreadPool; + } } } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/WriteComputeResultWrapper.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/WriteComputeResultWrapper.java new file mode 100644 index 00000000000..ea7669ef12f --- /dev/null +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/WriteComputeResultWrapper.java @@ -0,0 +1,35 @@ +package com.linkedin.davinci.kafka.consumer; + +import com.linkedin.venice.kafka.protocol.Put; +import com.linkedin.venice.storage.protocol.ChunkedValueManifest; + + +/** + * This result wrapper is for non-Active-Active stores. + */ +public class WriteComputeResultWrapper { + private final Put newPut; + private final ChunkedValueManifest oldValueManifest; + /** + * This can be true when there is some delete op against a non-existing entry. + */ + private final boolean skipProduce; + + public WriteComputeResultWrapper(Put newPut, ChunkedValueManifest oldValueManifest, boolean skipProduce) { + this.newPut = newPut; + this.oldValueManifest = oldValueManifest; + this.skipProduce = skipProduce; + } + + public Put getNewPut() { + return newPut; + } + + public ChunkedValueManifest getOldValueManifest() { + return oldValueManifest; + } + + public boolean isSkipProduce() { + return skipProduce; + } +} diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java index fd323b32574..afedb84fe1d 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTaskTest.java @@ -88,6 +88,7 @@ import com.linkedin.venice.writer.VeniceWriterOptions; import it.unimi.dsi.fastutil.ints.Int2ObjectArrayMap; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; +import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; import it.unimi.dsi.fastutil.objects.Object2IntArrayMap; import java.io.IOException; import java.nio.ByteBuffer; @@ -139,7 +140,14 @@ public void testHandleDeleteBeforeEOP() { kafkaValue.payloadUnion = deletePayload; ArgumentCaptor leaderProducedRecordContextArgumentCaptor = ArgumentCaptor.forClass(LeaderProducedRecordContext.class); - ingestionTask.processMessageAndMaybeProduceToKafka(consumerRecord, pcs, 0, "dummyUrl", 0, 0L, 0L); + ingestionTask.processMessageAndMaybeProduceToKafka( + new PubSubMessageProcessedResultWrapper<>(consumerRecord), + pcs, + 0, + "dummyUrl", + 0, + 0L, + 0L); verify(ingestionTask, times(1)).produceToLocalKafka( any(), any(), @@ -697,4 +705,32 @@ private VeniceCompressor getCompressor(CompressionStrategy strategy) { } return new CompressorFactory().getCompressor(strategy); } + + @Test + public void getKeyLevelLockMaxPoolSizeBasedOnServerConfigTest() { + VeniceServerConfig serverConfig = mock(VeniceServerConfig.class); + Int2ObjectMap clusterIdToUrlMap = new Int2ObjectOpenHashMap<>(); + clusterIdToUrlMap.put(1, "region_1_url"); + clusterIdToUrlMap.put(2, "region_2_url"); + clusterIdToUrlMap.put(3, "region_3_url"); + when(serverConfig.getConsumerPoolStrategyType()) + .thenReturn(KafkaConsumerServiceDelegator.ConsumerPoolStrategyType.DEFAULT); + when(serverConfig.getConsumerPoolSizePerKafkaCluster()).thenReturn(100); + when(serverConfig.getKafkaClusterIdToUrlMap()).thenReturn(clusterIdToUrlMap); + assertEquals(ActiveActiveStoreIngestionTask.getKeyLevelLockMaxPoolSizeBasedOnServerConfig(serverConfig, 10), 31); + + // Test when current version prioritization strategy is enabled. + when(serverConfig.getConsumerPoolStrategyType()) + .thenReturn(KafkaConsumerServiceDelegator.ConsumerPoolStrategyType.CURRENT_VERSION_PRIORITIZATION); + when(serverConfig.getConsumerPoolSizeForCurrentVersionAAWCLeader()).thenReturn(10); + when(serverConfig.getConsumerPoolSizeForNonCurrentVersionAAWCLeader()).thenReturn(20); + when(serverConfig.getConsumerPoolSizeForCurrentVersionNonAAWCLeader()).thenReturn(30); + when(serverConfig.getConsumerPoolSizeForNonCurrentVersionNonAAWCLeader()).thenReturn(40); + assertEquals(ActiveActiveStoreIngestionTask.getKeyLevelLockMaxPoolSizeBasedOnServerConfig(serverConfig, 1000), 91); + + // Test with parallel compute is enabled + when(serverConfig.getAAWCWorkloadParallelProcessingThreadPoolSize()).thenReturn(8); + when(serverConfig.isAAWCWorkloadParallelProcessingEnabled()).thenReturn(true); + assertEquals(ActiveActiveStoreIngestionTask.getKeyLevelLockMaxPoolSizeBasedOnServerConfig(serverConfig, 1000), 721); + } } diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/IngestionBatchProcessorTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/IngestionBatchProcessorTest.java new file mode 100644 index 00000000000..92259806100 --- /dev/null +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/IngestionBatchProcessorTest.java @@ -0,0 +1,190 @@ +package com.linkedin.davinci.kafka.consumer; + +import static org.mockito.Mockito.mock; +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.assertTrue; + +import com.linkedin.alpini.base.concurrency.ExecutorService; +import com.linkedin.alpini.base.concurrency.Executors; +import com.linkedin.davinci.utils.ByteArrayKey; +import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; +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.ImmutablePubSubMessage; +import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl; +import com.linkedin.venice.pubsub.PubSubTopicRepository; +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.utils.DaemonThreadFactory; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.locks.ReentrantLock; +import org.testng.annotations.Test; + + +public class IngestionBatchProcessorTest { + private static final PubSubTopicRepository TOPIC_REPOSITORY = new PubSubTopicRepository(); + + @Test + public void isAllMessagesFromRTTopicTest() { + PubSubTopic versionTopic = TOPIC_REPOSITORY.getTopic("store_v1"); + PubSubTopic rtTopic = TOPIC_REPOSITORY.getTopic("store_rt"); + + PubSubTopicPartition versionTopicPartition = new PubSubTopicPartitionImpl(versionTopic, 1); + PubSubTopicPartition rtTopicPartition = new PubSubTopicPartitionImpl(rtTopic, 1); + + PubSubMessage vtMessage1 = new ImmutablePubSubMessage<>( + mock(KafkaKey.class), + mock(KafkaMessageEnvelope.class), + versionTopicPartition, + 1, + 100, + 100); + PubSubMessage vtMessage2 = new ImmutablePubSubMessage<>( + mock(KafkaKey.class), + mock(KafkaMessageEnvelope.class), + versionTopicPartition, + 2, + 101, + 100); + PubSubMessage rtMessage1 = new ImmutablePubSubMessage<>( + mock(KafkaKey.class), + mock(KafkaMessageEnvelope.class), + rtTopicPartition, + 1, + 100, + 100); + PubSubMessage rtMessage2 = new ImmutablePubSubMessage<>( + mock(KafkaKey.class), + mock(KafkaMessageEnvelope.class), + rtTopicPartition, + 2, + 101, + 100); + + assertFalse(IngestionBatchProcessor.isAllMessagesFromRTTopic(Arrays.asList(vtMessage1, vtMessage2))); + assertFalse(IngestionBatchProcessor.isAllMessagesFromRTTopic(Arrays.asList(vtMessage1, rtMessage1))); + assertTrue(IngestionBatchProcessor.isAllMessagesFromRTTopic(Arrays.asList(rtMessage1, rtMessage2))); + } + + @Test + public void lockKeysTest() { + KeyLevelLocksManager mockKeyLevelLocksManager = mock(KeyLevelLocksManager.class); + ReentrantLock lockForKey1 = mock(ReentrantLock.class); + ReentrantLock lockForKey2 = mock(ReentrantLock.class); + byte[] key1 = "key1".getBytes(); + byte[] key2 = "key2".getBytes(); + when(mockKeyLevelLocksManager.acquireLockByKey(ByteArrayKey.wrap(key1))).thenReturn(lockForKey1); + when(mockKeyLevelLocksManager.acquireLockByKey(ByteArrayKey.wrap(key2))).thenReturn(lockForKey2); + + PubSubTopic rtTopic = TOPIC_REPOSITORY.getTopic("store_rt"); + PubSubTopicPartition rtTopicPartition = new PubSubTopicPartitionImpl(rtTopic, 1); + PubSubMessage rtMessage1 = new ImmutablePubSubMessage<>( + new KafkaKey(MessageType.PUT, key1), + mock(KafkaMessageEnvelope.class), + rtTopicPartition, + 1, + 100, + 100); + PubSubMessage rtMessage2 = new ImmutablePubSubMessage<>( + new KafkaKey(MessageType.PUT, key2), + mock(KafkaMessageEnvelope.class), + rtTopicPartition, + 2, + 101, + 100); + + IngestionBatchProcessor batchProcessor = new IngestionBatchProcessor( + "store_v1", + mock(ExecutorService.class), + mockKeyLevelLocksManager, + (ignored1, ignored2, ignored3, ignored4, ignored5, ignored6, ignored7) -> null, + true, + true); + List locks = batchProcessor.lockKeys(Arrays.asList(rtMessage1, rtMessage2)); + verify(mockKeyLevelLocksManager).acquireLockByKey(ByteArrayKey.wrap(key1)); + verify(mockKeyLevelLocksManager).acquireLockByKey(ByteArrayKey.wrap(key2)); + verify(lockForKey1).lock(); + verify(lockForKey2).lock(); + assertEquals(locks.get(0), lockForKey1); + assertEquals(locks.get(1), lockForKey2); + + // unlock test + batchProcessor.unlockKeys(Arrays.asList(rtMessage1, rtMessage2), locks); + + verify(lockForKey1).unlock(); + verify(lockForKey2).unlock(); + verify(mockKeyLevelLocksManager).releaseLock(ByteArrayKey.wrap(key1)); + verify(mockKeyLevelLocksManager).releaseLock(ByteArrayKey.wrap(key2)); + } + + @Test + public void processTest() { + KeyLevelLocksManager mockKeyLevelLocksManager = mock(KeyLevelLocksManager.class); + PubSubTopic rtTopic = TOPIC_REPOSITORY.getTopic("store_rt"); + byte[] key1 = "key1".getBytes(); + byte[] key2 = "key2".getBytes(); + PubSubTopicPartition rtTopicPartition = new PubSubTopicPartitionImpl(rtTopic, 1); + PubSubMessage rtMessage1 = new ImmutablePubSubMessage<>( + new KafkaKey(MessageType.PUT, key1), + mock(KafkaMessageEnvelope.class), + rtTopicPartition, + 1, + 100, + 100); + PubSubMessage rtMessage2 = new ImmutablePubSubMessage<>( + new KafkaKey(MessageType.PUT, key2), + mock(KafkaMessageEnvelope.class), + rtTopicPartition, + 2, + 101, + 100); + + IngestionBatchProcessor batchProcessor = new IngestionBatchProcessor( + "store_v1", + Executors.newFixedThreadPool(1, new DaemonThreadFactory("test")), + mockKeyLevelLocksManager, + (consumerRecord, ignored2, ignored3, ignored4, ignored5, ignored6, ignored7) -> { + if (Arrays.equals(consumerRecord.getKey().getKey(), "key1".getBytes())) { + Put put = new Put(); + put.setPutValue(ByteBuffer.wrap("value1".getBytes())); + WriteComputeResultWrapper writeComputeResultWrapper = new WriteComputeResultWrapper(put, null, true); + return new PubSubMessageProcessedResult(writeComputeResultWrapper); + } else if (Arrays.equals(consumerRecord.getKey().getKey(), "key2".getBytes())) { + Put put = new Put(); + put.setPutValue(ByteBuffer.wrap("value2".getBytes())); + WriteComputeResultWrapper writeComputeResultWrapper = new WriteComputeResultWrapper(put, null, true); + return new PubSubMessageProcessedResult(writeComputeResultWrapper); + } + return null; + }, + true, + true); + + List> result = batchProcessor.process( + Arrays.asList(rtMessage1, rtMessage2), + mock(PartitionConsumptionState.class), + 1, + "test_kafka", + 1, + 1, + 1); + + assertEquals(result.size(), 2); + PubSubMessageProcessedResultWrapper resultForKey1 = result.get(0); + assertEquals( + resultForKey1.getProcessedResult().getWriteComputeResultWrapper().getNewPut().putValue.array(), + "value1".getBytes()); + PubSubMessageProcessedResultWrapper resultForKey2 = result.get(1); + assertEquals( + resultForKey2.getProcessedResult().getWriteComputeResultWrapper().getNewPut().putValue.array(), + "value2".getBytes()); + } + +} diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/SITWithPWiseAndBufferAfterLeaderTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/SITWithPWiseAndBufferAfterLeaderTest.java index c694ce59495..297b8f77742 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/SITWithPWiseAndBufferAfterLeaderTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/kafka/consumer/SITWithPWiseAndBufferAfterLeaderTest.java @@ -8,4 +8,9 @@ protected KafkaConsumerService.ConsumerAssignmentStrategy getConsumerAssignmentS protected boolean isStoreWriterBufferAfterLeaderLogicEnabled() { return true; } + + @Override + protected boolean isAaWCParallelProcessingEnabled() { + return true; + } } 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 962d4e8bf8c..a5b5969a806 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 @@ -17,6 +17,7 @@ import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; import static com.linkedin.venice.ConfigKeys.KAFKA_CLUSTER_MAP_KEY_NAME; import static com.linkedin.venice.ConfigKeys.KAFKA_CLUSTER_MAP_KEY_URL; +import static com.linkedin.venice.ConfigKeys.SERVER_AA_WC_WORKLOAD_PARALLEL_PROCESSING_ENABLED; import static com.linkedin.venice.ConfigKeys.SERVER_DATABASE_CHECKSUM_VERIFICATION_ENABLED; import static com.linkedin.venice.ConfigKeys.SERVER_ENABLE_LIVE_CONFIG_BASED_KAFKA_THROTTLING; import static com.linkedin.venice.ConfigKeys.SERVER_INGESTION_HEARTBEAT_INTERVAL_MS; @@ -190,6 +191,7 @@ import com.linkedin.venice.utils.ByteArray; import com.linkedin.venice.utils.ByteUtils; import com.linkedin.venice.utils.ChunkingTestUtils; +import com.linkedin.venice.utils.DaemonThreadFactory; import com.linkedin.venice.utils.DataProviderUtils; import com.linkedin.venice.utils.DiskUsage; import com.linkedin.venice.utils.Pair; @@ -449,6 +451,10 @@ private static byte[] createReplicationMetadataWithValueSchemaId(long timestamp, return ByteUtils.extractByteArray(replicationMetadataWithValueSchemaId); } + protected boolean isAaWCParallelProcessingEnabled() { + return false; + } + @BeforeClass(alwaysRun = true) public void suiteSetUp() throws Exception { final Sensor mockSensor = mock(Sensor.class); @@ -1062,7 +1068,9 @@ private StoreIngestionTaskFactory.Builder getIngestionTaskFactoryBuilder( .setCompressorFactory(new StorageEngineBackedCompressorFactory(mockStorageMetadataService)) .setPubSubTopicRepository(pubSubTopicRepository) .setPartitionStateSerializer(partitionStateSerializer) - .setRunnableForKillIngestionTasksForNonCurrentVersions(runnableForKillNonCurrentVersion); + .setRunnableForKillIngestionTasksForNonCurrentVersions(runnableForKillNonCurrentVersion) + .setAAWCWorkLoadProcessingThreadPool( + Executors.newFixedThreadPool(2, new DaemonThreadFactory("AA_WC_PARALLEL_PROCESSING"))); } abstract KafkaConsumerService.ConsumerAssignmentStrategy getConsumerAssignmentStrategy(); @@ -2658,6 +2666,8 @@ private VeniceServerConfig buildVeniceServerConfig(Map extraProp remoteKafkaMapping.put(KAFKA_CLUSTER_MAP_KEY_URL, inMemoryRemoteKafkaBroker.getKafkaBootstrapServer()); kafkaClusterMap.put(String.valueOf(1), remoteKafkaMapping); + propertyBuilder.put(SERVER_AA_WC_WORKLOAD_PARALLEL_PROCESSING_ENABLED, isAaWCParallelProcessingEnabled()); + return new VeniceServerConfig(propertyBuilder.build(), kafkaClusterMap); } diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/VenicePushJob.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/VenicePushJob.java index f4f9343145c..024bd232d59 100755 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/VenicePushJob.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/VenicePushJob.java @@ -87,6 +87,7 @@ import static com.linkedin.venice.utils.ByteUtils.generateHumanReadableByteCountString; import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; +import com.linkedin.venice.PushJobCheckpoints; import com.linkedin.venice.compression.CompressionStrategy; import com.linkedin.venice.compression.ZstdWithDictCompressor; import com.linkedin.venice.controllerapi.ControllerClient; @@ -261,33 +262,6 @@ public class VenicePushJob implements AutoCloseable { private final PushJobHeartbeatSenderFactory pushJobHeartbeatSenderFactory; private boolean pushJobStatusUploadDisabledHasBeenLogged = false; - /** - * Different successful checkpoints and known error scenarios of the VPJ flow. - * 1. The enums are not sequential - * 2. Non-negative enums are successful checkpoints - * 3. Negative enums are error scenarios (Can be user or system errors) - */ - public enum PushJobCheckpoints { - INITIALIZE_PUSH_JOB(0), NEW_VERSION_CREATED(1), START_DATA_WRITER_JOB(2), DATA_WRITER_JOB_COMPLETED(3), - START_JOB_STATUS_POLLING(4), JOB_STATUS_POLLING_COMPLETED(5), START_VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB(6), - VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB_COMPLETED(7), QUOTA_EXCEEDED(-1), WRITE_ACL_FAILED(-2), - DUP_KEY_WITH_DIFF_VALUE(-3), INPUT_DATA_SCHEMA_VALIDATION_FAILED(-4), - EXTENDED_INPUT_DATA_SCHEMA_VALIDATION_FAILED(-5), RECORD_TOO_LARGE_FAILED(-6), CONCURRENT_BATCH_PUSH(-7), - DATASET_CHANGED(-8), INVALID_INPUT_FILE(-9), ZSTD_DICTIONARY_CREATION_FAILED(-10), - DVC_INGESTION_ERROR_DISK_FULL(-11), DVC_INGESTION_ERROR_MEMORY_LIMIT_REACHED(-12), - DVC_INGESTION_ERROR_TOO_MANY_DEAD_INSTANCES(-13), DVC_INGESTION_ERROR_OTHER(-14); - - private final int value; - - PushJobCheckpoints(int value) { - this.value = value; - } - - public int getValue() { - return value; - } - } - /** * @param jobId id of the job * @param vanillaProps Property bag for the job diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/TestVenicePushJobCheckpoints.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/TestVenicePushJobCheckpoints.java index cb6f295b540..d2bec29432c 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/TestVenicePushJobCheckpoints.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/TestVenicePushJobCheckpoints.java @@ -29,6 +29,7 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; +import com.linkedin.venice.PushJobCheckpoints; import com.linkedin.venice.compression.CompressionStrategy; import com.linkedin.venice.compression.ZstdWithDictCompressor; import com.linkedin.venice.controllerapi.ControllerClient; @@ -100,9 +101,9 @@ public void testHandleQuotaExceeded() throws Exception { // All reducers closed new MockCounterInfo(MRJobCounterHelper.REDUCER_CLOSED_COUNT_GROUP_COUNTER_NAME, PARTITION_COUNT)), Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.QUOTA_EXCEEDED), + PushJobCheckpoints.INITIALIZE_PUSH_JOB, + PushJobCheckpoints.NEW_VERSION_CREATED, + PushJobCheckpoints.QUOTA_EXCEEDED), properties -> { properties.setProperty(USE_MAPPER_TO_BUILD_DICTIONARY, "false"); properties.setProperty(COMPRESSION_METRIC_COLLECTION_ENABLED, "false"); @@ -128,10 +129,10 @@ public void testHandleQuotaExceededWithMapperToBuildDict() throws Exception { MRJobCounterHelper.MAPPER_NUM_RECORDS_SUCCESSFULLY_PROCESSED_GROUP_COUNTER_NAME, NUMBER_OF_FILES_TO_READ_AND_BUILD_DICT_COUNT + 1)), Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB_COMPLETED, - VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.QUOTA_EXCEEDED), + PushJobCheckpoints.INITIALIZE_PUSH_JOB, + PushJobCheckpoints.VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB_COMPLETED, + PushJobCheckpoints.NEW_VERSION_CREATED, + PushJobCheckpoints.QUOTA_EXCEEDED), properties -> { properties.setProperty(USE_MAPPER_TO_BUILD_DICTIONARY, "true"); properties.setProperty(COMPRESSION_METRIC_COLLECTION_ENABLED, "false"); @@ -143,18 +144,18 @@ public void testHandleQuotaExceededWithMapperToBuildDict() throws Exception { */ @Test(expectedExceptions = VeniceException.class, expectedExceptionsMessageRegExp = "Storage quota exceeded.*", dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) public void testHandleQuotaExceededWithCompressionCollectionEnabled(boolean useMapperToBuildDict) throws Exception { - List expectedCheckpoints; + List expectedCheckpoints; if (useMapperToBuildDict) { expectedCheckpoints = Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB_COMPLETED, - VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.QUOTA_EXCEEDED); + PushJobCheckpoints.INITIALIZE_PUSH_JOB, + PushJobCheckpoints.VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB_COMPLETED, + PushJobCheckpoints.NEW_VERSION_CREATED, + PushJobCheckpoints.QUOTA_EXCEEDED); } else { expectedCheckpoints = Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.QUOTA_EXCEEDED); + PushJobCheckpoints.INITIALIZE_PUSH_JOB, + PushJobCheckpoints.NEW_VERSION_CREATED, + PushJobCheckpoints.QUOTA_EXCEEDED); } testHandleErrorsInCounter( @@ -189,10 +190,10 @@ public void testWithNoMapperToBuildDictionary() throws Exception { // All reducers closed new MockCounterInfo(MRJobCounterHelper.REDUCER_CLOSED_COUNT_GROUP_COUNTER_NAME, PARTITION_COUNT)), Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, - VenicePushJob.PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), + PushJobCheckpoints.INITIALIZE_PUSH_JOB, + PushJobCheckpoints.NEW_VERSION_CREATED, + PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, + PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), properties -> { properties.setProperty(USE_MAPPER_TO_BUILD_DICTIONARY, "false"); properties.setProperty(COMPRESSION_METRIC_COLLECTION_ENABLED, "false"); @@ -214,11 +215,11 @@ public void testWithMapperToBuildDictionary() throws Exception { MRJobCounterHelper.MAPPER_NUM_RECORDS_SUCCESSFULLY_PROCESSED_GROUP_COUNTER_NAME, NUMBER_OF_FILES_TO_READ_AND_BUILD_DICT_COUNT + 1)), Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB_COMPLETED, - VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, - VenicePushJob.PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), + PushJobCheckpoints.INITIALIZE_PUSH_JOB, + PushJobCheckpoints.VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB_COMPLETED, + PushJobCheckpoints.NEW_VERSION_CREATED, + PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, + PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), properties -> { properties.setProperty(USE_MAPPER_TO_BUILD_DICTIONARY, "true"); properties.setProperty(COMPRESSION_METRIC_COLLECTION_ENABLED, "false"); @@ -235,10 +236,10 @@ public void testWithCompressionCollectionDisabled() throws Exception { // All reducers closed new MockCounterInfo(MRJobCounterHelper.REDUCER_CLOSED_COUNT_GROUP_COUNTER_NAME, PARTITION_COUNT)), Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, - VenicePushJob.PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), + PushJobCheckpoints.INITIALIZE_PUSH_JOB, + PushJobCheckpoints.NEW_VERSION_CREATED, + PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, + PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), properties -> { properties.setProperty(USE_MAPPER_TO_BUILD_DICTIONARY, "false"); properties.setProperty(COMPRESSION_METRIC_COLLECTION_ENABLED, "false"); @@ -247,20 +248,20 @@ public void testWithCompressionCollectionDisabled() throws Exception { @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) public void testWithCompressionCollectionEnabled(boolean useMapperToBuildDict) throws Exception { - List expectedCheckpoints; + List expectedCheckpoints; if (useMapperToBuildDict) { expectedCheckpoints = Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB_COMPLETED, - VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, - VenicePushJob.PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED); + PushJobCheckpoints.INITIALIZE_PUSH_JOB, + PushJobCheckpoints.VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB_COMPLETED, + PushJobCheckpoints.NEW_VERSION_CREATED, + PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, + PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED); } else { expectedCheckpoints = Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, - VenicePushJob.PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED); + PushJobCheckpoints.INITIALIZE_PUSH_JOB, + PushJobCheckpoints.NEW_VERSION_CREATED, + PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, + PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED); } testHandleErrorsInCounter( @@ -306,11 +307,11 @@ public void testHandlingFailureWithCompressionCollectionEnabled() throws Excepti // Dictionary building succeeded if enabled new MockCounterInfo(MRJobCounterHelper.MAPPER_ZSTD_DICT_TRAIN_FAILURE_GROUP_COUNTER_NAME, 1)), Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB_COMPLETED, - VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, - VenicePushJob.PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), + PushJobCheckpoints.INITIALIZE_PUSH_JOB, + PushJobCheckpoints.VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB_COMPLETED, + PushJobCheckpoints.NEW_VERSION_CREATED, + PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, + PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), properties -> { properties.setProperty(USE_MAPPER_TO_BUILD_DICTIONARY, "true"); properties.setProperty(COMPRESSION_METRIC_COLLECTION_ENABLED, "true"); @@ -338,9 +339,7 @@ public void testHandlingFailureWithCompressionCollectionEnabledAndZstdCompressio NUMBER_OF_FILES_TO_READ_AND_BUILD_DICT_COUNT), // no +1 as the last part (build dict) failed // Dictionary building succeeded if enabled new MockCounterInfo(MRJobCounterHelper.MAPPER_ZSTD_DICT_TRAIN_FAILURE_GROUP_COUNTER_NAME, 1)), - Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.ZSTD_DICTIONARY_CREATION_FAILED), + Arrays.asList(PushJobCheckpoints.INITIALIZE_PUSH_JOB, PushJobCheckpoints.ZSTD_DICTIONARY_CREATION_FAILED), properties -> { properties.setProperty(USE_MAPPER_TO_BUILD_DICTIONARY, "true"); properties.setProperty(COMPRESSION_METRIC_COLLECTION_ENABLED, "true"); @@ -370,11 +369,11 @@ public void testHandlingSkippedWithCompressionCollectionEnabled() throws Excepti // Dictionary building succeeded if enabled new MockCounterInfo(MRJobCounterHelper.MAPPER_ZSTD_DICT_TRAIN_SKIPPED_GROUP_COUNTER_NAME, 1)), Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB_COMPLETED, - VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, - VenicePushJob.PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), + PushJobCheckpoints.INITIALIZE_PUSH_JOB, + PushJobCheckpoints.VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB_COMPLETED, + PushJobCheckpoints.NEW_VERSION_CREATED, + PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, + PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), properties -> { properties.setProperty(USE_MAPPER_TO_BUILD_DICTIONARY, "true"); properties.setProperty(COMPRESSION_METRIC_COLLECTION_ENABLED, "true"); @@ -402,9 +401,7 @@ public void testHandlingSkippedWithCompressionCollectionEnabledAndZstdCompressio NUMBER_OF_FILES_TO_READ_AND_BUILD_DICT_COUNT), // no +1 as the last part (build dict) failed // Dictionary building succeeded if enabled new MockCounterInfo(MRJobCounterHelper.MAPPER_ZSTD_DICT_TRAIN_SKIPPED_GROUP_COUNTER_NAME, 1)), - Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.ZSTD_DICTIONARY_CREATION_FAILED), + Arrays.asList(PushJobCheckpoints.INITIALIZE_PUSH_JOB, PushJobCheckpoints.ZSTD_DICTIONARY_CREATION_FAILED), properties -> { properties.setProperty(USE_MAPPER_TO_BUILD_DICTIONARY, "true"); properties.setProperty(COMPRESSION_METRIC_COLLECTION_ENABLED, "true"); @@ -422,9 +419,9 @@ public void testHandleWriteAclFailed() throws Exception { // All reducers closed new MockCounterInfo(MRJobCounterHelper.REDUCER_CLOSED_COUNT_GROUP_COUNTER_NAME, PARTITION_COUNT)), Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.WRITE_ACL_FAILED), + PushJobCheckpoints.INITIALIZE_PUSH_JOB, + PushJobCheckpoints.NEW_VERSION_CREATED, + PushJobCheckpoints.WRITE_ACL_FAILED), properties -> { properties.setProperty(COMPRESSION_METRIC_COLLECTION_ENABLED, "false"); properties.setProperty(USE_MAPPER_TO_BUILD_DICTIONARY, "false"); @@ -442,9 +439,9 @@ public void testHandleDuplicatedKeyWithDistinctValue() throws Exception { // All reducers closed new MockCounterInfo(MRJobCounterHelper.REDUCER_CLOSED_COUNT_GROUP_COUNTER_NAME, PARTITION_COUNT)), Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.DUP_KEY_WITH_DIFF_VALUE), + PushJobCheckpoints.INITIALIZE_PUSH_JOB, + PushJobCheckpoints.NEW_VERSION_CREATED, + PushJobCheckpoints.DUP_KEY_WITH_DIFF_VALUE), properties -> { properties.setProperty(COMPRESSION_METRIC_COLLECTION_ENABLED, "false"); properties.setProperty(USE_MAPPER_TO_BUILD_DICTIONARY, "false"); @@ -466,9 +463,9 @@ public void testHandleZeroClosedReducersFailure() throws Exception { // No reducers at all closed new MockCounterInfo(MRJobCounterHelper.REDUCER_CLOSED_COUNT_GROUP_COUNTER_NAME, 0)), Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.START_DATA_WRITER_JOB), + PushJobCheckpoints.INITIALIZE_PUSH_JOB, + PushJobCheckpoints.NEW_VERSION_CREATED, + PushJobCheckpoints.START_DATA_WRITER_JOB), 10L, // Non-empty input data file properties -> { properties.setProperty(COMPRESSION_METRIC_COLLECTION_ENABLED, "false"); @@ -491,9 +488,9 @@ public void testUnreliableMapReduceCounter() throws Exception { // No reducers at all closed new MockCounterInfo(MRJobCounterHelper.REDUCER_CLOSED_COUNT_GROUP_COUNTER_NAME, 0)), Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.START_DATA_WRITER_JOB), + PushJobCheckpoints.INITIALIZE_PUSH_JOB, + PushJobCheckpoints.NEW_VERSION_CREATED, + PushJobCheckpoints.START_DATA_WRITER_JOB), 10L, // Non-empty input data file 1, true, @@ -516,10 +513,10 @@ public void testHandleZeroClosedReducersWithNoRecordInputDataFile() throws Excep // No reducers at all closed new MockCounterInfo(MRJobCounterHelper.REDUCER_CLOSED_COUNT_GROUP_COUNTER_NAME, 0)), Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, - VenicePushJob.PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED // Expect the job to finish successfully + PushJobCheckpoints.INITIALIZE_PUSH_JOB, + PushJobCheckpoints.NEW_VERSION_CREATED, + PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, + PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED // Expect the job to finish successfully ), 10L, 1, @@ -557,9 +554,9 @@ public void testHandleInsufficientClosedReducersFailure() throws Exception { // Some but not all reducers closed new MockCounterInfo(MRJobCounterHelper.REDUCER_CLOSED_COUNT_GROUP_COUNTER_NAME, PARTITION_COUNT - 1)), Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.START_DATA_WRITER_JOB), + PushJobCheckpoints.INITIALIZE_PUSH_JOB, + PushJobCheckpoints.NEW_VERSION_CREATED, + PushJobCheckpoints.START_DATA_WRITER_JOB), properties -> { properties.setProperty(COMPRESSION_METRIC_COLLECTION_ENABLED, "false"); properties.setProperty(USE_MAPPER_TO_BUILD_DICTIONARY, "false"); @@ -582,10 +579,10 @@ public void testCounterValidationWhenSprayAllPartitionsNotTriggeredButWithMismat // Some but not all reducers closed new MockCounterInfo(MRJobCounterHelper.REDUCER_CLOSED_COUNT_GROUP_COUNTER_NAME, PARTITION_COUNT - 1)), Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, - VenicePushJob.PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), + PushJobCheckpoints.INITIALIZE_PUSH_JOB, + PushJobCheckpoints.NEW_VERSION_CREATED, + PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, + PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), properties -> { properties.setProperty(COMPRESSION_METRIC_COLLECTION_ENABLED, "false"); properties.setProperty(USE_MAPPER_TO_BUILD_DICTIONARY, "false"); @@ -605,10 +602,10 @@ public void testHandleNoErrorInCounters() throws Exception { // Successful workf // All reducers closed new MockCounterInfo(MRJobCounterHelper.REDUCER_CLOSED_COUNT_GROUP_COUNTER_NAME, PARTITION_COUNT)), Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, - VenicePushJob.PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), + PushJobCheckpoints.INITIALIZE_PUSH_JOB, + PushJobCheckpoints.NEW_VERSION_CREATED, + PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, + PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), properties -> { properties.setProperty(COMPRESSION_METRIC_COLLECTION_ENABLED, "false"); properties.setProperty(USE_MAPPER_TO_BUILD_DICTIONARY, "false"); @@ -623,19 +620,18 @@ public void testHandleMRFailureAndDatasetChange( JobClientWrapper jobClientWrapper = mock(JobClientWrapper.class); when(jobClientWrapper.runJobWithConfig(any())).thenThrow(new IOException("Job failed!")); - final List expectedReportedCheckpoints; + final List expectedReportedCheckpoints; if (useMapperToBuildDict) { /** Uses {@link ValidateSchemaAndBuildDictMapper} to validate schema and build dictionary which will checkpoint DATASET_CHANGED before NEW_VERSION_CREATED */ - expectedReportedCheckpoints = Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.DATASET_CHANGED); + expectedReportedCheckpoints = + Arrays.asList(PushJobCheckpoints.INITIALIZE_PUSH_JOB, PushJobCheckpoints.DATASET_CHANGED); } else { /** {@link InputDataInfoProvider#validateInputAndGetInfo} in VPJ driver validates schema and build dictionary which will checkpoint NEW_VERSION_CREATED before DATASET_CHANGED. * DATASET_CHANGED will only be checked in the MR job to process data after creating the new version */ expectedReportedCheckpoints = Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.DATASET_CHANGED); + PushJobCheckpoints.INITIALIZE_PUSH_JOB, + PushJobCheckpoints.NEW_VERSION_CREATED, + PushJobCheckpoints.DATASET_CHANGED); } runJobAndAssertCheckpoints(jobClientWrapper, 10, 1, true, true, ExecutionStatus.COMPLETED, properties -> { @@ -660,12 +656,12 @@ public void testHandleDVCFailureCheckpoints(ExecutionStatus status) throws Excep JobClientWrapper jobClientWrapper = mock(JobClientWrapper.class); doAnswer(invocation -> null).when(jobClientWrapper).runJobWithConfig(any()); - final List expectedReportedCheckpoints; + final List expectedReportedCheckpoints; expectedReportedCheckpoints = Arrays.asList( - VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, - VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, - VenicePushJob.PushJobCheckpoints.valueOf(status.toString())); + PushJobCheckpoints.INITIALIZE_PUSH_JOB, + PushJobCheckpoints.NEW_VERSION_CREATED, + PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, + PushJobCheckpoints.valueOf(status.toString())); runJobAndAssertCheckpoints( jobClientWrapper, @@ -680,14 +676,14 @@ public void testHandleDVCFailureCheckpoints(ExecutionStatus status) throws Excep private void testHandleErrorsInCounter( List mockCounterInfos, - List expectedReportedCheckpoints, + List expectedReportedCheckpoints, Consumer extraProps) throws Exception { testHandleErrorsInCounter(mockCounterInfos, expectedReportedCheckpoints, 10L, extraProps); } private void testHandleErrorsInCounter( List mockCounterInfos, - List expectedReportedCheckpoints, + List expectedReportedCheckpoints, long inputFileDataSizeInBytes, Consumer extraProps) throws Exception { testHandleErrorsInCounter( @@ -701,7 +697,7 @@ private void testHandleErrorsInCounter( private void testHandleErrorsInCounter( List mockCounterInfos, - List expectedReportedCheckpoints, + List expectedReportedCheckpoints, long inputFileDataSizeInBytes, int numInputFiles, boolean inputFileHasRecords, @@ -725,7 +721,7 @@ private void runJobAndAssertCheckpoints( boolean datasetChanged, ExecutionStatus executionStatus, Consumer extraProps, - List expectedReportedCheckpoints) throws Exception { + List expectedReportedCheckpoints) throws Exception { Properties props = getVPJProps(); if (extraProps != null) { extraProps.accept(props); @@ -769,9 +765,8 @@ private void runJobAndAssertCheckpoints( for (PushJobDetails pushJobDetails: pushJobDetailsTracker.getRecordedPushJobDetails()) { actualReportedCheckpointValues.add(pushJobDetails.pushJobLatestCheckpoint); } - List expectedCheckpointValues = expectedReportedCheckpoints.stream() - .map(VenicePushJob.PushJobCheckpoints::getValue) - .collect(Collectors.toList()); + List expectedCheckpointValues = + expectedReportedCheckpoints.stream().map(PushJobCheckpoints::getValue).collect(Collectors.toList()); Assert.assertEquals(actualReportedCheckpointValues, expectedCheckpointValues); } diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/VenicePushJobTest.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/VenicePushJobTest.java index 2a0295fc17b..f5ca106d241 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/VenicePushJobTest.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/VenicePushJobTest.java @@ -54,6 +54,7 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import com.linkedin.venice.PushJobCheckpoints; import com.linkedin.venice.compression.CompressionStrategy; import com.linkedin.venice.controllerapi.ControllerClient; import com.linkedin.venice.controllerapi.ControllerResponse; @@ -883,7 +884,7 @@ public void testGetPerColoPushJobDetailsStatusFromExecutionStatus() { } /** - * Tests that the error message for the {@link VenicePushJob.PushJobCheckpoints#RECORD_TOO_LARGE_FAILED} code path of + * Tests that the error message for the {@link com.linkedin.venice.PushJobCheckpoints#RECORD_TOO_LARGE_FAILED} code path of * {@link VenicePushJob#updatePushJobDetailsWithJobDetails(DataWriterTaskTracker)} uses maxRecordSizeBytes. */ @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) @@ -901,7 +902,7 @@ public void testUpdatePushJobDetailsWithJobDetailsRecordTooLarge(boolean chunkin final String errorMessage = vpj.updatePushJobDetailsWithJobDetails(dataWriterTaskTracker); final int latestCheckpoint = pushJobDetails.pushJobLatestCheckpoint; Assert.assertTrue(errorMessage.contains((chunkingEnabled) ? "100.0 MiB" : "950.0 KiB"), errorMessage); - Assert.assertEquals(latestCheckpoint, VenicePushJob.PushJobCheckpoints.RECORD_TOO_LARGE_FAILED.getValue()); + Assert.assertEquals(latestCheckpoint, PushJobCheckpoints.RECORD_TOO_LARGE_FAILED.getValue()); } } diff --git a/clients/venice-thin-client/src/main/java/com/linkedin/venice/client/schema/RouterBackedSchemaReader.java b/clients/venice-thin-client/src/main/java/com/linkedin/venice/client/schema/RouterBackedSchemaReader.java index 51d18e3fb64..9ca49460a8c 100644 --- a/clients/venice-thin-client/src/main/java/com/linkedin/venice/client/schema/RouterBackedSchemaReader.java +++ b/clients/venice-thin-client/src/main/java/com/linkedin/venice/client/schema/RouterBackedSchemaReader.java @@ -324,9 +324,13 @@ private void updateAllValueSchemas(boolean forceRefresh) { valueSchemaIdSet = fetchAllValueSchemaIdsFromRouter(); } catch (Exception e) { LOGGER.warn( - "Caught exception when trying to fetch all value schema IDs from router, will fetch all value schema entries instead."); + "Caught exception when trying to fetch all value schema IDs from router, will fetch all value schema entries instead.", + e); // Fall back to fetch all value schema. for (SchemaEntry valueSchemaEntry: fetchAllValueSchemaEntriesFromRouter()) { + if (!isValidSchemaEntry(valueSchemaEntry)) { + continue; + } valueSchemaEntryMap.put(valueSchemaEntry.getId(), valueSchemaEntry); cacheValueAndCanonicalSchemas(valueSchemaEntry.getSchema(), valueSchemaEntry.getId()); } @@ -441,7 +445,7 @@ private SchemaEntry maybeFetchLatestValueSchemaEntry() { * one active value schema. */ synchronized (this) { - if (latest != null && !shouldRefreshLatestValueSchemaEntry.get()) { + if (latest != null && !shouldRefreshLatestValueSchemaEntry.get() && isValidSchemaEntry(latest)) { return latest; } updateAllValueSchemaEntriesAndLatestValueSchemaEntry(false); @@ -450,6 +454,9 @@ private SchemaEntry maybeFetchLatestValueSchemaEntry() { latest = latestValueSchemaEntry.get(); } } + if (latest == null || !isValidSchemaEntry(latest)) { + throw new VeniceClientException("Failed to get latest value schema for store: " + storeName); + } return latest; } diff --git a/clients/venice-thin-client/src/main/java/com/linkedin/venice/client/store/AbstractAvroComputeRequestBuilder.java b/clients/venice-thin-client/src/main/java/com/linkedin/venice/client/store/AbstractAvroComputeRequestBuilder.java index be265614cdb..ca8ef1cbb9a 100644 --- a/clients/venice-thin-client/src/main/java/com/linkedin/venice/client/store/AbstractAvroComputeRequestBuilder.java +++ b/clients/venice-thin-client/src/main/java/com/linkedin/venice/client/store/AbstractAvroComputeRequestBuilder.java @@ -21,6 +21,7 @@ import com.linkedin.venice.compute.protocol.request.DotProduct; import com.linkedin.venice.compute.protocol.request.HadamardProduct; import com.linkedin.venice.compute.protocol.request.enums.ComputeOperationType; +import com.linkedin.venice.schema.SchemaData; import com.linkedin.venice.schema.SchemaReader; import com.linkedin.venice.utils.Pair; import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap; @@ -80,6 +81,9 @@ public abstract class AbstractAvroComputeRequestBuilder implements ComputeReq public AbstractAvroComputeRequestBuilder(AvroGenericReadComputeStoreClient storeClient, SchemaReader schemaReader) { this.latestValueSchemaId = schemaReader.getLatestValueSchemaId(); + if (latestValueSchemaId == SchemaData.INVALID_VALUE_SCHEMA_ID) { + throw new VeniceClientException("Invalid value schema ID: " + latestValueSchemaId); + } this.latestValueSchema = schemaReader.getValueSchema(latestValueSchemaId); if (latestValueSchema.getType() != Schema.Type.RECORD) { throw new VeniceClientException("Only value schema with 'RECORD' type is supported"); diff --git a/clients/venice-thin-client/src/test/java/com/linkedin/venice/client/schema/RouterBackedSchemaReaderTest.java b/clients/venice-thin-client/src/test/java/com/linkedin/venice/client/schema/RouterBackedSchemaReaderTest.java index 07d545319dc..9ce30dc5d4d 100644 --- a/clients/venice-thin-client/src/test/java/com/linkedin/venice/client/schema/RouterBackedSchemaReaderTest.java +++ b/clients/venice-thin-client/src/test/java/com/linkedin/venice/client/schema/RouterBackedSchemaReaderTest.java @@ -386,7 +386,7 @@ public void testGetLatestValueSchemaWhenNoValueSchema() 0); try (SchemaReader schemaReader = new RouterBackedSchemaReader(() -> mockClient)) { - Assert.assertNull(schemaReader.getLatestValueSchema()); + Assert.assertThrows(VeniceClientException.class, () -> schemaReader.getLatestValueSchema()); Mockito.verify(mockClient, Mockito.timeout(TIMEOUT).times(1)).getRaw(Mockito.anyString()); } } diff --git a/docs/dev_guide/how_to/style_guide.md b/docs/dev_guide/how_to/style_guide.md index 44faa6dbdb6..37a30b73237 100644 --- a/docs/dev_guide/how_to/style_guide.md +++ b/docs/dev_guide/how_to/style_guide.md @@ -42,6 +42,20 @@ rules and there may be good reasons to deviate from them occasionally. When devi explaining why we deviated, whether it was intentional, or due to the need for expediency. This helps future maintainers understand what is actually worth cleaning up and how careful they need to be when doing it. +### Compatibility + +We care about compatibility across versions of the software. This is a bidirectional statement. Old clients need to be +able to talk with new servers, and new clients with old servers as well. It also includes interactions across lifetimes +of the same process, for example, state persisted by an old version of the server code should be usable by a newer +version of the server code, and vice versa. If compatibility is impossible, then we should look for ways to achieve +correct behavior anyway (e.g. potentially at the cost of efficiency, such as the server needing to throw away and +regenerate the state). + +For enums which are going to be communicated across processes or across lifetimes of the same process, consider using +[VeniceEnumValue](http://venicedb.org/javadoc/com/linkedin/venice/utils/VeniceEnumValue.html), [EnumUtils](http://venicedb.org/javadoc/com/linkedin/venice/utils/EnumUtils.html) +and related unit test classes, which provide a structure to minimize the chance that we mistakenly change the mapping of +numeric ID -> enum value. + ### JavaDoc Speaking of comments, we ideally want JavaDoc at the top of all classes. The top of class JavaDoc should indicate the diff --git a/internal/venice-client-common/src/main/java/com/linkedin/venice/compression/CompressionStrategy.java b/internal/venice-client-common/src/main/java/com/linkedin/venice/compression/CompressionStrategy.java index 99d9366a0d5..30a1e1de69b 100644 --- a/internal/venice-client-common/src/main/java/com/linkedin/venice/compression/CompressionStrategy.java +++ b/internal/venice-client-common/src/main/java/com/linkedin/venice/compression/CompressionStrategy.java @@ -1,8 +1,8 @@ package com.linkedin.venice.compression; -import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.utils.EnumUtils; import com.linkedin.venice.utils.VeniceEnumValue; +import java.util.List; /** @@ -15,13 +15,14 @@ public enum CompressionStrategy implements VeniceEnumValue { private final int value; private final boolean compressionEnabled; - private static final CompressionStrategy[] TYPES_ARRAY = EnumUtils.getEnumValuesArray(CompressionStrategy.class); + private static final List TYPES = EnumUtils.getEnumValuesList(CompressionStrategy.class); CompressionStrategy(int value, boolean compressionEnabled) { this.value = value; this.compressionEnabled = compressionEnabled; } + @Override public int getValue() { return value; } @@ -31,14 +32,10 @@ public boolean isCompressionEnabled() { } public static CompressionStrategy valueOf(int value) { - try { - return TYPES_ARRAY[value]; - } catch (IndexOutOfBoundsException e) { - throw new VeniceException("Invalid compression strategy: " + value); - } + return EnumUtils.valueOf(TYPES, value, CompressionStrategy.class); } public static int getCompressionStrategyTypesArrayLength() { - return TYPES_ARRAY.length; + return TYPES.size(); } } diff --git a/internal/venice-client-common/src/main/java/com/linkedin/venice/compute/protocol/request/enums/ComputeOperationType.java b/internal/venice-client-common/src/main/java/com/linkedin/venice/compute/protocol/request/enums/ComputeOperationType.java index 1e574e262d1..340b0544452 100644 --- a/internal/venice-client-common/src/main/java/com/linkedin/venice/compute/protocol/request/enums/ComputeOperationType.java +++ b/internal/venice-client-common/src/main/java/com/linkedin/venice/compute/protocol/request/enums/ComputeOperationType.java @@ -13,6 +13,7 @@ import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.utils.EnumUtils; import com.linkedin.venice.utils.VeniceEnumValue; +import java.util.List; public enum ComputeOperationType implements VeniceEnumValue { @@ -21,7 +22,7 @@ public enum ComputeOperationType implements VeniceEnumValue { private final ReadComputeOperator operator; private final int value; - private static final ComputeOperationType[] TYPES_ARRAY = EnumUtils.getEnumValuesArray(ComputeOperationType.class); + private static final List TYPES = EnumUtils.getEnumValuesList(ComputeOperationType.class); ComputeOperationType(int value, ReadComputeOperator operator) { this.value = value; @@ -44,17 +45,14 @@ public Object getNewInstance() { } public static ComputeOperationType valueOf(int value) { - try { - return TYPES_ARRAY[value]; - } catch (IndexOutOfBoundsException e) { - throw new VeniceException("Invalid compute operation type: " + value); - } + return EnumUtils.valueOf(TYPES, value, ComputeOperationType.class); } public static ComputeOperationType valueOf(ComputeOperation operation) { return valueOf(operation.operationType); } + @Override public int getValue() { return value; } diff --git a/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/EnumUtils.java b/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/EnumUtils.java index fb1258547d6..417d73a9527 100644 --- a/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/EnumUtils.java +++ b/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/EnumUtils.java @@ -1,6 +1,11 @@ package com.linkedin.venice.utils; +import com.linkedin.venice.exceptions.VeniceException; import java.lang.reflect.Array; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.function.Function; public class EnumUtils { @@ -15,8 +20,12 @@ public class EnumUtils { * valuable since it's a hot path call. If these assumptions change (e.g. if we deprecate some message * types such that there are gaps, then we may need to relax some constraints here and increase checks * in valueOf(int) instead. + * + * The list returned by this utility function should: + * - be stored statically + * - be accessed via {@link #valueOf(List, int, Class)} */ - public static V[] getEnumValuesArray(Class enumToProvideArrayOf) { + public static List getEnumValuesList(Class enumToProvideArrayOf) { int maxValue = -1; String name = enumToProvideArrayOf.getSimpleName(); for (V type: enumToProvideArrayOf.getEnumConstants()) { @@ -44,6 +53,22 @@ public static V[] getEnumValuesArray(Class enumTo name + " values should not have gaps, but " + i + " is not associated with any type!"); } } - return array; + return Collections.unmodifiableList(Arrays.asList(array)); + } + + public static V valueOf(List valuesList, int value, Class enumClass) { + return valueOf(valuesList, value, enumClass, VeniceException::new); + } + + public static V valueOf( + List valuesList, + int value, + Class enumClass, + Function exceptionConstructor) { + try { + return valuesList.get(value); + } catch (IndexOutOfBoundsException e) { + throw exceptionConstructor.apply("Invalid enum value for " + enumClass.getSimpleName() + ": " + value); + } } } diff --git a/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/ValueHolder.java b/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/ValueHolder.java new file mode 100644 index 00000000000..4511e567a07 --- /dev/null +++ b/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/ValueHolder.java @@ -0,0 +1,17 @@ +package com.linkedin.venice.utils; + +public class ValueHolder { + private T value; + + public ValueHolder(T value) { + this.value = value; + } + + public T getValue() { + return value; + } + + public void setValue(T value) { + this.value = value; + } +} diff --git a/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/VeniceEnumValue.java b/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/VeniceEnumValue.java index d3fdd433a9a..fbbca028162 100644 --- a/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/VeniceEnumValue.java +++ b/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/VeniceEnumValue.java @@ -1,5 +1,18 @@ package com.linkedin.venice.utils; +/** + * N.B.: Although there is no way to force this via Java interfaces, the convention is that all enums implementing this + * interface should have static "valueOf" function to return the correct enum value from a given numeric value, i.e.: + * + * {@snippet id='valueOf': + * public static MyEnumType valueOf(int value) { + * return EnumUtils.valueOf(TYPES_ARRAY, value, MyEnumTpe.class); + * } + * } + * + * Note that VeniceEnumValueTest makes it easy to test the above, and we should have a subclass of that test for all + * implementations of this interface. + */ public interface VeniceEnumValue { int getValue(); } diff --git a/internal/venice-client-common/src/test/java/com/linkedin/venice/compression/CompressionStrategyTest.java b/internal/venice-client-common/src/test/java/com/linkedin/venice/compression/CompressionStrategyTest.java new file mode 100644 index 00000000000..a8f54c296be --- /dev/null +++ b/internal/venice-client-common/src/test/java/com/linkedin/venice/compression/CompressionStrategyTest.java @@ -0,0 +1,22 @@ +package com.linkedin.venice.compression; + +import com.linkedin.alpini.base.misc.CollectionUtil; +import com.linkedin.venice.utils.VeniceEnumValueTest; +import java.util.Map; + + +public class CompressionStrategyTest extends VeniceEnumValueTest { + public CompressionStrategyTest() { + super(CompressionStrategy.class); + } + + @Override + protected Map expectedMapping() { + return CollectionUtil.mapBuilder() + .put(0, CompressionStrategy.NO_OP) + .put(1, CompressionStrategy.GZIP) + .put(2, CompressionStrategy.ZSTD) + .put(3, CompressionStrategy.ZSTD_WITH_DICT) + .build(); + } +} diff --git a/internal/venice-client-common/src/test/java/com/linkedin/venice/compute/protocol/request/enums/ComputeOperationTypeTest.java b/internal/venice-client-common/src/test/java/com/linkedin/venice/compute/protocol/request/enums/ComputeOperationTypeTest.java new file mode 100644 index 00000000000..b2f4568d7b9 --- /dev/null +++ b/internal/venice-client-common/src/test/java/com/linkedin/venice/compute/protocol/request/enums/ComputeOperationTypeTest.java @@ -0,0 +1,22 @@ +package com.linkedin.venice.compute.protocol.request.enums; + +import com.linkedin.alpini.base.misc.CollectionUtil; +import com.linkedin.venice.utils.VeniceEnumValueTest; +import java.util.Map; + + +public class ComputeOperationTypeTest extends VeniceEnumValueTest { + public ComputeOperationTypeTest() { + super(ComputeOperationType.class); + } + + @Override + protected Map expectedMapping() { + return CollectionUtil.mapBuilder() + .put(0, ComputeOperationType.DOT_PRODUCT) + .put(1, ComputeOperationType.COSINE_SIMILARITY) + .put(2, ComputeOperationType.HADAMARD_PRODUCT) + .put(3, ComputeOperationType.COUNT) + .build(); + } +} 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 8dd658f7b8f..3aed8933cf9 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 @@ -2234,4 +2234,10 @@ private ConfigKeys() { public static final String SERVER_CHANNEL_OPTION_WRITE_BUFFER_WATERMARK_HIGH_BYTES = "server.channel.option.write.buffer.watermark.high.bytes"; + + public static final String SERVER_AA_WC_WORKLOAD_PARALLEL_PROCESSING_ENABLED = + "server.aa.wc.workload.parallel.processing.enabled"; + + public static final String SERVER_AA_WC_WORKLOAD_PARALLEL_PROCESSING_THREAD_POOL_SIZE = + "server.aa.wc.workload.parallel.processing.thread.pool.size"; } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/PushJobCheckpoints.java b/internal/venice-common/src/main/java/com/linkedin/venice/PushJobCheckpoints.java new file mode 100644 index 00000000000..e3dda2cf8f5 --- /dev/null +++ b/internal/venice-common/src/main/java/com/linkedin/venice/PushJobCheckpoints.java @@ -0,0 +1,45 @@ +package com.linkedin.venice; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + + +/** + * Different successful checkpoints and known error scenarios of the VPJ flow. + * 1. The enums are not sequential + * 2. Non-negative enums are successful checkpoints + * 3. Negative enums are error scenarios (Can be user or system errors) + */ +public enum PushJobCheckpoints { + INITIALIZE_PUSH_JOB(0), NEW_VERSION_CREATED(1), START_DATA_WRITER_JOB(2), DATA_WRITER_JOB_COMPLETED(3), + START_JOB_STATUS_POLLING(4), JOB_STATUS_POLLING_COMPLETED(5), START_VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB(6), + VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB_COMPLETED(7), QUOTA_EXCEEDED(-1), WRITE_ACL_FAILED(-2), + DUP_KEY_WITH_DIFF_VALUE(-3), INPUT_DATA_SCHEMA_VALIDATION_FAILED(-4), + EXTENDED_INPUT_DATA_SCHEMA_VALIDATION_FAILED(-5), RECORD_TOO_LARGE_FAILED(-6), CONCURRENT_BATCH_PUSH(-7), + DATASET_CHANGED(-8), INVALID_INPUT_FILE(-9), ZSTD_DICTIONARY_CREATION_FAILED(-10), DVC_INGESTION_ERROR_DISK_FULL(-11), + DVC_INGESTION_ERROR_MEMORY_LIMIT_REACHED(-12), DVC_INGESTION_ERROR_TOO_MANY_DEAD_INSTANCES(-13), + DVC_INGESTION_ERROR_OTHER(-14); + + private final int value; + + PushJobCheckpoints(int value) { + this.value = value; + } + + public int getValue() { + return value; + } + + /** + * Get PushJobCheckpoints from integer ordinal value + */ + private static final Map idMapping = new HashMap<>(); + static { + Arrays.stream(values()).forEach(s -> idMapping.put(s.value, s)); + } + + public static PushJobCheckpoints fromInt(int v) { + return idMapping.get(v); + } +} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/acl/handler/AbstractStoreAclHandler.java b/internal/venice-common/src/main/java/com/linkedin/venice/acl/handler/AbstractStoreAclHandler.java new file mode 100644 index 00000000000..2d1c45be99d --- /dev/null +++ b/internal/venice-common/src/main/java/com/linkedin/venice/acl/handler/AbstractStoreAclHandler.java @@ -0,0 +1,204 @@ +package com.linkedin.venice.acl.handler; + +import static com.linkedin.venice.listener.ServerHandlerUtils.extractClientCert; + +import com.linkedin.venice.acl.AclCreationDeletionListener; +import com.linkedin.venice.acl.AclException; +import com.linkedin.venice.acl.DynamicAccessController; +import com.linkedin.venice.authorization.IdentityParser; +import com.linkedin.venice.common.VeniceSystemStoreUtils; +import com.linkedin.venice.meta.ReadOnlyStoreRepository; +import com.linkedin.venice.meta.Store; +import com.linkedin.venice.utils.NettyUtils; +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.handler.codec.http.HttpRequest; +import io.netty.handler.codec.http.HttpResponseStatus; +import io.netty.util.ReferenceCountUtil; +import java.net.URI; +import java.security.cert.X509Certificate; +import java.util.stream.Collectors; +import javax.net.ssl.SSLPeerUnverifiedException; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + + +/** + * Store-level access control handler, which is being used by both Router and Server. + */ +@ChannelHandler.Sharable +public abstract class AbstractStoreAclHandler extends SimpleChannelInboundHandler { + private static final Logger LOGGER = LogManager.getLogger(AbstractStoreAclHandler.class); + + private final IdentityParser identityParser; + private final ReadOnlyStoreRepository metadataRepository; + private final DynamicAccessController accessController; + + public AbstractStoreAclHandler( + IdentityParser identityParser, + DynamicAccessController accessController, + ReadOnlyStoreRepository metadataRepository) { + this.identityParser = identityParser; + this.metadataRepository = metadataRepository; + this.accessController = accessController + .init(metadataRepository.getAllStores().stream().map(Store::getName).collect(Collectors.toList())); + this.metadataRepository.registerStoreDataChangedListener(new AclCreationDeletionListener(accessController)); + } + + /** + * Verify if client has permission to access. + * + * @param ctx + * @param req + * @throws SSLPeerUnverifiedException + */ + @Override + public void channelRead0(ChannelHandlerContext ctx, HttpRequest req) throws SSLPeerUnverifiedException { + if (isAccessAlreadyApproved(ctx)) { + ReferenceCountUtil.retain(req); + ctx.fireChannelRead(req); + return; + } + + String uri = req.uri(); + String method = req.method().name(); + + // Parse resource type and store name + String[] requestParts = URI.create(uri).getPath().split("/"); + REQUEST_TYPE requestType = validateRequest(requestParts); + + if (requestType == null) { + String errorMessage = "Invalid request uri: " + uri; + NettyUtils.setupResponseAndFlush(HttpResponseStatus.BAD_REQUEST, errorMessage.getBytes(), false, ctx); + return; + } + + if (!needsAclValidation(requestType)) { + ReferenceCountUtil.retain(req); + ctx.fireChannelRead(req); + return; + } + + String storeName = extractStoreName(requestType, requestParts); + + // When there is no store present in the metadata repository, pass the ACL check and let the next handler handle the + // case of deleted or migrated store + if (metadataRepository.getStore(storeName) == null) { + ReferenceCountUtil.retain(req); + ctx.fireChannelRead(req); + return; + } + + X509Certificate clientCert = extractClientCert(ctx); + + AccessResult accessResult = checkAccess(uri, clientCert, storeName, method); + switch (accessResult) { + case GRANTED: + ReferenceCountUtil.retain(req); + ctx.fireChannelRead(req); + break; + case UNAUTHORIZED: + NettyUtils + .setupResponseAndFlush(HttpResponseStatus.UNAUTHORIZED, accessResult.getMessage().getBytes(), false, ctx); + break; + case FORBIDDEN: + case ERROR_FORBIDDEN: + NettyUtils + .setupResponseAndFlush(HttpResponseStatus.FORBIDDEN, accessResult.getMessage().getBytes(), false, ctx); + break; + } + } + + protected boolean isAccessAlreadyApproved(ChannelHandlerContext ctx) { + return false; + } + + protected abstract boolean needsAclValidation(REQUEST_TYPE requestType); + + protected abstract String extractStoreName(REQUEST_TYPE requestType, String[] requestParts); + + /** + * Validate the request and return the request type. If the request is invalid, return {@code null} + * + * @param requestParts the parts of the request URI + * @return the request type; null if the request is invalid + */ + protected abstract REQUEST_TYPE validateRequest(String[] requestParts); + + protected AccessResult checkAccess(String uri, X509Certificate clientCert, String storeName, String method) { + if (VeniceSystemStoreUtils.isSystemStore(storeName)) { + return AccessResult.GRANTED; + } + + String client = identityParser.parseIdentityFromCert(clientCert); + try { + /** + * TODO: Consider making this the first check, so that we optimize for the hot path. If rejected, then we + * could check whether the request is for a system store, METADATA, etc. + */ + if (accessController.hasAccess(clientCert, storeName, method)) { + return AccessResult.GRANTED; + } + + // Fact: + // Request gets rejected. + // Possible Reasons: + // A. ACL not found. OR, + // B. ACL exists but caller does not have permission. + String errLine = String.format("%s requested %s %s", client, method, uri); + + if (!accessController.isFailOpen() && !accessController.hasAcl(storeName)) { // short circuit, order matters + // Case A + // Conditions: + // 0. (outside) Store exists and is being access controlled. AND, + // 1. (left) The following policy is applied: if ACL not found, reject the request. AND, + // 2. (right) ACL not found. + // Result: + // Request is rejected by DynamicAccessController#hasAccess() + // Root cause: + // Requested resource exists but does not have ACL. + // Action: + // return 401 Unauthorized + LOGGER.warn("Requested store does not have ACL: {}", errLine); + LOGGER.debug( + "Existing stores: {}", + () -> metadataRepository.getAllStores().stream().map(Store::getName).sorted().collect(Collectors.toList())); + LOGGER.debug( + "Access-controlled stores: {}", + () -> accessController.getAccessControlledResources().stream().sorted().collect(Collectors.toList())); + return AccessResult.UNAUTHORIZED; + } else { + // Case B + // Conditions: + // 1. Fail closed, and ACL found. OR, + // 2. Fail open, and ACL found. OR, + // 3. Fail open, and ACL not found. + // Analyses: + // (1) ACL exists, therefore result is determined by ACL. + // Since the request has been rejected, it must be due to lack of permission. + // (2) ACL exists, therefore result is determined by ACL. + // Since the request has been rejected, it must be due to lack of permission. + // (3) In such case, request would NOT be rejected in the first place, + // according to the definition of hasAccess() in DynamicAccessController interface. + // Contradiction to the fact, therefore this case is impossible. + // Root cause: + // Caller does not have permission to access the resource. + // Action: + // return 403 Forbidden + LOGGER.debug("Unauthorized access rejected: {}", errLine); + return AccessResult.FORBIDDEN; + } + } catch (AclException e) { + String errLine = String.format("%s requested %s %s", client, method, uri); + + if (accessController.isFailOpen()) { + LOGGER.warn("Exception occurred! Access granted: {} {}", errLine, e); + return AccessResult.GRANTED; + } else { + LOGGER.warn("Exception occurred! Access rejected: {} {}", errLine, e); + return AccessResult.ERROR_FORBIDDEN; + } + } + } +} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/acl/handler/AccessResult.java b/internal/venice-common/src/main/java/com/linkedin/venice/acl/handler/AccessResult.java new file mode 100644 index 00000000000..6283b872791 --- /dev/null +++ b/internal/venice-common/src/main/java/com/linkedin/venice/acl/handler/AccessResult.java @@ -0,0 +1,28 @@ +package com.linkedin.venice.acl.handler; + +public enum AccessResult { + GRANTED(), + FORBIDDEN( + "Access denied!\n" + + "If you are the store owner, add this application (or your own username for Venice shell client) to the store ACL.\n" + + "Otherwise, ask the store owner for read permission." + ), + UNAUTHORIZED( + "ACL not found!\n" + "Either it has not been created, or can not be loaded.\n" + + "Please create the ACL, or report the error if you know for sure that ACL exists for the store" + ), ERROR_FORBIDDEN("Internal error occurred while checking ACL.\n" + "Please report this error to the Venice team."); + + private final String message; + + AccessResult() { + this.message = null; + } + + AccessResult(String message) { + this.message = message; + } + + public String getMessage() { + return message; + } +} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/acl/handler/StoreAclHandler.java b/internal/venice-common/src/main/java/com/linkedin/venice/acl/handler/StoreAclHandler.java deleted file mode 100644 index df4781f5039..00000000000 --- a/internal/venice-common/src/main/java/com/linkedin/venice/acl/handler/StoreAclHandler.java +++ /dev/null @@ -1,290 +0,0 @@ -package com.linkedin.venice.acl.handler; - -import static com.linkedin.venice.grpc.GrpcUtils.extractGrpcClientCert; -import static com.linkedin.venice.grpc.GrpcUtils.httpResponseStatusToGrpcStatus; -import static com.linkedin.venice.listener.ServerHandlerUtils.extractClientCert; - -import com.google.common.annotations.VisibleForTesting; -import com.linkedin.venice.acl.AclCreationDeletionListener; -import com.linkedin.venice.acl.AclException; -import com.linkedin.venice.acl.DynamicAccessController; -import com.linkedin.venice.common.VeniceSystemStoreUtils; -import com.linkedin.venice.exceptions.VeniceException; -import com.linkedin.venice.exceptions.VeniceNoStoreException; -import com.linkedin.venice.meta.QueryAction; -import com.linkedin.venice.meta.ReadOnlyStoreRepository; -import com.linkedin.venice.meta.Store; -import com.linkedin.venice.protocols.VeniceClientRequest; -import com.linkedin.venice.utils.NettyUtils; -import io.grpc.ForwardingServerCallListener; -import io.grpc.Grpc; -import io.grpc.Metadata; -import io.grpc.ServerCall; -import io.grpc.ServerCallHandler; -import io.grpc.ServerInterceptor; -import io.grpc.Status; -import io.netty.channel.ChannelHandler; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.SimpleChannelInboundHandler; -import io.netty.handler.codec.http.HttpRequest; -import io.netty.handler.codec.http.HttpResponseStatus; -import io.netty.util.ReferenceCountUtil; -import java.net.URI; -import java.security.cert.X509Certificate; -import java.util.Arrays; -import java.util.HashSet; -import java.util.Objects; -import java.util.Set; -import java.util.function.BiConsumer; -import java.util.stream.Collectors; -import javax.net.ssl.SSLPeerUnverifiedException; -import org.apache.commons.lang.StringUtils; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - - -/** - * Store-level access control handler, which is being used by both Router and Server. - */ -@ChannelHandler.Sharable -public class StoreAclHandler extends SimpleChannelInboundHandler implements ServerInterceptor { - private static final Logger LOGGER = LogManager.getLogger(StoreAclHandler.class); - - /** - * Skip ACL for requests to /metadata, /admin, /current_version, /health and /topic_partition_ingestion_context - * as there's no sensitive information in the response. - */ - private static final Set QUERIES_TO_SKIP_ACL = new HashSet<>( - Arrays.asList( - QueryAction.METADATA, - QueryAction.ADMIN, - QueryAction.HEALTH, - QueryAction.CURRENT_VERSION, - QueryAction.TOPIC_PARTITION_INGESTION_CONTEXT)); - - private final ReadOnlyStoreRepository metadataRepository; - private final DynamicAccessController accessController; - - public StoreAclHandler(DynamicAccessController accessController, ReadOnlyStoreRepository metadataRepository) { - this.metadataRepository = metadataRepository; - this.accessController = accessController - .init(metadataRepository.getAllStores().stream().map(Store::getName).collect(Collectors.toList())); - this.metadataRepository.registerStoreDataChangedListener(new AclCreationDeletionListener(accessController)); - } - - /** - * Verify if client has permission to access. - * - * @param ctx - * @param req - * @throws SSLPeerUnverifiedException - */ - @Override - public void channelRead0(ChannelHandlerContext ctx, HttpRequest req) throws SSLPeerUnverifiedException { - String uri = req.uri(); - String method = req.method().name(); - String client = ctx.channel().remoteAddress().toString(); // ip and port - BiConsumer errorHandler = - (status, errorMessage) -> NettyUtils.setupResponseAndFlush(status, errorMessage.getBytes(), false, ctx); - - // Parse resource type and store name - String[] requestParts = URI.create(uri).getPath().split("/"); - - if (isInvalidRequest(requestParts)) { - errorHandler.accept(HttpResponseStatus.BAD_REQUEST, "Invalid request uri: " + uri); - return; - } - - /* - * Skip request uri validations for store name and certificates due to special actions - * TODO: Identify validations for each query actions and have a flow to perform validations and actions based on - * query actions - */ - QueryAction queryAction = QueryAction.valueOf(requestParts[1].toUpperCase()); - if (QUERIES_TO_SKIP_ACL.contains(queryAction)) { - ReferenceCountUtil.retain(req); - ctx.fireChannelRead(req); - return; - } - - X509Certificate clientCert = extractClientCert(ctx); - String resourceName = requestParts[2]; - String storeName = extractStoreName(resourceName, queryAction); - - try { - // Check ACL in case of non system store as system store contain public information - if (VeniceSystemStoreUtils.isSystemStore(storeName) - || hasAccess(client, uri, clientCert, storeName, method, errorHandler)) { - ReferenceCountUtil.retain(req); - ctx.fireChannelRead(req); - } - } catch (VeniceNoStoreException noStoreException) { - LOGGER.debug("Requested store does not exist: {} requested {} {}", client, method, req.uri()); - errorHandler.accept(HttpResponseStatus.BAD_REQUEST, "Invalid Venice store name: " + storeName); - } - } - - @Override - public ServerCall.Listener interceptCall( - ServerCall call, - Metadata headers, - ServerCallHandler next) { - return new ForwardingServerCallListener.SimpleForwardingServerCallListener(next.startCall(call, headers)) { - @Override - public void onMessage(ReqT message) { - VeniceClientRequest request = (VeniceClientRequest) message; - // For now, GRPC only supports STORAGE query - String storeName = extractStoreName(request.getResourceName(), QueryAction.STORAGE); - String method = request.getMethod(); - - BiConsumer grpcCloseConsumer = call::close; - BiConsumer errorHandler = ((httpResponseStatus, s) -> grpcCloseConsumer - .accept(httpResponseStatusToGrpcStatus(httpResponseStatus, s), headers)); - - if (StringUtils.isEmpty(storeName) || StringUtils.isEmpty(method)) { - LOGGER.error("Invalid store name {} or method {}", storeName, method); - grpcCloseConsumer.accept(Status.INVALID_ARGUMENT.withDescription("Invalid request"), headers); - return; - } - - try { - X509Certificate clientCert = extractGrpcClientCert(call); - String client = Objects.requireNonNull(call.getAttributes().get(Grpc.TRANSPORT_ATTR_REMOTE_ADDR)).toString(); - - if (VeniceSystemStoreUtils.isSystemStore(storeName) - || hasAccess(client, call.getAuthority(), clientCert, storeName, method, errorHandler)) { - LOGGER.info("Requested principal has access to resource. Processing request"); - super.onMessage(message); - } - } catch (SSLPeerUnverifiedException e) { - LOGGER.error("Cannot verify the certificate.", e); - grpcCloseConsumer.accept(Status.UNAUTHENTICATED.withDescription("Invalid certificate"), headers); - } catch (VeniceException e) { - LOGGER.error("Cannot process request successfully due to", e); - grpcCloseConsumer.accept(Status.INTERNAL.withDescription(e.getMessage()), headers); - } - } - }; - } - - /** - * Extract the store name from the incoming resource name. - */ - protected String extractStoreName(String resourceName, QueryAction queryAction) { - return resourceName; - } - - @VisibleForTesting - boolean isInvalidRequest(String[] requestParts) { - int partsLength = requestParts.length; - boolean invalidRequest = false; - - // Only for HEALTH queries, parts length can be 2 - if (partsLength == 2) { - invalidRequest = !requestParts[1].equalsIgnoreCase(QueryAction.HEALTH.name()); - } else if (partsLength < 3) { // invalid request if parts length < 3 except health queries - invalidRequest = true; - } else { // throw exception to retain current behavior for invalid query actions - try { - QueryAction.valueOf(requestParts[1].toUpperCase()); - } catch (IllegalArgumentException exception) { - throw new VeniceException("Unknown query action: " + requestParts[1]); - } - } - - return invalidRequest; - } - - @VisibleForTesting - boolean hasAccess( - String client, - String uri, - X509Certificate clientCert, - String storeName, - String method, - BiConsumer errorHandler) { - boolean allowRequest = false; - try { - /** - * TODO: Consider making this the first check, so that we optimize for the hot path. If rejected, then we - * could check whether the request is for a system store, METADATA, etc. - */ - allowRequest = accessController.hasAccess(clientCert, storeName, method); - if (!allowRequest) { - // Fact: - // Request gets rejected. - // Possible Reasons: - // A. ACL not found. OR, - // B. ACL exists but caller does not have permission. - - String errLine = String.format("%s requested %s %s", client, method, uri); - - if (!accessController.isFailOpen() && !accessController.hasAcl(storeName)) { // short circuit, order matters - // Case A - // Conditions: - // 0. (outside) Store exists and is being access controlled. AND, - // 1. (left) The following policy is applied: if ACL not found, reject the request. AND, - // 2. (right) ACL not found. - // Result: - // Request is rejected by DynamicAccessController#hasAccess() - // Root cause: - // Requested resource exists but does not have ACL. - // Action: - // return 401 Unauthorized - LOGGER.warn("Requested store does not have ACL: {}", errLine); - LOGGER.debug( - "Existing stores: {}", - () -> metadataRepository.getAllStores() - .stream() - .map(Store::getName) - .sorted() - .collect(Collectors.toList())); - LOGGER.debug( - "Access-controlled stores: {}", - () -> accessController.getAccessControlledResources().stream().sorted().collect(Collectors.toList())); - errorHandler.accept( - HttpResponseStatus.UNAUTHORIZED, - "ACL not found!\n" + "Either it has not been created, or can not be loaded.\n" - + "Please create the ACL, or report the error if you know for sure that ACL exists for this store: " - + storeName); - } else { - // Case B - // Conditions: - // 1. Fail closed, and ACL found. OR, - // 2. Fail open, and ACL found. OR, - // 3. Fail open, and ACL not found. - // Analyses: - // (1) ACL exists, therefore result is determined by ACL. - // Since the request has been rejected, it must be due to lack of permission. - // (2) ACL exists, therefore result is determined by ACL. - // Since the request has been rejected, it must be due to lack of permission. - // (3) In such case, request would NOT be rejected in the first place, - // according to the definition of hasAccess() in DynamicAccessController interface. - // Contradiction to the fact, therefore this case is impossible. - // Root cause: - // Caller does not have permission to access the resource. - // Action: - // return 403 Forbidden - LOGGER.debug("Unauthorized access rejected: {}", errLine); - errorHandler.accept( - HttpResponseStatus.FORBIDDEN, - "Access denied!\n" - + "If you are the store owner, add this application (or your own username for Venice shell client) to the store ACL.\n" - + "Otherwise, ask the store owner for read permission."); - } - } - } catch (AclException e) { - String errLine = String.format("%s requested %s %s", client, method, uri); - - if (accessController.isFailOpen()) { - LOGGER.warn("Exception occurred! Access granted: {} {}", errLine, e); - allowRequest = true; - } else { - LOGGER.warn("Exception occurred! Access rejected: {} {}", errLine, e); - errorHandler.accept(HttpResponseStatus.FORBIDDEN, "Access denied!"); - } - } - - return allowRequest; - } -} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/ControllerClient.java b/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/ControllerClient.java index e8853edc39d..5d812b44639 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/ControllerClient.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/ControllerClient.java @@ -40,7 +40,6 @@ import static com.linkedin.venice.controllerapi.ControllerApiConstants.PERSONA_QUOTA; import static com.linkedin.venice.controllerapi.ControllerApiConstants.PERSONA_STORES; import static com.linkedin.venice.controllerapi.ControllerApiConstants.PUSH_IN_SORTED_ORDER; -import static com.linkedin.venice.controllerapi.ControllerApiConstants.PUSH_JOB_DETAILS; import static com.linkedin.venice.controllerapi.ControllerApiConstants.PUSH_JOB_ID; import static com.linkedin.venice.controllerapi.ControllerApiConstants.PUSH_STRATEGY; import static com.linkedin.venice.controllerapi.ControllerApiConstants.PUSH_TYPE; @@ -822,13 +821,6 @@ public JobStatusQueryResponse queryDetailedJobStatus(String kafkaTopic, String r return request(ControllerRoute.JOB, params, JobStatusQueryResponse.class, QUERY_JOB_STATUS_TIMEOUT, 1, null); } - // TODO remove passing PushJobDetails as JSON string once all VPJ plugins are updated. - public ControllerResponse sendPushJobDetails(String storeName, int version, String pushJobDetailsString) { - QueryParams params = - newParams().add(NAME, storeName).add(VERSION, version).add(PUSH_JOB_DETAILS, pushJobDetailsString); - return request(ControllerRoute.SEND_PUSH_JOB_DETAILS, params, ControllerResponse.class); - } - public ControllerResponse sendPushJobDetails(String storeName, int version, byte[] pushJobDetails) { QueryParams params = newParams().add(NAME, storeName).add(VERSION, version); return request(ControllerRoute.SEND_PUSH_JOB_DETAILS, params, ControllerResponse.class, pushJobDetails); diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/grpc/GrpcUtils.java b/internal/venice-common/src/main/java/com/linkedin/venice/grpc/GrpcUtils.java index 44020f9eedc..f2fcb1c0e7a 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/grpc/GrpcUtils.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/grpc/GrpcUtils.java @@ -1,5 +1,6 @@ package com.linkedin.venice.grpc; +import com.linkedin.venice.acl.handler.AccessResult; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.security.SSLConfig; import com.linkedin.venice.security.SSLFactory; @@ -7,7 +8,6 @@ import io.grpc.Grpc; import io.grpc.ServerCall; import io.grpc.Status; -import io.netty.handler.codec.http.HttpResponseStatus; import java.io.IOException; import java.io.InputStream; import java.nio.file.Files; @@ -50,12 +50,17 @@ public static TrustManager[] getTrustManagers(SSLFactory sslFactory) return trustManagerFactory.getTrustManagers(); } - public static Status httpResponseStatusToGrpcStatus(HttpResponseStatus status, String errorMessage) { - if (status.equals(HttpResponseStatus.FORBIDDEN) || status.equals(HttpResponseStatus.UNAUTHORIZED)) { - return Status.PERMISSION_DENIED.withDescription(errorMessage); + public static Status accessResultToGrpcStatus(AccessResult accessResult) { + switch (accessResult) { + case GRANTED: + return Status.OK; + case FORBIDDEN: + case UNAUTHORIZED: + case ERROR_FORBIDDEN: + return Status.PERMISSION_DENIED.withDescription(accessResult.getMessage()); + default: + return Status.UNKNOWN.withDescription(accessResult.getMessage()); } - - return Status.UNKNOWN.withDescription(errorMessage); } public static X509Certificate extractGrpcClientCert(ServerCall call) throws SSLPeerUnverifiedException { diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/protocol/enums/ControlMessageType.java b/internal/venice-common/src/main/java/com/linkedin/venice/kafka/protocol/enums/ControlMessageType.java index a4f83e000f3..9170b59b3a8 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/protocol/enums/ControlMessageType.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/kafka/protocol/enums/ControlMessageType.java @@ -13,6 +13,7 @@ import com.linkedin.venice.kafka.protocol.VersionSwap; import com.linkedin.venice.utils.EnumUtils; import com.linkedin.venice.utils.VeniceEnumValue; +import java.util.List; /** @@ -28,12 +29,13 @@ public enum ControlMessageType implements VeniceEnumValue { /** The value is the byte used on the wire format */ private final int value; - private static final ControlMessageType[] TYPES_ARRAY = EnumUtils.getEnumValuesArray(ControlMessageType.class); + private static final List TYPES = EnumUtils.getEnumValuesList(ControlMessageType.class); ControlMessageType(int value) { this.value = value; } + @Override public int getValue() { return value; } @@ -76,11 +78,7 @@ public Object getNewInstance() { } public static ControlMessageType valueOf(int value) { - try { - return TYPES_ARRAY[value]; - } catch (IndexOutOfBoundsException e) { - throw new VeniceMessageException("Invalid control message type: " + value); - } + return EnumUtils.valueOf(TYPES, value, ControlMessageType.class, VeniceMessageException::new); } public static ControlMessageType valueOf(ControlMessage controlMessage) { diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/protocol/enums/MessageType.java b/internal/venice-common/src/main/java/com/linkedin/venice/kafka/protocol/enums/MessageType.java index 06c096d43a2..68e99e53468 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/kafka/protocol/enums/MessageType.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/kafka/protocol/enums/MessageType.java @@ -9,6 +9,7 @@ import com.linkedin.venice.kafka.protocol.Update; import com.linkedin.venice.utils.EnumUtils; import com.linkedin.venice.utils.VeniceEnumValue; +import java.util.List; /** @@ -22,7 +23,7 @@ public enum MessageType implements VeniceEnumValue { PUT(0, Constants.PUT_KEY_HEADER_BYTE), DELETE(1, Constants.PUT_KEY_HEADER_BYTE), CONTROL_MESSAGE(2, Constants.CONTROL_MESSAGE_KEY_HEADER_BYTE), UPDATE(3, Constants.UPDATE_KEY_HEADER_BYTE); - private static final MessageType[] TYPES_ARRAY = EnumUtils.getEnumValuesArray(MessageType.class); + private static final List TYPES = EnumUtils.getEnumValuesList(MessageType.class); private final int value; private final byte keyHeaderByte; @@ -36,6 +37,7 @@ public enum MessageType implements VeniceEnumValue { * @return This is the value used in {@link com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope#messageType} * to distinguish message types. */ + @Override public int getValue() { return value; } @@ -73,11 +75,7 @@ public Object getNewInstance() { } public static MessageType valueOf(int value) { - try { - return TYPES_ARRAY[value]; - } catch (IndexOutOfBoundsException e) { - throw new VeniceMessageException("Invalid message type: " + value); - } + return EnumUtils.valueOf(TYPES, value, MessageType.class, VeniceMessageException::new); } public static MessageType valueOf(KafkaMessageEnvelope kafkaMessageEnvelope) { diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/status/PushJobDetailsStatus.java b/internal/venice-common/src/main/java/com/linkedin/venice/status/PushJobDetailsStatus.java index c5ba1a618d2..736f9e58cbe 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/status/PushJobDetailsStatus.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/status/PushJobDetailsStatus.java @@ -1,7 +1,9 @@ package com.linkedin.venice.status; import java.util.Arrays; +import java.util.HashMap; import java.util.HashSet; +import java.util.Map; import java.util.Set; @@ -11,6 +13,8 @@ public enum PushJobDetailsStatus { private static final Set TERMINAL_STATUSES = new HashSet<>(Arrays.asList(COMPLETED.getValue(), ERROR.getValue(), KILLED.getValue())); + + private static final Set TERMINAL_FAILED_STATUSES = new HashSet<>(Arrays.asList(ERROR, KILLED)); private final int value; PushJobDetailsStatus(int value) { @@ -24,4 +28,28 @@ public int getValue() { public static boolean isTerminal(int status) { return TERMINAL_STATUSES.contains(status); } + + public static boolean isSucceeded(PushJobDetailsStatus status) { + return status == COMPLETED; + } + + public static boolean isFailed(PushJobDetailsStatus status) { + return TERMINAL_FAILED_STATUSES.contains(status); + } + + /** + * Get PushJobDetailsStatus from integer ordinal value. + */ + private static final Map idMapping = new HashMap<>(); + static { + Arrays.stream(values()).forEach(s -> idMapping.put(s.value, s)); + } + + public static PushJobDetailsStatus fromInt(int v) { + PushJobDetailsStatus status = idMapping.get(v); + if (status == null) { + return PushJobDetailsStatus.UNKNOWN; + } + return status; + } } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/system/store/MetaStoreDataType.java b/internal/venice-common/src/main/java/com/linkedin/venice/system/store/MetaStoreDataType.java index ad135031308..ad4727acba0 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/system/store/MetaStoreDataType.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/system/store/MetaStoreDataType.java @@ -32,7 +32,7 @@ public enum MetaStoreDataType implements VeniceEnumValue { VALUE_SCHEMAS_WRITTEN_PER_STORE_VERSION(6, Arrays.asList(KEY_STRING_STORE_NAME, KEY_STRING_VERSION_NUMBER)), HEARTBEAT(7, Collections.singletonList(KEY_STRING_STORE_NAME)); - private static final MetaStoreDataType[] TYPES_ARRAY = EnumUtils.getEnumValuesArray(MetaStoreDataType.class); + private static final List TYPES = EnumUtils.getEnumValuesList(MetaStoreDataType.class); private final int value; private final List requiredKeys; @@ -42,16 +42,13 @@ public enum MetaStoreDataType implements VeniceEnumValue { this.requiredKeys = requiredKeys; } + @Override public int getValue() { return value; } public static MetaStoreDataType valueOf(int value) { - try { - return TYPES_ARRAY[value]; - } catch (IndexOutOfBoundsException e) { - throw new VeniceException("Invalid compression strategy: " + value); - } + return EnumUtils.valueOf(TYPES, value, MetaStoreDataType.class); } public StoreMetaKey getStoreMetaKey(Map params) { diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/writer/LeaderCompleteState.java b/internal/venice-common/src/main/java/com/linkedin/venice/writer/LeaderCompleteState.java index 482b789a3e4..3797c3b8a4b 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/writer/LeaderCompleteState.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/writer/LeaderCompleteState.java @@ -1,8 +1,8 @@ package com.linkedin.venice.writer; -import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.utils.EnumUtils; import com.linkedin.venice.utils.VeniceEnumValue; +import java.util.List; /** @@ -19,7 +19,7 @@ public enum LeaderCompleteState implements VeniceEnumValue { LEADER_COMPLETED(1); private final int value; - private static final LeaderCompleteState[] TYPES_ARRAY = EnumUtils.getEnumValuesArray(LeaderCompleteState.class); + private static final List TYPES = EnumUtils.getEnumValuesList(LeaderCompleteState.class); LeaderCompleteState(int value) { this.value = value; @@ -34,11 +34,7 @@ public static LeaderCompleteState getLeaderCompleteState(boolean isLeaderComplet } public static LeaderCompleteState valueOf(int value) { - try { - return TYPES_ARRAY[value]; - } catch (IndexOutOfBoundsException e) { - throw new VeniceException("Invalid LeaderCompleteState: " + value); - } + return EnumUtils.valueOf(TYPES, value, LeaderCompleteState.class); } @Override diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/acl/StoreAclHandlerTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/acl/handler/AbstractStoreAclHandlerTest.java similarity index 54% rename from internal/venice-common/src/test/java/com/linkedin/venice/acl/StoreAclHandlerTest.java rename to internal/venice-common/src/test/java/com/linkedin/venice/acl/handler/AbstractStoreAclHandlerTest.java index 4bd7989a3ba..5fff574240d 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/acl/StoreAclHandlerTest.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/acl/handler/AbstractStoreAclHandlerTest.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.acl; +package com.linkedin.venice.acl.handler; import static org.mockito.Mockito.any; import static org.mockito.Mockito.argThat; @@ -9,9 +9,9 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import com.linkedin.venice.acl.handler.StoreAclHandler; +import com.linkedin.venice.acl.DynamicAccessController; +import com.linkedin.venice.authorization.IdentityParser; import com.linkedin.venice.common.VeniceSystemStoreUtils; -import com.linkedin.venice.exceptions.VeniceNoStoreException; import com.linkedin.venice.helix.HelixReadOnlyStoreRepository; import com.linkedin.venice.meta.Store; import io.netty.channel.Channel; @@ -32,43 +32,37 @@ import org.testng.annotations.Test; -public class StoreAclHandlerTest { +public class AbstractStoreAclHandlerTest { + private IdentityParser identityParser; private DynamicAccessController accessController; private HelixReadOnlyStoreRepository metadataRepo; private ChannelHandlerContext ctx; - private Channel channel; private HttpRequest req; - private StoreAclHandler aclHandler; private Store store; + private boolean[] needsAcl = { true }; private boolean[] hasAccess = { false }; private boolean[] hasAcl = { false }; private boolean[] hasStore = { false }; private boolean[] isSystemStore = { false }; private boolean[] isFailOpen = { false }; - private boolean[] isMetadata = { false }; - - private boolean[] isCurrentVersion = { false }; - - private boolean[] isHealthCheck = { false }; private boolean[] isBadUri = { false }; private String storeName; private void resetAllConditions() { + needsAcl[0] = true; hasAccess[0] = false; hasAcl[0] = false; hasStore[0] = false; isSystemStore[0] = false; isFailOpen[0] = false; - isMetadata[0] = false; - isHealthCheck[0] = false; isBadUri[0] = false; - isCurrentVersion[0] = false; } @BeforeMethod public void setUp() throws Exception { storeName = "testStore"; + identityParser = mock(IdentityParser.class); accessController = mock(DynamicAccessController.class); ctx = mock(ChannelHandlerContext.class); req = mock(HttpRequest.class); @@ -89,7 +83,7 @@ public void setUp() throws Exception { when(sslSession.getPeerCertificates()).thenReturn(new Certificate[] { cert }); // Host - channel = mock(Channel.class); + Channel channel = mock(Channel.class); when(ctx.channel()).thenReturn(channel); SocketAddress address = mock(SocketAddress.class); when(channel.remoteAddress()).thenReturn(address); @@ -97,67 +91,82 @@ public void setUp() throws Exception { when(req.method()).thenReturn(HttpMethod.GET); } + @Test + public void noAclNeeded() throws Exception { + needsAcl[0] = false; + enumerate(hasAccess, hasAcl, hasStore, isSystemStore, isFailOpen); + + verify(ctx, never()).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.FORBIDDEN))); + verify(ctx, never()).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.UNAUTHORIZED))); + verify(ctx, never()).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.BAD_REQUEST))); + + // No access control needed. There are 32 possible combinations of the 5 variables + verify(ctx, times(32)).fireChannelRead(req); + } + @Test public void accessGranted() throws Exception { hasAccess[0] = true; - enumerate(hasAcl, hasStore, isSystemStore, isFailOpen, isMetadata, isHealthCheck); + enumerate(needsAcl, hasAcl, hasStore, isSystemStore, isFailOpen); verify(ctx, never()).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.FORBIDDEN))); verify(ctx, never()).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.UNAUTHORIZED))); - - // Store doesn't exist 8 times but shouldn't throw exception because it could be a migrated store verify(ctx, never()).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.BAD_REQUEST))); - // No access control (METADATA/HEALTH/ADMIN or system store) => 52 times, access control => 4 times, store DNE => 8 - // times - verify(ctx, times(64)).fireChannelRead(req); + // !needsAcl = 16 times + // needsAcl && !hasStore = 8 times + // needsAcl && hasStore && isSystemStore = 4 times + // needsAcl && hasStore && !isSystemStore = 4 times + verify(ctx, times(32)).fireChannelRead(req); } @Test public void accessDenied() throws Exception { hasAccess[0] = false; - enumerate(hasAcl, hasStore, isSystemStore, isFailOpen, isMetadata, isHealthCheck); + enumerate(needsAcl, hasAcl, hasStore, isSystemStore, isFailOpen); - // Store doesn't exist 8 times but shouldn't throw exception because it could be a migrated store verify(ctx, never()).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.BAD_REQUEST))); - // No access control ((METADATA/HEALTH/ADMIN or system store) + 4 times (!hasStore && isSystemStore) => 56 times - verify(ctx, times(56)).fireChannelRead(req); + // No access control needed 16 times + // needsAcl && !hasStore = 8 times + // needsAcl && hasStore && isSystemStore = 4 times + verify(ctx, times(28)).fireChannelRead(req); - // 2 of the 8 rejects is due to internal error - verify(ctx, times(2)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.UNAUTHORIZED))); + // UNAUTHORIZED when needsAcl && hasStore && !hasAccess && !hasAcl && !isFailOpen && !isSystemStore + verify(ctx, times(1)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.UNAUTHORIZED))); - // The other 6 are regular rejects - verify(ctx, times(6)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.FORBIDDEN))); + // FORBIDDEN when needsAcl && hasStore && !hasAccess && (hasAcl || isFailOpen) && !isSystemStore + verify(ctx, times(3)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.FORBIDDEN))); } @Test public void storeExists() throws Exception { hasStore[0] = true; - enumerate(hasAccess, hasAcl, isFailOpen, isSystemStore, isMetadata, isHealthCheck); + enumerate(needsAcl, hasAccess, hasAcl, isFailOpen, isSystemStore); verify(ctx, never()).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.BAD_REQUEST))); - // No access control (METADATA/HEALTH/ADMIN or system store) => 56 times, access control => 4 times granted - verify(ctx, times(60)).fireChannelRead(req); + // !needsAcl = 16 times + // needsAcl && hasStore && isSystemStore = 8 times + // needsAcl && hasStore && !isSystemStore && hasAccess = 4 times + verify(ctx, times(28)).fireChannelRead(req); - // 1 of the 4 rejects is due to internal error + // needsAcl && hasStore && !isSystemStore && !hasAccess && !hasAcl && !isFailOpen = 1 time verify(ctx, times(1)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.UNAUTHORIZED))); - // The other 3 are regular rejects + // needsAcl && hasStore && !isSystemStore && !hasAccess && (hasAcl || isFailOpen) = 3 times verify(ctx, times(3)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.FORBIDDEN))); } @Test public void storeMissing() throws Exception { hasStore[0] = false; - enumerate(hasAccess, hasAcl, isFailOpen, isSystemStore, isMetadata, isHealthCheck); + enumerate(needsAcl, hasAccess, hasAcl, isFailOpen, isSystemStore); + + // !needsAcl = 16 times + // needsAcl && !hasStore = 16 times + verify(ctx, times(32)).fireChannelRead(req); - // No access control (METADATA/HEALTH/ADMIN or system store) => 52 times + 8 times granted - verify(ctx, times(60)).fireChannelRead(req); - // Although we don't check for store existence due to potentially migrated store, we will still reject based on ACL - verify(ctx, times(3)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.FORBIDDEN))); - verify(ctx, times(1)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.UNAUTHORIZED))); verify(ctx, never()).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.BAD_REQUEST))); } @@ -165,89 +174,81 @@ public void storeMissing() throws Exception { public void aclDisabledForSystemStore() throws Exception { isSystemStore[0] = true; hasStore[0] = true; - enumerate(hasAccess, hasAcl, isFailOpen, isMetadata, isHealthCheck); + enumerate(needsAcl, hasAccess, hasAcl, isFailOpen); verify(ctx, never()).writeAndFlush(any()); - // No access control (METADATA/HEALTH/ADMIN or system store) => 32 times - verify(ctx, times(32)).fireChannelRead(req); - } - - @Test - public void aclDisabledForMetadataEndpoint() throws Exception { - isMetadata[0] = true; - enumerate(hasAccess, hasAcl, isSystemStore, isFailOpen, isHealthCheck); - - verify(ctx, never()).writeAndFlush(any()); - // No access control (METADATA) => 32 times - verify(ctx, times(32)).fireChannelRead(req); - } - - @Test - public void aclDisabledForCurrentVersionEndpoint() throws Exception { - isCurrentVersion[0] = true; - enumerate(hasAccess, hasAcl, isSystemStore, isFailOpen, isHealthCheck); - - verify(ctx, never()).writeAndFlush(any()); - // No access control (CURRENT_VERSION) => 32 times - verify(ctx, times(32)).fireChannelRead(req); - } - - @Test - public void aclDisabledForHealthCheckEndpoint() throws Exception { - isHealthCheck[0] = true; - enumerate(hasAccess, hasAcl, isSystemStore, isFailOpen, isMetadata); - - verify(ctx, never()).writeAndFlush(any()); - // No access control (HEALTH) => 32 times - verify(ctx, times(32)).fireChannelRead(req); + // No access control (!needsAcl) => 8 times, needsAcl && hasStore && isSystemStore => 8 times + verify(ctx, times(16)).fireChannelRead(req); } @Test public void isBadUri() throws Exception { isBadUri[0] = true; - enumerate(hasAcl, hasStore, hasAccess, isSystemStore, isFailOpen, isMetadata, isHealthCheck); + enumerate(hasAcl, hasStore, hasAccess, isSystemStore, isFailOpen); - // all 128 times should fail for BAD_REQUEST - verify(ctx, times(128)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.BAD_REQUEST))); + // all 32 times should fail for BAD_REQUEST + verify(ctx, times(32)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.BAD_REQUEST))); } @Test public void aclMissing() throws Exception { hasAcl[0] = false; - enumerate(hasStore, hasAccess, isSystemStore, isFailOpen, isMetadata, isHealthCheck); + enumerate(needsAcl, hasStore, hasAccess, isSystemStore, isFailOpen); + + // !needsAcl = 16 times + // needsAcl && !hasStore = 8 times + // needsAcl && hasStore && isSystemStore = 4 times + // needsAcl && hasStore && !isSystemStore && hasAccess = 2 times + verify(ctx, times(30)).fireChannelRead(req); - // No access control (METADATA/HEALTH/ADMIN or system store) => 52 times, access control => 8 times granted - // (including !hasStore) - verify(ctx, times(60)).fireChannelRead(req); verify(ctx, never()).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.BAD_REQUEST))); - // Although we don't check for store existence due to potentially migrated store, we will still reject based on ACL - verify(ctx, times(2)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.UNAUTHORIZED))); - verify(ctx, times(2)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.FORBIDDEN))); + + // needsAcl && hasStore && !isSystemStore && !hasAccess && !hasAcl && !isFailOpen + verify(ctx, times(1)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.UNAUTHORIZED))); + + // needsAcl && hasStore && !isSystemStore && !hasAccess && (hasAcl || isFailOpen) + verify(ctx, times(1)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.FORBIDDEN))); } @Test public void aclPresent() throws Exception { hasAcl[0] = true; - enumerate(hasStore, hasAccess, isSystemStore, isFailOpen, isMetadata, isHealthCheck); + enumerate(needsAcl, hasStore, hasAccess, isSystemStore, isFailOpen); + + // !needsAcl = 16 times + // needsAcl && !hasStore + // needsAcl && hasStore && isSystemStore = 4 times + // needsAcl && hasStore && !isSystemStore && hasAccess = 2 times + verify(ctx, times(30)).fireChannelRead(req); - // No access control (METADATA/HEALTH/ADMIN or system store) => 52 times, access control => 8 times granted - // (including !hasStore) - verify(ctx, times(60)).fireChannelRead(req); + // needsAcl && !hasStore verify(ctx, never()).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.BAD_REQUEST))); + + // Since hasAcl = true, UNAUTHORIZED is not possible verify(ctx, never()).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.UNAUTHORIZED))); - // Although we don't check for store existence due to potentially migrated store, we will still reject based on ACL - verify(ctx, times(4)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.FORBIDDEN))); + + // needsAcl && hasStore && !isSystemStore && !hasAccess && hasAcl + verify(ctx, times(2)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.FORBIDDEN))); } @Test public void testAllCases() throws Exception { - enumerate(hasAcl, hasStore, hasAccess, isSystemStore, isFailOpen, isMetadata, isHealthCheck, isBadUri); + enumerate(needsAcl, hasAcl, hasStore, hasAccess, isSystemStore, isFailOpen, isBadUri); + + // !needsAcl = 64 times + // needsAcl && !isBadUri && !hasStore = 16 times + // needsAcl && !isBadUri && hasStore && isSystemStore = 8 times + // needsAcl && !isBadUri && hasStore && !isSystemStore && hasAccess = 4 times + verify(ctx, times(92)).fireChannelRead(req); - verify(ctx, times(120)).fireChannelRead(req); - verify(ctx, times(128)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.BAD_REQUEST))); - verify(ctx, times(2)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.UNAUTHORIZED))); - // One of the cases is impossible in reality. See StoreAclHandler.java comments - verify(ctx, times(6)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.FORBIDDEN))); + // needsAcl && isBadUri = 32 times + verify(ctx, times(32)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.BAD_REQUEST))); + + // needsAcl && !isBadUri && hasStore && !isSystemStore && !hasAccess && !hasAcl && !isFailOpen = 1 time + verify(ctx, times(1)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.UNAUTHORIZED))); + + // needsAcl && !isBadUri && hasStore && !isSystemStore && !hasAccess && (hasAcl || isFailOpen) = 3 times + verify(ctx, times(3)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.FORBIDDEN))); } private void update() throws Exception { @@ -256,42 +257,37 @@ private void update() throws Exception { when(accessController.isFailOpen()).thenReturn(isFailOpen[0]); when(metadataRepo.hasStore(any())).thenReturn(hasStore[0]); if (hasStore[0]) { - when(metadataRepo.getStoreOrThrow(any())).thenReturn(store); + when(metadataRepo.getStore(any())).thenReturn(store); } else { - when(metadataRepo.getStoreOrThrow(any())).thenThrow(new VeniceNoStoreException(storeName)); + when(metadataRepo.getStore(any())).thenReturn(null); } String storeNameInRequest = storeName; if (isSystemStore[0]) { storeNameInRequest = VeniceSystemStoreUtils.getMetaStoreName(storeName); } when(store.isSystemStore()).thenReturn(isSystemStore[0]); - if (isBadUri[0]) { + if (!needsAcl[0]) { + when(req.uri()).thenReturn("/noAcl"); + } else if (isBadUri[0]) { when(req.uri()).thenReturn("/badUri"); - } else if (isMetadata[0]) { - when(req.uri()).thenReturn(String.format("/metadata/%s/random", storeNameInRequest)); - } else if (isCurrentVersion[0]) { - when(req.uri()).thenReturn("/current_version/storename/random"); - } else if (isHealthCheck[0]) { - when(req.uri()).thenReturn("/health"); } else { - when(req.uri()).thenReturn(String.format("/storage/%s/random", storeNameInRequest)); + when(req.uri()).thenReturn(String.format("/goodUri/%s/random", storeNameInRequest)); } } /** * Generate every possible combination for a given list of booleans based on variables passed - * to boolean[]... conditions. If all variables (8 in count) are passed, then there will be 256 + * to boolean[]... conditions. If all variables (7 in count) are passed, then there will be 128 * combinations: * - * for (int i = 0; i < 256; i++) { | i= 0 1 2 3 4 ... + * for (int i = 0; i < 128; i++) { | i= 0 1 2 3 4 ... * _hasAccess= (i>>0) % 2 == 1| F T F T F ... * _hasAcl= (i>>1) % 2 == 1| F F T T F ... * _hasStore= (i>>2) % 2 == 1| F F F F T ... * _isAccessControlled= (i>>3) % 2 == 1| F F F F F ... * _isFailOpen= (i>>4) % 2 == 1| F F F F F ... - * _isMetadata= (i>>5) % 2 == 1| F F F F F ... - * _isHealthCheck= (i>>6) % 2 == 1| F F F F F ... - * _isBadUri= (i>>7) % 2 == 1| F F F F F ... + * _isBadUri= (i>>5) % 2 == 1| F F F F F ... + * _needsAcl= (i>>6) % 2 == 1| F F F F F ... * } */ private void enumerate(boolean[]... conditions) throws Exception { @@ -303,7 +299,7 @@ private void enumerate(boolean[]... conditions) throws Exception { } // New metadataRepo mock and aclHandler every update since thenThrow cannot be re-mocked. metadataRepo = mock(HelixReadOnlyStoreRepository.class); - aclHandler = spy(new StoreAclHandler(accessController, metadataRepo)); + AbstractStoreAclHandler aclHandler = spy(new MockStoreAclHandler(identityParser, accessController, metadataRepo)); update(); aclHandler.channelRead0(ctx, req); } @@ -312,7 +308,7 @@ private void enumerate(boolean[]... conditions) throws Exception { resetAllConditions(); } - public static class ContextMatcher implements ArgumentMatcher { + private static class ContextMatcher implements ArgumentMatcher { private HttpResponseStatus status; public ContextMatcher(HttpResponseStatus status) { @@ -324,4 +320,43 @@ public boolean matches(FullHttpResponse argument) { return argument.status().equals(status); } } + + private enum TestRequestType { + GOOD_URI, NO_ACL + } + + /** + * Assume a service with the following endpoints: + * /goodUri/{storeName}/random + * /noAcl + */ + private static class MockStoreAclHandler extends AbstractStoreAclHandler { + public MockStoreAclHandler( + IdentityParser identityParser, + DynamicAccessController accessController, + HelixReadOnlyStoreRepository metadataRepository) { + super(identityParser, accessController, metadataRepository); + } + + @Override + protected boolean needsAclValidation(TestRequestType requestType) { + return requestType != TestRequestType.NO_ACL; + } + + @Override + protected String extractStoreName(TestRequestType requestType, String[] requestParts) { + return requestParts[2]; + } + + @Override + protected TestRequestType validateRequest(String[] requestParts) { + if (requestParts[1].equals("noAcl")) { + return TestRequestType.NO_ACL; + } else if (requestParts[1].equals("goodUri")) { + return TestRequestType.GOOD_URI; + } else { + return null; + } + } + } } diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/grpc/GrpcUtilsTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/grpc/GrpcUtilsTest.java index e877b4e4803..d89a2d36751 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/grpc/GrpcUtilsTest.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/grpc/GrpcUtilsTest.java @@ -1,12 +1,13 @@ package com.linkedin.venice.grpc; -import static org.mockito.Mockito.*; -import static org.testng.Assert.*; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; +import com.linkedin.venice.acl.handler.AccessResult; import com.linkedin.venice.security.SSLFactory; import com.linkedin.venice.utils.SslUtils; import io.grpc.Status; -import io.netty.handler.codec.http.HttpResponseStatus; import javax.net.ssl.KeyManager; import javax.net.ssl.TrustManager; import org.testng.annotations.BeforeTest; @@ -39,35 +40,33 @@ public void testGetKeyManagers() throws Exception { @Test public void testHttpResponseStatusToGrpcStatus() { - final String permissionDeniedErrorMessage = "permission denied error message"; - Status grpcStatus = - GrpcUtils.httpResponseStatusToGrpcStatus(HttpResponseStatus.FORBIDDEN, permissionDeniedErrorMessage); - + Status grpcStatus = GrpcUtils.accessResultToGrpcStatus(AccessResult.GRANTED); assertEquals( grpcStatus.getCode(), - Status.PERMISSION_DENIED.getCode(), + Status.OK.getCode(), "Mismatch in GRPC status for the http response status permission denied"); assertEquals( - permissionDeniedErrorMessage, + AccessResult.GRANTED.getMessage(), grpcStatus.getDescription(), "Mismatch in error description for the mapped grpc status"); - final String unauthorizedErrorMessage = "unauthorized error message"; - grpcStatus = GrpcUtils.httpResponseStatusToGrpcStatus(HttpResponseStatus.UNAUTHORIZED, unauthorizedErrorMessage); + grpcStatus = GrpcUtils.accessResultToGrpcStatus(AccessResult.FORBIDDEN); assertEquals( grpcStatus.getCode(), Status.PERMISSION_DENIED.getCode(), - "Mismatch in GRPC status for the http response status unauthorized"); + "Mismatch in GRPC status for the http response status permission denied"); assertEquals( - unauthorizedErrorMessage, + AccessResult.FORBIDDEN.getMessage(), grpcStatus.getDescription(), "Mismatch in error description for the mapped grpc status"); - final String badRequestErrorMessage = "bad request error message"; - grpcStatus = GrpcUtils.httpResponseStatusToGrpcStatus(HttpResponseStatus.BAD_REQUEST, badRequestErrorMessage); - assertEquals(grpcStatus.getCode(), Status.UNKNOWN.getCode(), "Expected unknown status for everything else"); + grpcStatus = GrpcUtils.accessResultToGrpcStatus(AccessResult.UNAUTHORIZED); + assertEquals( + grpcStatus.getCode(), + Status.PERMISSION_DENIED.getCode(), + "Mismatch in GRPC status for the http response status unauthorized"); assertEquals( - badRequestErrorMessage, + AccessResult.UNAUTHORIZED.getMessage(), grpcStatus.getDescription(), "Mismatch in error description for the mapped grpc status"); } diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/kafka/protocol/enums/ControlMessageTypeTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/kafka/protocol/enums/ControlMessageTypeTest.java index 52ffaca0c93..d7f84865847 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/kafka/protocol/enums/ControlMessageTypeTest.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/kafka/protocol/enums/ControlMessageTypeTest.java @@ -1,30 +1,27 @@ package com.linkedin.venice.kafka.protocol.enums; -import static com.linkedin.venice.kafka.protocol.enums.ControlMessageType.END_OF_INCREMENTAL_PUSH; -import static com.linkedin.venice.kafka.protocol.enums.ControlMessageType.END_OF_PUSH; -import static com.linkedin.venice.kafka.protocol.enums.ControlMessageType.END_OF_SEGMENT; -import static com.linkedin.venice.kafka.protocol.enums.ControlMessageType.START_OF_INCREMENTAL_PUSH; -import static com.linkedin.venice.kafka.protocol.enums.ControlMessageType.START_OF_PUSH; -import static com.linkedin.venice.kafka.protocol.enums.ControlMessageType.START_OF_SEGMENT; -import static com.linkedin.venice.kafka.protocol.enums.ControlMessageType.TOPIC_SWITCH; -import static com.linkedin.venice.kafka.protocol.enums.ControlMessageType.VERSION_SWAP; +import com.linkedin.alpini.base.misc.CollectionUtil; +import com.linkedin.venice.utils.VeniceEnumValueTest; +import java.util.Map; -import org.testng.Assert; -import org.testng.annotations.Test; +public class ControlMessageTypeTest extends VeniceEnumValueTest { + public ControlMessageTypeTest() { + super(ControlMessageType.class); + } -public class ControlMessageTypeTest { - @Test - public void test() { - String assertionErrorMessage = "The value ID of enums should not be changed, as that is backwards incompatible."; - - Assert.assertEquals(ControlMessageType.valueOf(0), START_OF_PUSH, assertionErrorMessage); - Assert.assertEquals(ControlMessageType.valueOf(1), END_OF_PUSH, assertionErrorMessage); - Assert.assertEquals(ControlMessageType.valueOf(2), START_OF_SEGMENT, assertionErrorMessage); - Assert.assertEquals(ControlMessageType.valueOf(3), END_OF_SEGMENT, assertionErrorMessage); - Assert.assertEquals(ControlMessageType.valueOf(5), START_OF_INCREMENTAL_PUSH, assertionErrorMessage); - Assert.assertEquals(ControlMessageType.valueOf(6), END_OF_INCREMENTAL_PUSH, assertionErrorMessage); - Assert.assertEquals(ControlMessageType.valueOf(7), TOPIC_SWITCH, assertionErrorMessage); - Assert.assertEquals(ControlMessageType.valueOf(8), VERSION_SWAP, assertionErrorMessage); + @Override + protected Map expectedMapping() { + return CollectionUtil.mapBuilder() + .put(0, ControlMessageType.START_OF_PUSH) + .put(1, ControlMessageType.END_OF_PUSH) + .put(2, ControlMessageType.START_OF_SEGMENT) + .put(3, ControlMessageType.END_OF_SEGMENT) + .put(4, ControlMessageType.START_OF_BUFFER_REPLAY) + .put(5, ControlMessageType.START_OF_INCREMENTAL_PUSH) + .put(6, ControlMessageType.END_OF_INCREMENTAL_PUSH) + .put(7, ControlMessageType.TOPIC_SWITCH) + .put(8, ControlMessageType.VERSION_SWAP) + .build(); } } diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/kafka/protocol/enums/MessageTypeTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/kafka/protocol/enums/MessageTypeTest.java new file mode 100644 index 00000000000..2477b2ef839 --- /dev/null +++ b/internal/venice-common/src/test/java/com/linkedin/venice/kafka/protocol/enums/MessageTypeTest.java @@ -0,0 +1,22 @@ +package com.linkedin.venice.kafka.protocol.enums; + +import com.linkedin.alpini.base.misc.CollectionUtil; +import com.linkedin.venice.utils.VeniceEnumValueTest; +import java.util.Map; + + +public class MessageTypeTest extends VeniceEnumValueTest { + public MessageTypeTest() { + super(MessageType.class); + } + + @Override + protected Map expectedMapping() { + return CollectionUtil.mapBuilder() + .put(0, MessageType.PUT) + .put(1, MessageType.DELETE) + .put(2, MessageType.CONTROL_MESSAGE) + .put(3, MessageType.UPDATE) + .build(); + } +} diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/status/TestPushJobDetailsStatus.java b/internal/venice-common/src/test/java/com/linkedin/venice/status/TestPushJobDetailsStatus.java new file mode 100644 index 00000000000..82f72c1da9e --- /dev/null +++ b/internal/venice-common/src/test/java/com/linkedin/venice/status/TestPushJobDetailsStatus.java @@ -0,0 +1,41 @@ +package com.linkedin.venice.status; + +import static com.linkedin.venice.status.PushJobDetailsStatus.COMPLETED; +import static com.linkedin.venice.status.PushJobDetailsStatus.ERROR; +import static com.linkedin.venice.status.PushJobDetailsStatus.KILLED; +import static com.linkedin.venice.status.PushJobDetailsStatus.isFailed; +import static com.linkedin.venice.status.PushJobDetailsStatus.isSucceeded; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; + +import org.testng.annotations.Test; + + +public class TestPushJobDetailsStatus { + @Test + public void testIsFailedOrIsSuccess() { + for (PushJobDetailsStatus status: PushJobDetailsStatus.values()) { + if (status == COMPLETED) { + assertTrue(isSucceeded(status)); + assertFalse(isFailed(status)); + } else if (status == ERROR || status == KILLED) { + assertTrue(isFailed(status)); + assertFalse(isSucceeded(status)); + } else { + assertFalse(isSucceeded(status)); + assertFalse(isFailed(status)); + } + } + } + + @Test + public void testIsTerminal() { + for (PushJobDetailsStatus status: PushJobDetailsStatus.values()) { + if (status == COMPLETED || status == ERROR || status == KILLED) { + assertTrue(PushJobDetailsStatus.isTerminal(status.getValue())); + } else { + assertFalse(PushJobDetailsStatus.isTerminal(status.getValue())); + } + } + } +} diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/system/store/MetaStoreDataTypeTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/system/store/MetaStoreDataTypeTest.java new file mode 100644 index 00000000000..fd6db7adbcd --- /dev/null +++ b/internal/venice-common/src/test/java/com/linkedin/venice/system/store/MetaStoreDataTypeTest.java @@ -0,0 +1,26 @@ +package com.linkedin.venice.system.store; + +import com.linkedin.alpini.base.misc.CollectionUtil; +import com.linkedin.venice.utils.VeniceEnumValueTest; +import java.util.Map; + + +public class MetaStoreDataTypeTest extends VeniceEnumValueTest { + public MetaStoreDataTypeTest() { + super(MetaStoreDataType.class); + } + + @Override + protected Map expectedMapping() { + return CollectionUtil.mapBuilder() + .put(0, MetaStoreDataType.STORE_PROPERTIES) + .put(1, MetaStoreDataType.STORE_KEY_SCHEMAS) + .put(2, MetaStoreDataType.STORE_VALUE_SCHEMAS) + .put(3, MetaStoreDataType.STORE_REPLICA_STATUSES) + .put(4, MetaStoreDataType.STORE_CLUSTER_CONFIG) + .put(5, MetaStoreDataType.STORE_VALUE_SCHEMA) + .put(6, MetaStoreDataType.VALUE_SCHEMAS_WRITTEN_PER_STORE_VERSION) + .put(7, MetaStoreDataType.HEARTBEAT) + .build(); + } +} diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/writer/LeaderCompleteStateTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/writer/LeaderCompleteStateTest.java new file mode 100644 index 00000000000..fd66d6643e3 --- /dev/null +++ b/internal/venice-common/src/test/java/com/linkedin/venice/writer/LeaderCompleteStateTest.java @@ -0,0 +1,20 @@ +package com.linkedin.venice.writer; + +import com.linkedin.alpini.base.misc.CollectionUtil; +import com.linkedin.venice.utils.VeniceEnumValueTest; +import java.util.Map; + + +public class LeaderCompleteStateTest extends VeniceEnumValueTest { + public LeaderCompleteStateTest() { + super(LeaderCompleteState.class); + } + + @Override + protected Map expectedMapping() { + return CollectionUtil.mapBuilder() + .put(0, LeaderCompleteState.LEADER_NOT_COMPLETED) + .put(1, LeaderCompleteState.LEADER_COMPLETED) + .build(); + } +} diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestChangelogConsumer.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestChangelogConsumer.java index 40ce572924c..64ed5190b13 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestChangelogConsumer.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestChangelogConsumer.java @@ -5,6 +5,7 @@ import static com.linkedin.venice.ConfigKeys.CLUSTER_NAME; import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; import static com.linkedin.venice.ConfigKeys.KAFKA_LINGER_MS; +import static com.linkedin.venice.ConfigKeys.SERVER_AA_WC_WORKLOAD_PARALLEL_PROCESSING_ENABLED; import static com.linkedin.venice.ConfigKeys.ZOOKEEPER_ADDRESS; import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; @@ -105,6 +106,10 @@ public class TestChangelogConsumer { private VeniceClusterWrapper clusterWrapper; private ControllerClient parentControllerClient; + protected boolean isAAWCParallelProcessingEnabled() { + return false; + } + @BeforeClass(alwaysRun = true) public void setUp() { Properties serverProperties = new Properties(); @@ -113,6 +118,7 @@ public void setUp() { serverProperties.put( CHILD_DATA_CENTER_KAFKA_URL_PREFIX + "." + DEFAULT_PARENT_DATA_CENTER_REGION_NAME, "localhost:" + TestUtils.getFreePort()); + serverProperties.put(SERVER_AA_WC_WORKLOAD_PARALLEL_PROCESSING_ENABLED, isAAWCParallelProcessingEnabled()); multiRegionMultiClusterWrapper = ServiceFactory.getVeniceTwoLayerMultiRegionMultiClusterWrapper( 1, 1, @@ -617,7 +623,7 @@ public void testAAIngestionWithStoreView() throws Exception { TestUtils.waitForNonDeterministicAssertion( 8, TimeUnit.SECONDS, - () -> Assert.assertEquals(TestView.getInstance().getRecordCountForStore(storeName), 86)); + () -> Assert.assertEquals(TestView.getInstance().getRecordCountForStore(storeName), 85)); parentControllerClient.disableAndDeleteStore(storeName); // Verify that topics and store is cleaned up TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestChangelogConsumerWithParallelProcessing.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestChangelogConsumerWithParallelProcessing.java new file mode 100644 index 00000000000..8e57e9c7f7e --- /dev/null +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestChangelogConsumerWithParallelProcessing.java @@ -0,0 +1,8 @@ +package com.linkedin.venice.consumer; + +public class TestChangelogConsumerWithParallelProcessing extends TestChangelogConsumer { + @Override + protected boolean isAAWCParallelProcessingEnabled() { + return true; + } +} diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientDiskFullTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientDiskFullTest.java index dda4f1fb6dd..6ad754f4ef4 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientDiskFullTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientDiskFullTest.java @@ -11,8 +11,6 @@ import static com.linkedin.venice.ConfigKeys.SERVER_DISK_FULL_THRESHOLD; import static com.linkedin.venice.ConfigKeys.SERVER_PROMOTION_TO_LEADER_REPLICA_DELAY_SECONDS; import static com.linkedin.venice.ConfigKeys.USE_DA_VINCI_SPECIFIC_EXECUTION_STATUS_FOR_ERROR; -import static com.linkedin.venice.hadoop.VenicePushJob.PushJobCheckpoints.DVC_INGESTION_ERROR_DISK_FULL; -import static com.linkedin.venice.hadoop.VenicePushJob.PushJobCheckpoints.START_JOB_STATUS_POLLING; import static com.linkedin.venice.hadoop.VenicePushJobConstants.PUSH_JOB_STATUS_UPLOAD_ENABLE; import static com.linkedin.venice.integration.utils.ServiceFactory.getVeniceCluster; import static com.linkedin.venice.meta.PersistenceType.ROCKS_DB; @@ -33,6 +31,7 @@ import com.linkedin.davinci.client.StorageClass; import com.linkedin.davinci.client.factory.CachingDaVinciClientFactory; import com.linkedin.venice.D2.D2ClientUtils; +import com.linkedin.venice.PushJobCheckpoints; import com.linkedin.venice.client.store.AvroGenericStoreClient; import com.linkedin.venice.client.store.ClientConfig; import com.linkedin.venice.client.store.ClientFactory; @@ -290,8 +289,8 @@ public void testDaVinciDiskFullFailure(boolean useDaVinciSpecificExecutionStatus .getPushJobLatestCheckpoint() .intValue(), useDaVinciSpecificExecutionStatusForError - ? DVC_INGESTION_ERROR_DISK_FULL.getValue() - : START_JOB_STATUS_POLLING.getValue()); + ? PushJobCheckpoints.DVC_INGESTION_ERROR_DISK_FULL.getValue() + : PushJobCheckpoints.START_JOB_STATUS_POLLING.getValue()); } finally { controllerClient.disableAndDeleteStore(storeName); } diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/PartialUpdateTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/PartialUpdateTest.java index b47eb872020..8f7fb3a6272 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/PartialUpdateTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/PartialUpdateTest.java @@ -160,6 +160,10 @@ public class PartialUpdateTest { private VeniceControllerWrapper parentController; private List childDatacenters; + protected boolean isAAWCParallelProcessingEnabled() { + return false; + } + @BeforeClass(alwaysRun = true) public void setUp() { Properties serverProperties = new Properties(); @@ -167,6 +171,9 @@ public void setUp() { serverProperties.put( ConfigKeys.SERVER_CONSUMER_POOL_ALLOCATION_STRATEGY, KafkaConsumerServiceDelegator.ConsumerPoolStrategyType.CURRENT_VERSION_PRIORITIZATION.name()); + serverProperties.put( + ConfigKeys.SERVER_AA_WC_WORKLOAD_PARALLEL_PROCESSING_ENABLED, + Boolean.toString(isAAWCParallelProcessingEnabled())); Properties controllerProps = new Properties(); controllerProps.put(ConfigKeys.CONTROLLER_AUTO_MATERIALIZE_META_SYSTEM_STORE, false); this.multiRegionMultiClusterWrapper = ServiceFactory.getVeniceTwoLayerMultiRegionMultiClusterWrapper( diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/PartialUpdateWithParallelProcessingTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/PartialUpdateWithParallelProcessingTest.java new file mode 100644 index 00000000000..91dae72a3d1 --- /dev/null +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/PartialUpdateWithParallelProcessingTest.java @@ -0,0 +1,8 @@ +package com.linkedin.venice.endToEnd; + +public class PartialUpdateWithParallelProcessingTest extends PartialUpdateTest { + @Override + protected boolean isAAWCParallelProcessingEnabled() { + return true; + } +} diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/PushJobDetailsTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/PushJobDetailsTest.java index 631403a9628..c02e902aa4f 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/PushJobDetailsTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/PushJobDetailsTest.java @@ -3,14 +3,18 @@ import static com.linkedin.davinci.store.rocksdb.RocksDBServerConfig.ROCKSDB_PLAIN_TABLE_FORMAT_ENABLED; import static com.linkedin.venice.ConfigKeys.SERVER_DATABASE_CHECKSUM_VERIFICATION_ENABLED; import static com.linkedin.venice.ConfigKeys.SERVER_DATABASE_SYNC_BYTES_INTERNAL_FOR_DEFERRED_WRITE_MODE; -import static com.linkedin.venice.hadoop.VenicePushJob.PushJobCheckpoints.RECORD_TOO_LARGE_FAILED; -import static com.linkedin.venice.hadoop.VenicePushJob.PushJobCheckpoints.START_DATA_WRITER_JOB; +import static com.linkedin.venice.PushJobCheckpoints.DUP_KEY_WITH_DIFF_VALUE; +import static com.linkedin.venice.PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED; +import static com.linkedin.venice.PushJobCheckpoints.START_DATA_WRITER_JOB; import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; +import static com.linkedin.venice.hadoop.VenicePushJobConstants.INCREMENTAL_PUSH; import static com.linkedin.venice.hadoop.VenicePushJobConstants.PUSH_JOB_STATUS_UPLOAD_ENABLE; +import static com.linkedin.venice.status.PushJobDetailsStatus.COMPLETED; +import static com.linkedin.venice.status.PushJobDetailsStatus.END_OF_INCREMENTAL_PUSH_RECEIVED; import static com.linkedin.venice.utils.IntegrationTestPushUtils.defaultVPJProps; +import static com.linkedin.venice.utils.TestWriteUtils.DEFAULT_USER_DATA_VALUE_PREFIX; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; -import static com.linkedin.venice.writer.VeniceWriter.MAX_SIZE_FOR_USER_PAYLOAD_PER_MESSAGE_IN_BYTES; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; @@ -19,6 +23,7 @@ import static org.testng.Assert.fail; import com.linkedin.venice.ConfigKeys; +import com.linkedin.venice.PushJobCheckpoints; import com.linkedin.venice.client.store.AvroGenericStoreClient; import com.linkedin.venice.client.store.AvroSpecificStoreClient; import com.linkedin.venice.client.store.ClientConfig; @@ -41,6 +46,8 @@ import com.linkedin.venice.utils.TestWriteUtils; import com.linkedin.venice.utils.Time; import com.linkedin.venice.utils.Utils; +import io.tehuti.Metric; +import io.tehuti.metrics.MetricsRepository; import java.io.File; import java.io.IOException; import java.util.Arrays; @@ -49,7 +56,6 @@ import java.util.Map; import java.util.Optional; import java.util.Properties; -import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import org.apache.avro.Schema; import org.testng.annotations.AfterClass; @@ -65,9 +71,18 @@ public class PushJobDetailsTest { private ControllerClient controllerClient; private ControllerClient parentControllerClient; private Schema recordSchema; - private String inputDirPath; + private String inputDirPathForFullPush; + private String inputDirPathForIncPush; + private MetricsRepository metricsRepository; - @BeforeClass + private double batchJobSuccessExpected = 0.0; + private double incrementalJobSuccessExpected = 0.0; + private double batchJobFailedUserErrorExpected = 0.0; + private double batchJobFailedNonUserErrorExpected = 0.0; + private double incrementalJobFailedUserErrorExpected = 0.0; + private double incrementalJobFailedNonUserErrorExpected = 0.0; + + @BeforeClass() public void setUp() throws IOException { Properties serverProperties = new Properties(); serverProperties.setProperty(ROCKSDB_PLAIN_TABLE_FORMAT_ENABLED, "false"); @@ -93,7 +108,7 @@ public void setUp() throws IOException { VeniceMultiClusterWrapper childRegionMultiClusterWrapper = multiRegionMultiClusterWrapper.getChildRegions().get(0); childRegionClusterWrapper = childRegionMultiClusterWrapper.getClusters().get(clusterName); - + metricsRepository = multiRegionMultiClusterWrapper.getParentControllers().get(0).getMetricRepository(); controllerClient = new ControllerClient(clusterName, childRegionMultiClusterWrapper.getControllerConnectString()); parentControllerClient = new ControllerClient(clusterName, multiRegionMultiClusterWrapper.getControllerConnectString()); @@ -103,11 +118,14 @@ public void setUp() throws IOException { 2, TimeUnit.MINUTES); File inputDir = getTempDataDirectory(); - inputDirPath = "file://" + inputDir.getAbsolutePath(); + inputDirPathForFullPush = "file://" + inputDir.getAbsolutePath(); recordSchema = TestWriteUtils.writeSimpleAvroFileWithStringToStringSchema(inputDir); for (int i = 1; i <= latestSchemaId; i++) { schemaVersionMap.put(i, Utils.getSchemaFromResource("avro/PushJobDetails/v" + i + "/PushJobDetails.avsc")); } + inputDir = getTempDataDirectory(); + inputDirPathForIncPush = "file://" + inputDir.getAbsolutePath(); + TestWriteUtils.writeSimpleAvroFileWithStringToStringSchema2(inputDir); } @AfterClass @@ -116,24 +134,163 @@ public void cleanUp() { Utils.closeQuietlyWithErrorLogged(multiRegionMultiClusterWrapper); } - @Test(timeOut = 60 * Time.MS_PER_SECOND) - public void testPushJobDetails() throws ExecutionException, InterruptedException, IOException { - String testStoreName = "test-push-store"; - parentControllerClient.createNewStore( - testStoreName, - "test-user", - recordSchema.getField(DEFAULT_KEY_FIELD_PROP).schema().toString(), - recordSchema.getField(DEFAULT_VALUE_FIELD_PROP).schema().toString()); - // Set store quota to unlimited else local VPJ jobs will fail due to quota enforcement NullPointerException because - // hadoop job client cannot fetch counters properly. - parentControllerClient - .updateStore(testStoreName, new UpdateStoreQueryParams().setStorageQuotaInByte(-1).setPartitionCount(2)); - Properties pushJobProps = defaultVPJProps(multiRegionMultiClusterWrapper, inputDirPath, testStoreName); - pushJobProps.setProperty(PUSH_JOB_STATUS_UPLOAD_ENABLE, String.valueOf(true)); - try (VenicePushJob testPushJob = new VenicePushJob("test-push-job-details-job", pushJobProps)) { - testPushJob.run(); + private void validatePushJobMetrics(boolean isSucceeded, boolean isUserError, boolean isIncrementalPush) { + double batchJobSuccessExpectedCountSinceLastMeasurement = 0.0; + double incrementalJobSuccessExpectedCountSinceLastMeasurement = 0.0; + double batchJobFailedUserErrorExpectedCountSinceLastMeasurement = 0.0; + double batchJobFailedNonUserErrorExpectedCountSinceLastMeasurement = 0.0; + double incrementalJobFailedUserErrorExpectedCountSinceLastMeasurement = 0.0; + double incrementalJobFailedNonUserErrorExpectedCountSinceLastMeasurement = 0.0; + + Map metrics = metricsRepository.metrics(); + if (isSucceeded) { + if (isIncrementalPush) { + incrementalJobSuccessExpected += 1.0; + incrementalJobSuccessExpectedCountSinceLastMeasurement = 1.0; + } else { + batchJobSuccessExpected += 1.0; + batchJobSuccessExpectedCountSinceLastMeasurement = 1.0; + } + } else { + if (isUserError) { + if (isIncrementalPush) { + incrementalJobFailedUserErrorExpected += 1.0; + incrementalJobFailedUserErrorExpectedCountSinceLastMeasurement = 1.0; + } else { + batchJobFailedUserErrorExpected += 1.0; + batchJobFailedUserErrorExpectedCountSinceLastMeasurement = 1.0; + } + } else { + if (isIncrementalPush) { + incrementalJobFailedNonUserErrorExpected += 1.0; + incrementalJobFailedNonUserErrorExpectedCountSinceLastMeasurement = 1.0; + } else { + batchJobFailedNonUserErrorExpected += 1.0; + batchJobFailedNonUserErrorExpectedCountSinceLastMeasurement = 1.0; + } + } } + double finalIncrementalJobSuccessExpectedCountSinceLastMeasurement = + incrementalJobSuccessExpectedCountSinceLastMeasurement; + double finalBatchJobSuccessExpectedCountSinceLastMeasurement = batchJobSuccessExpectedCountSinceLastMeasurement; + double finalBatchJobFailedUserErrorExpectedCountSinceLastMeasurement = + batchJobFailedUserErrorExpectedCountSinceLastMeasurement; + double finalBatchJobFailedNonUserErrorExpectedCountSinceLastMeasurement = + batchJobFailedNonUserErrorExpectedCountSinceLastMeasurement; + double finalIncrementalJobFailedUserErrorExpectedCountSinceLastMeasurement = + incrementalJobFailedUserErrorExpectedCountSinceLastMeasurement; + double finalIncrementalJobFailedNonUserErrorExpectedCountSinceLastMeasurement = + incrementalJobFailedNonUserErrorExpectedCountSinceLastMeasurement; + + TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { + try { + double batchJobSuccess = metrics.containsKey(".venice-cluster0--batch_push_job_success.Count") + ? metrics.get(".venice-cluster0--batch_push_job_success.Count").value() + : 0.0; + double batchJobSuccessCountSinceLastMeasurement = + metrics.containsKey(".venice-cluster0--batch_push_job_success.CountSinceLastMeasurement") + ? metrics.get(".venice-cluster0--batch_push_job_success.CountSinceLastMeasurement").value() + : 0.0; + double incrementalJobSuccess = metrics.containsKey(".venice-cluster0--incremental_push_job_success.Count") + ? metrics.get(".venice-cluster0--incremental_push_job_success.Count").value() + : 0.0; + double incrementalJobSuccessCountSinceLastMeasurement = + metrics.containsKey(".venice-cluster0--incremental_push_job_success.CountSinceLastMeasurement") + ? metrics.get(".venice-cluster0--incremental_push_job_success.CountSinceLastMeasurement").value() + : 0.0; + double batchJobFailedUserError = metrics.containsKey(".venice-cluster0--batch_push_job_failed_user_error.Count") + ? metrics.get(".venice-cluster0--batch_push_job_failed_user_error.Count").value() + : 0.0; + double batchJobFailedUserErrorCountSinceLastMeasurement = + metrics.containsKey(".venice-cluster0--batch_push_job_failed_user_error.CountSinceLastMeasurement") + ? metrics.get(".venice-cluster0--batch_push_job_failed_user_error.CountSinceLastMeasurement").value() + : 0.0; + double batchJobFailedNonUserError = + metrics.containsKey(".venice-cluster0--batch_push_job_failed_non_user_error.Count") + ? metrics.get(".venice-cluster0--batch_push_job_failed_non_user_error.Count").value() + : 0.0; + double batchJobFailedNonUserErrorCountSinceLastMeasurement = + metrics.containsKey(".venice-cluster0--batch_push_job_failed_non_user_error.CountSinceLastMeasurement") + ? metrics.get(".venice-cluster0--batch_push_job_failed_non_user_error.CountSinceLastMeasurement") + .value() + : 0.0; + double incrementalJobFailedUserError = + metrics.containsKey(".venice-cluster0--incremental_push_job_failed_user_error.Count") + ? metrics.get(".venice-cluster0--incremental_push_job_failed_user_error.Count").value() + : 0.0; + double incrementalJobFailedUserCountSinceLastMeasurement = + metrics.containsKey(".venice-cluster0--incremental_push_job_failed_user_error.CountSinceLastMeasurement") + ? metrics.get(".venice-cluster0--incremental_push_job_failed_user_error.CountSinceLastMeasurement") + .value() + : 0.0; + double incrementalJobFailedNonUserError = + metrics.containsKey(".venice-cluster0--incremental_push_job_failed_non_user_error.Count") + ? metrics.get(".venice-cluster0--incremental_push_job_failed_non_user_error.Count").value() + : 0.0; + double incrementalJobFailedNonUserCountSinceLastMeasurement = metrics + .containsKey(".venice-cluster0--incremental_push_job_failed_non_user_error.CountSinceLastMeasurement") + ? metrics.get(".venice-cluster0--incremental_push_job_failed_non_user_error.CountSinceLastMeasurement") + .value() + : 0.0; + assertEquals( + incrementalJobSuccess, + incrementalJobSuccessExpected, + "Incremental push job success metric is incorrect"); + assertEquals( + incrementalJobSuccessCountSinceLastMeasurement, + finalIncrementalJobSuccessExpectedCountSinceLastMeasurement, + "Incremental push job success metric is incorrect"); + assertEquals(batchJobSuccess, batchJobSuccessExpected, "Batch push job success metric is incorrect"); + assertEquals( + batchJobSuccessCountSinceLastMeasurement, + finalBatchJobSuccessExpectedCountSinceLastMeasurement, + "Batch push job success metric is incorrect"); + assertEquals( + batchJobFailedUserError, + batchJobFailedUserErrorExpected, + "Batch push job failed user error metric is incorrect"); + assertEquals( + batchJobFailedUserErrorCountSinceLastMeasurement, + finalBatchJobFailedUserErrorExpectedCountSinceLastMeasurement, + "Batch push job failed user error metric is incorrect"); + assertEquals( + batchJobFailedNonUserError, + batchJobFailedNonUserErrorExpected, + "Batch push job failed non user error metric is incorrect"); + assertEquals( + batchJobFailedNonUserErrorCountSinceLastMeasurement, + finalBatchJobFailedNonUserErrorExpectedCountSinceLastMeasurement, + "Batch push job failed non user error metric is incorrect"); + assertEquals( + incrementalJobFailedUserError, + incrementalJobFailedUserErrorExpected, + "Incremental push job failed user error metric is incorrect"); + assertEquals( + incrementalJobFailedUserCountSinceLastMeasurement, + finalIncrementalJobFailedUserErrorExpectedCountSinceLastMeasurement, + "Incremental push job failed user error metric is incorrect"); + assertEquals( + incrementalJobFailedNonUserError, + incrementalJobFailedNonUserErrorExpected, + "Incremental push job failed non user error metric is incorrect"); + assertEquals( + incrementalJobFailedNonUserCountSinceLastMeasurement, + finalIncrementalJobFailedNonUserErrorExpectedCountSinceLastMeasurement, + "Incremental push job failed non user error metric is incorrect"); + } catch (Exception e) { + throw new VeniceException(e); + } + }); + } + private void validatePushJobDetailsStatus( + boolean isIncPush, + String testStoreName, + int version, + List expectedStatuses, + PushJobCheckpoints checkpoint, + boolean isSuccess, + String failureDetails) { // Verify the sent push job details. try (AvroSpecificStoreClient client = ClientFactory.getAndStartSpecificAvroClient( @@ -142,7 +299,7 @@ public void testPushJobDetails() throws ExecutionException, InterruptedException .setVeniceURL(childRegionClusterWrapper.getRandomRouterURL()))) { PushJobStatusRecordKey key = new PushJobStatusRecordKey(); key.storeName = testStoreName; - key.versionNumber = 1; + key.versionNumber = version; TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { try { assertNotNull(client.get(key).get(), "RT writes are not reflected in store yet"); @@ -151,12 +308,6 @@ public void testPushJobDetails() throws ExecutionException, InterruptedException } }); - List expectedStatuses = Arrays.asList( - PushJobDetailsStatus.STARTED.getValue(), - PushJobDetailsStatus.TOPIC_CREATED.getValue(), - PushJobDetailsStatus.DATA_WRITER_COMPLETED.getValue(), - PushJobDetailsStatus.COMPLETED.getValue()); - TestUtils.waitForNonDeterministicAssertion(60, TimeUnit.SECONDS, true, () -> { PushJobDetails value = client.get(key).get(); assertEquals( @@ -167,43 +318,55 @@ public void testPushJobDetails() throws ExecutionException, InterruptedException assertEquals( value.overallStatus.size(), expectedStatuses.size(), - "Unexpected number of overall statuses in push job details"); + "Unexpected number of overall statuses in push job details. curr: " + value.overallStatus + ", expected: " + + expectedStatuses); for (int i = 0; i < expectedStatuses.size(); i++) { assertEquals(value.overallStatus.get(i).status, (int) expectedStatuses.get(i)); assertTrue(value.overallStatus.get(i).timestamp > 0, "Timestamp for status tuple is missing"); } - assertFalse(value.coloStatus.isEmpty(), "Region status shouldn't be empty"); - for (List tuple: value.coloStatus.values()) { - assertEquals( - tuple.get(tuple.size() - 1).status, - PushJobDetailsStatus.COMPLETED.getValue(), - "Latest status for every region should be COMPLETED"); - assertTrue(tuple.get(tuple.size() - 1).timestamp > 0, "Timestamp for region status tuple is missing"); + + if (isSuccess) { + assertFalse(value.coloStatus.isEmpty(), "Region status shouldn't be empty"); + for (List tuple: value.coloStatus.values()) { + assertEquals( + tuple.get(tuple.size() - 1).status, + isIncPush ? END_OF_INCREMENTAL_PUSH_RECEIVED.getValue() : COMPLETED.getValue(), + "Latest status for every region should be COMPLETED"); + assertTrue(tuple.get(tuple.size() - 1).timestamp > 0, "Timestamp for region status tuple is missing"); + } + assertTrue(value.jobDurationInMs > 0); + assertTrue(value.totalNumberOfRecords > 0); + assertTrue(value.totalKeyBytes > 0); + assertTrue(value.totalRawValueBytes > 0); + assertTrue(value.totalCompressedValueBytes > 0); + assertNotNull(value.pushJobConfigs); + assertFalse(value.pushJobConfigs.isEmpty()); + assertNotNull(value.producerConfigs); + assertTrue(value.producerConfigs.isEmpty()); } - assertTrue(value.jobDurationInMs > 0); - assertTrue(value.totalNumberOfRecords > 0); - assertTrue(value.totalKeyBytes > 0); - assertTrue(value.totalRawValueBytes > 0); - assertTrue(value.totalCompressedValueBytes > 0); - assertNotNull(value.pushJobConfigs); - assertFalse(value.pushJobConfigs.isEmpty()); - assertNotNull(value.producerConfigs); - assertTrue(value.producerConfigs.isEmpty()); + + assertEquals( + value.pushJobLatestCheckpoint.intValue(), + checkpoint.getValue(), + "Unexpected latest push job checkpoint reported"); + + assertEquals(value.failureDetails.toString(), failureDetails); }); } + } - // Verify records (note, records 1-100 have been pushed) + private void validatePushJobData(String testStoreName, int start, int end, boolean isIncPush) { try (AvroGenericStoreClient client = ClientFactory.getAndStartGenericAvroClient( ClientConfig.defaultGenericClientConfig(testStoreName) .setVeniceURL(childRegionClusterWrapper.getRandomRouterURL()))) { TestUtils.waitForNonDeterministicAssertion(10, TimeUnit.SECONDS, true, () -> { try { - for (int i = 1; i < 100; i++) { + for (int i = start; i <= end; i++) { String key = String.valueOf(i); Object value = client.get(key).get(); assertNotNull(value, "Key " + i + " should not be missing!"); - assertEquals(value.toString(), "test_name_" + key); + assertEquals(value.toString(), DEFAULT_USER_DATA_VALUE_PREFIX + (isIncPush ? (i * 2) : i)); } } catch (Exception e) { throw new VeniceException(e); @@ -212,83 +375,131 @@ public void testPushJobDetails() throws ExecutionException, InterruptedException } } - @Test(timeOut = 60 * Time.MS_PER_SECOND) - public void testPushJobDetailsFailureTags() throws ExecutionException, InterruptedException { - String testStoreName = "test-push-failure-store"; + @Test(timeOut = 120 * Time.MS_PER_SECOND) + public void testPushJobDetails() throws IOException { + // case 1: successful batch push job + String testStoreName = "test-push-store"; parentControllerClient.createNewStore( testStoreName, "test-user", recordSchema.getField(DEFAULT_KEY_FIELD_PROP).schema().toString(), recordSchema.getField(DEFAULT_VALUE_FIELD_PROP).schema().toString()); - // hadoop job client cannot fetch counters properly and should fail the job - parentControllerClient.updateStore(testStoreName, new UpdateStoreQueryParams().setStorageQuotaInByte(0)); - Properties pushJobProps = defaultVPJProps(multiRegionMultiClusterWrapper, inputDirPath, testStoreName); + // Set store quota to unlimited else local VPJ jobs will fail due to quota enforcement NullPointerException because + // hadoop job client cannot fetch counters properly. + parentControllerClient.updateStore( + testStoreName, + new UpdateStoreQueryParams().setStorageQuotaInByte(-1).setPartitionCount(2).setIncrementalPushEnabled(true)); + Properties pushJobProps = defaultVPJProps(multiRegionMultiClusterWrapper, inputDirPathForFullPush, testStoreName); pushJobProps.setProperty(PUSH_JOB_STATUS_UPLOAD_ENABLE, String.valueOf(true)); try (VenicePushJob testPushJob = new VenicePushJob("test-push-job-details-job", pushJobProps)) { + testPushJob.run(); + } + + validatePushJobData(testStoreName, 1, 100, false); + List expectedStatuses = Arrays.asList( + PushJobDetailsStatus.STARTED.getValue(), + PushJobDetailsStatus.TOPIC_CREATED.getValue(), + PushJobDetailsStatus.DATA_WRITER_COMPLETED.getValue(), + COMPLETED.getValue()); + validatePushJobDetailsStatus(false, testStoreName, 1, expectedStatuses, JOB_STATUS_POLLING_COMPLETED, true, ""); + validatePushJobMetrics(true, false, false); + + // case 2: successful incremental push job + Properties pushJobPropsInc = defaultVPJProps(multiRegionMultiClusterWrapper, inputDirPathForIncPush, testStoreName); + pushJobPropsInc.setProperty(PUSH_JOB_STATUS_UPLOAD_ENABLE, String.valueOf(true)); + pushJobPropsInc.setProperty(INCREMENTAL_PUSH, String.valueOf(true)); + try (VenicePushJob testPushJob = new VenicePushJob("test-push-job-details-job-with-inc-push", pushJobPropsInc)) { + testPushJob.run(); + } + + validatePushJobData(testStoreName, 51, 150, true); + expectedStatuses = Arrays.asList( + PushJobDetailsStatus.STARTED.getValue(), + PushJobDetailsStatus.TOPIC_CREATED.getValue(), + PushJobDetailsStatus.DATA_WRITER_COMPLETED.getValue(), + COMPLETED.getValue()); + validatePushJobDetailsStatus(true, testStoreName, 1, expectedStatuses, JOB_STATUS_POLLING_COMPLETED, true, ""); + validatePushJobMetrics(true, false, true); + + // case 3: failed batch push job, non-user error: + // setting the quota to be 0, hadoop job client cannot fetch counters properly and should fail the job + parentControllerClient.updateStore(testStoreName, new UpdateStoreQueryParams().setStorageQuotaInByte(0)); + try (VenicePushJob testPushJob = new VenicePushJob("test-push-job-details-job-v2", pushJobProps)) { assertThrows(VeniceException.class, testPushJob::run); } - try (AvroSpecificStoreClient client = - ClientFactory.getAndStartSpecificAvroClient( - ClientConfig - .defaultSpecificClientConfig(VeniceSystemStoreUtils.getPushJobDetailsStoreName(), PushJobDetails.class) - .setVeniceURL(childRegionClusterWrapper.getRandomRouterURL()))) { - PushJobStatusRecordKey key = new PushJobStatusRecordKey(); - key.storeName = testStoreName; - key.versionNumber = 1; - TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { - try { - assertNotNull(client.get(key).get(), "RT writes are not reflected in store yet"); - } catch (Exception e) { - fail("Unexpected exception thrown while reading from the venice store", e); - } - }); - PushJobDetails value = client.get(key).get(); - assertEquals( - value.pushJobLatestCheckpoint.intValue(), - START_DATA_WRITER_JOB.getValue(), - "Unexpected latest push job checkpoint reported"); - assertFalse(value.failureDetails.toString().isEmpty()); + + expectedStatuses = Arrays.asList( + PushJobDetailsStatus.STARTED.getValue(), + PushJobDetailsStatus.TOPIC_CREATED.getValue(), + PushJobDetailsStatus.ERROR.getValue()); + validatePushJobDetailsStatus( + false, + testStoreName, + 2, + expectedStatuses, + START_DATA_WRITER_JOB, + false, + "com.linkedin.venice.exceptions.VeniceException: Exception or error caught during VenicePushJob: java.io.IOException: Job failed!"); + validatePushJobMetrics(false, false, false); + + // case 4: failed incremental push job, non-user error + pushJobPropsInc = defaultVPJProps(multiRegionMultiClusterWrapper, inputDirPathForIncPush, testStoreName); + pushJobPropsInc.setProperty(PUSH_JOB_STATUS_UPLOAD_ENABLE, String.valueOf(true)); + pushJobPropsInc.setProperty(INCREMENTAL_PUSH, String.valueOf(true)); + try (VenicePushJob testPushJob = new VenicePushJob("test-push-job-details-job-with-inc-push-v2", pushJobPropsInc)) { + assertThrows(VeniceException.class, testPushJob::run); } - } - /** - * Test that the push job details are correctly updated when a large record is pushed. - * The settings `MAX_RECORD_SIZE_BYTES` and `MAX_SIZE_FOR_USER_PAYLOAD_PER_MESSAGE_IN_BYTES` are set to - * extremely low values, so that regular records will trigger the "large" condition and fail to be pushed. - */ - @Test(timeOut = 60 * Time.MS_PER_SECOND) - public void testPushJobDetailsRecordTooLarge() throws ExecutionException, InterruptedException { - String testStoreName = "test-push-store"; - parentControllerClient.createNewStore( + validatePushJobDetailsStatus( + true, testStoreName, - "test-user", - recordSchema.getField(DEFAULT_KEY_FIELD_PROP).schema().toString(), - recordSchema.getField(DEFAULT_VALUE_FIELD_PROP).schema().toString()); - // Set store quota to unlimited else local VPJ jobs will fail due to quota enforcement NullPointerException - final UpdateStoreQueryParams queryParams = new UpdateStoreQueryParams().setStorageQuotaInByte(-1) - .setPartitionCount(2) - .setChunkingEnabled(true) - .setMaxRecordSizeBytes(0); + 2, + expectedStatuses, + START_DATA_WRITER_JOB, + false, + "com.linkedin.venice.exceptions.VeniceException: Exception or error caught during VenicePushJob: java.io.IOException: Job failed!"); + validatePushJobMetrics(false, false, true); + + // case 5: failed batch push job, user error: data with duplicate keys + final UpdateStoreQueryParams queryParams = new UpdateStoreQueryParams().setStorageQuotaInByte(-1); parentControllerClient.updateStore(testStoreName, queryParams); - Properties pushJobProps = defaultVPJProps(multiRegionMultiClusterWrapper, inputDirPath, testStoreName); + File inputDir = getTempDataDirectory(); + String inputDirPathWithDupKeys = "file://" + inputDir.getAbsolutePath(); + TestWriteUtils.writeSimpleAvroFileWithDuplicateKey(inputDir); + + pushJobProps = defaultVPJProps(multiRegionMultiClusterWrapper, inputDirPathWithDupKeys, testStoreName); pushJobProps.setProperty(PUSH_JOB_STATUS_UPLOAD_ENABLE, String.valueOf(true)); - pushJobProps.setProperty(MAX_SIZE_FOR_USER_PAYLOAD_PER_MESSAGE_IN_BYTES, "0"); - try (final VenicePushJob testPushJob = new VenicePushJob("test-push-job-details-job", pushJobProps)) { - assertThrows(VeniceException.class, testPushJob::run); // Push job should fail due to large record + try (final VenicePushJob testPushJob = new VenicePushJob("test-push-job-details-job-v3", pushJobProps)) { + assertThrows(VeniceException.class, testPushJob::run); // Push job should fail } - try (final AvroSpecificStoreClient client = - ClientFactory.getAndStartSpecificAvroClient( - ClientConfig - .defaultSpecificClientConfig(VeniceSystemStoreUtils.getPushJobDetailsStoreName(), PushJobDetails.class) - .setVeniceURL(childRegionClusterWrapper.getRandomRouterURL()))) { - final PushJobStatusRecordKey key = new PushJobStatusRecordKey(testStoreName, 1); - TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { - assertNotNull(client.get(key).get(), "RT writes are not reflected in store yet"); - }); - final PushJobDetails value = client.get(key).get(); - assertEquals(value.pushJobLatestCheckpoint.intValue(), RECORD_TOO_LARGE_FAILED.getValue()); - assertFalse(value.failureDetails.toString().isEmpty(), "Assert failure recorded in PushJobDetails"); + + validatePushJobDetailsStatus( + false, + testStoreName, + 3, + expectedStatuses, + DUP_KEY_WITH_DIFF_VALUE, + false, + "com.linkedin.venice.exceptions.VeniceException: Input data has at least 9 keys that appear more than once but have different values"); + validatePushJobMetrics(false, true, false); + + // case 6: failed incremental push job, user error + pushJobPropsInc = defaultVPJProps(multiRegionMultiClusterWrapper, inputDirPathWithDupKeys, testStoreName); + pushJobPropsInc.setProperty(PUSH_JOB_STATUS_UPLOAD_ENABLE, String.valueOf(true)); + pushJobPropsInc.setProperty(INCREMENTAL_PUSH, String.valueOf(true)); + try (VenicePushJob testPushJob = new VenicePushJob("test-push-job-details-job-with-inc-push-v3", pushJobPropsInc)) { + assertThrows(VeniceException.class, testPushJob::run); } + + validatePushJobDetailsStatus( + true, + testStoreName, + 3, + expectedStatuses, + DUP_KEY_WITH_DIFF_VALUE, + false, + "com.linkedin.venice.exceptions.VeniceException: Input data has at least 9 keys that appear more than once but have different values"); + validatePushJobMetrics(false, true, true); } } diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveIngestion.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveIngestion.java index fc3fe3e0c7f..ebfdfcb4195 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveIngestion.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveIngestion.java @@ -4,6 +4,7 @@ import static com.linkedin.davinci.store.rocksdb.RocksDBServerConfig.ROCKSDB_PLAIN_TABLE_FORMAT_ENABLED; import static com.linkedin.venice.ConfigKeys.CHILD_DATA_CENTER_KAFKA_URL_PREFIX; import static com.linkedin.venice.ConfigKeys.DEFAULT_MAX_NUMBER_OF_PARTITIONS; +import static com.linkedin.venice.ConfigKeys.SERVER_AA_WC_WORKLOAD_PARALLEL_PROCESSING_ENABLED; import static com.linkedin.venice.ConfigKeys.SERVER_CONSUMER_POOL_ALLOCATION_STRATEGY; import static com.linkedin.venice.ConfigKeys.SERVER_DEDICATED_DRAINER_FOR_SORTED_INPUT_ENABLED; import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; @@ -98,6 +99,10 @@ protected boolean isLevel0CompactionTuningForReadWriteLeaderEnabled() { return false; } + protected boolean isAAWCParallelProcessingEnabled() { + return false; + } + @BeforeClass(alwaysRun = true) public void setUp() { serializer = new AvroSerializer(STRING_SCHEMA); @@ -114,6 +119,7 @@ public void setUp() { serverProperties.put( SERVER_CONSUMER_POOL_ALLOCATION_STRATEGY, KafkaConsumerServiceDelegator.ConsumerPoolStrategyType.CURRENT_VERSION_PRIORITIZATION.name()); + serverProperties.put(SERVER_AA_WC_WORKLOAD_PARALLEL_PROCESSING_ENABLED, isAAWCParallelProcessingEnabled()); Properties controllerProps = new Properties(); controllerProps.put(DEFAULT_MAX_NUMBER_OF_PARTITIONS, 20); multiRegionMultiClusterWrapper = ServiceFactory.getVeniceTwoLayerMultiRegionMultiClusterWrapper( diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveIngestionWithReadWriteLeaderCompactionTuning.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveIngestionWithReadWriteLeaderCompactionTuningAndParallelProcessing.java similarity index 52% rename from internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveIngestionWithReadWriteLeaderCompactionTuning.java rename to internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveIngestionWithReadWriteLeaderCompactionTuningAndParallelProcessing.java index d8996e5d3fc..50c19f4239e 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveIngestionWithReadWriteLeaderCompactionTuning.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveIngestionWithReadWriteLeaderCompactionTuningAndParallelProcessing.java @@ -1,11 +1,18 @@ package com.linkedin.venice.endToEnd; /** - * Integration test to verify active/active replication when applying read-write leader compaction optimization. + * Integration test to verify active/active replication when applying read-write leader compaction optimization + * and parallel processing of AA/WC workload. */ -public class TestActiveActiveIngestionWithReadWriteLeaderCompactionTuning extends TestActiveActiveIngestion { +public class TestActiveActiveIngestionWithReadWriteLeaderCompactionTuningAndParallelProcessing + extends TestActiveActiveIngestion { @Override protected boolean isLevel0CompactionTuningForReadWriteLeaderEnabled() { return true; } + + @Override + protected boolean isAAWCParallelProcessingEnabled() { + return true; + } } diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestFatalDataValidationExceptionHandling.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestFatalDataValidationExceptionHandling.java index 2dcfd00c200..cdd919d02cd 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestFatalDataValidationExceptionHandling.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestFatalDataValidationExceptionHandling.java @@ -27,6 +27,7 @@ import static org.testng.Assert.assertFalse; import com.linkedin.davinci.kafka.consumer.KafkaConsumerService; +import com.linkedin.venice.PushJobCheckpoints; import com.linkedin.venice.client.store.AvroSpecificStoreClient; import com.linkedin.venice.compression.CompressionStrategy; import com.linkedin.venice.controller.VeniceHelixAdmin; @@ -34,7 +35,6 @@ import com.linkedin.venice.controllerapi.JobStatusQueryResponse; import com.linkedin.venice.controllerapi.VersionResponse; import com.linkedin.venice.guid.GuidUtils; -import com.linkedin.venice.hadoop.VenicePushJob; import com.linkedin.venice.integration.utils.PubSubBrokerWrapper; import com.linkedin.venice.integration.utils.ServiceFactory; import com.linkedin.venice.integration.utils.VeniceClusterWrapper; @@ -246,7 +246,7 @@ private void initPushJobDetails(PushJobDetails pushJobDetails) { pushJobDetails.totalRawValueBytes = -1; pushJobDetails.totalCompressedValueBytes = -1; pushJobDetails.failureDetails = ""; - pushJobDetails.pushJobLatestCheckpoint = VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB.getValue(); + pushJobDetails.pushJobLatestCheckpoint = PushJobCheckpoints.INITIALIZE_PUSH_JOB.getValue(); pushJobDetails.pushJobConfigs = Collections.singletonMap(HEARTBEAT_ENABLED_CONFIG.getConfigName(), String.valueOf(true)); } diff --git a/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/VeniceEnumValueTest.java b/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/VeniceEnumValueTest.java new file mode 100644 index 00000000000..04602ca5f1d --- /dev/null +++ b/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/VeniceEnumValueTest.java @@ -0,0 +1,114 @@ +package com.linkedin.venice.utils; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotEquals; +import static org.testng.Assert.assertThrows; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; + +import com.linkedin.venice.exceptions.VeniceException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.lang.reflect.Modifier; +import java.util.Map; +import java.util.function.Function; +import org.testng.annotations.Test; + + +/** + * Abstract class which makes it as easy as possible to generically test all the assumptions for enums which implement + * the {@link VeniceEnumValue} interface. Subclasses only need to implement the constructor and the abstract function. + * + * @param the enum class under test + */ +@Test +public abstract class VeniceEnumValueTest { + private static final int INVALID_NEGATIVE_VALUE = -1; + private static final String VALUE_OF_METHOD_NAME = "valueOf"; + private static final String VALUES_METHOD_NAME = "values"; + private static final String ASSERTION_ERROR_MESSAGE = + "The value ID of enums should not be changed, as that is backwards incompatible."; + private final Class enumClass; + + protected VeniceEnumValueTest(Class enumClass) { + this.enumClass = enumClass; + } + + protected abstract Map expectedMapping(); + + @Test + public void test() { + int highestValue = INVALID_NEGATIVE_VALUE; + + // Check that there is a valueOf function which respects the expected contract + Method valueOfMethod = getPublicStaticFunction(this.enumClass, VALUE_OF_METHOD_NAME, int.class); + + assertTrue(Modifier.isStatic(valueOfMethod.getModifiers()), "The " + VALUE_OF_METHOD_NAME + " should be static!"); + assertTrue(Modifier.isPublic(valueOfMethod.getModifiers()), "The " + VALUE_OF_METHOD_NAME + " should be public!"); + + Function valueOfFunction = value -> { + try { + return (T) valueOfMethod.invoke(null, value); + } catch (Exception e) { + if (e.getClass() == InvocationTargetException.class && e.getCause() instanceof VeniceException) { + // Those are expected for invalid values, so we bubble them up. + throw (VeniceException) e.getCause(); + } + fail("The " + VALUE_OF_METHOD_NAME + " threw an exception!", e); + // N.B.: Although the return statement below is unreachable, since fail will throw, the compiler does not know + // that. + return null; + } + }; + + Map expectedMapping = expectedMapping(); + assertFalse(expectedMapping.isEmpty()); + + // Check that all mappings are as expected + for (Map.Entry entry: expectedMapping.entrySet()) { + assertEquals(valueOfFunction.apply(entry.getKey()), entry.getValue(), ASSERTION_ERROR_MESSAGE); + assertEquals(entry.getValue().getValue(), entry.getKey().intValue(), ASSERTION_ERROR_MESSAGE); + highestValue = Math.max(entry.getKey(), highestValue); + } + + // Check that out of bound IDs throw exceptions + assertNotEquals(highestValue, INVALID_NEGATIVE_VALUE, "There are no values at all in the enum!"); + + assertThrows(VeniceException.class, () -> valueOfFunction.apply(INVALID_NEGATIVE_VALUE)); + + final int tooHighValue = highestValue + 1; + assertThrows(VeniceException.class, () -> valueOfFunction.apply(tooHighValue)); + + // Check that no other enum values exist besides those that are expected + Method valuesFunction = getPublicStaticFunction(this.enumClass, VALUES_METHOD_NAME, new Class[0]); + try { + T[] types = (T[]) valuesFunction.invoke(null, new Class[0]); + for (T type: types) { + assertTrue( + expectedMapping.containsKey(type.getValue()), + "Class " + this.enumClass.getSimpleName() + " contains an unexpected value: " + type.getValue()); + } + } catch (Exception e) { + fail("The " + VALUES_METHOD_NAME + " threw an exception!", e); + } + } + + private static Method getPublicStaticFunction(Class klass, String functionName, Class... params) { + try { + Method function = klass.getDeclaredMethod(functionName, params); + assertTrue( + Modifier.isStatic(function.getModifiers()), + "Class " + klass.getSimpleName() + " should have a static " + functionName + " function!"); + assertTrue( + Modifier.isPublic(function.getModifiers()), + "Class " + klass.getSimpleName() + " should have a public " + functionName + " function!"); + return function; + } catch (NoSuchMethodException e) { + fail("Class " + klass.getSimpleName() + " should have a " + functionName + " method!", e); + // N.B.: Although the return statement below is unreachable, since fail will throw, the compiler does not know + // that. + return null; + } + } +} 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 517f0f0a4cf..cbc14a66e66 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 @@ -33,6 +33,7 @@ import com.linkedin.d2.balancer.D2Client; import com.linkedin.venice.ConfigKeys; import com.linkedin.venice.D2.D2ClientUtils; +import com.linkedin.venice.PushJobCheckpoints; import com.linkedin.venice.SSLConfig; import com.linkedin.venice.acl.DynamicAccessController; import com.linkedin.venice.client.store.AvroSpecificStoreClient; @@ -56,6 +57,7 @@ import com.linkedin.venice.controller.kafka.protocol.admin.HybridStoreConfigRecord; import com.linkedin.venice.controller.kafka.protocol.admin.StoreViewConfigRecord; import com.linkedin.venice.controller.stats.DisabledPartitionStats; +import com.linkedin.venice.controller.stats.PushJobStatusStats; import com.linkedin.venice.controllerapi.ControllerClient; import com.linkedin.venice.controllerapi.ControllerResponse; import com.linkedin.venice.controllerapi.ControllerRoute; @@ -182,10 +184,12 @@ import com.linkedin.venice.service.ICProvider; import com.linkedin.venice.stats.AbstractVeniceAggStats; import com.linkedin.venice.stats.ZkClientStatusStats; +import com.linkedin.venice.status.PushJobDetailsStatus; import com.linkedin.venice.status.StatusMessageChannel; import com.linkedin.venice.status.protocol.BatchJobHeartbeatKey; import com.linkedin.venice.status.protocol.BatchJobHeartbeatValue; import com.linkedin.venice.status.protocol.PushJobDetails; +import com.linkedin.venice.status.protocol.PushJobDetailsStatusTuple; import com.linkedin.venice.status.protocol.PushJobStatusRecordKey; import com.linkedin.venice.system.store.MetaStoreDataType; import com.linkedin.venice.system.store.MetaStoreReader; @@ -246,6 +250,7 @@ import java.util.stream.Collectors; import javax.annotation.Nonnull; import org.apache.avro.Schema; +import org.apache.avro.util.Utf8; import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.Validate; import org.apache.helix.AccessOption; @@ -394,6 +399,7 @@ public class VeniceHelixAdmin implements Admin, StoreCleaner { private int pushJobDetailsSchemaId = -1; private final Map disabledPartitionStatMap = new HashMap<>(); + private final Map pushJobStatusStatsMap = new HashMap<>(); private static final String PUSH_JOB_DETAILS_WRITER = "PUSH_JOB_DETAILS_WRITER"; private final Map jobTrackingVeniceWriterMap = new VeniceConcurrentHashMap<>(); @@ -673,8 +679,10 @@ public VeniceHelixAdmin( } HelixLiveInstanceMonitor liveInstanceMonitor = new HelixLiveInstanceMonitor(this.zkClient, clusterName); DisabledPartitionStats disabledPartitionStats = new DisabledPartitionStats(metricsRepository, clusterName); + PushJobStatusStats pushJobStatusStats = new PushJobStatusStats(metricsRepository, clusterName); disabledPartitionStatMap.put(clusterName, disabledPartitionStats); liveInstanceMonitorMap.put(clusterName, liveInstanceMonitor); + pushJobStatusStatsMap.put(clusterName, pushJobStatusStats); // Register new instance callback liveInstanceMonitor.registerLiveInstanceChangedListener(new LiveInstanceChangedListener() { @Override @@ -1174,6 +1182,70 @@ private Integer fetchSystemStoreSchemaId(String clusterName, String storeName, S return response.getId(); } + static boolean isPushJobFailedUserError(PushJobDetailsStatus status, PushJobCheckpoints pushJobCheckpoint) { + if (PushJobDetailsStatus.isFailed(status)) { + if (pushJobCheckpoint != null) { + switch (pushJobCheckpoint) { + case QUOTA_EXCEEDED: + case WRITE_ACL_FAILED: + case DUP_KEY_WITH_DIFF_VALUE: + case INPUT_DATA_SCHEMA_VALIDATION_FAILED: + case EXTENDED_INPUT_DATA_SCHEMA_VALIDATION_FAILED: + case RECORD_TOO_LARGE_FAILED: + case CONCURRENT_BATCH_PUSH: + case DATASET_CHANGED: + case INVALID_INPUT_FILE: + case DVC_INGESTION_ERROR_DISK_FULL: + case DVC_INGESTION_ERROR_MEMORY_LIMIT_REACHED: + return true; + + default: + return false; + } + } + } + return false; + } + + static void emitPushJobDetailsMetrics(Map pushJobStatusStatsMap, PushJobDetails value) { + List overallStatuses = value.getOverallStatus(); + if (overallStatuses.isEmpty()) { + return; + } + boolean isIncrementalPush = false; + String cluster = value.getClusterName().toString(); + PushJobStatusStats pushJobStatusStats = pushJobStatusStatsMap.get(cluster); + Utf8 incPushKey = new Utf8("incremental.push"); + if (value.getPushJobConfigs().containsKey(incPushKey)) { + isIncrementalPush = Boolean.parseBoolean(value.getPushJobConfigs().get(incPushKey).toString()); + } + PushJobDetailsStatus overallStatus = + PushJobDetailsStatus.fromInt(overallStatuses.get(overallStatuses.size() - 1).getStatus()); + PushJobCheckpoints checkpoint = PushJobCheckpoints.fromInt(value.getPushJobLatestCheckpoint()); + if (PushJobDetailsStatus.isFailed(overallStatus)) { + if (isPushJobFailedUserError(overallStatus, checkpoint)) { + if (isIncrementalPush) { + pushJobStatusStats.recordIncrementalPushFailureDueToUserErrorSensor(); + } else { + pushJobStatusStats.recordBatchPushFailureDueToUserErrorSensor(); + } + } else { + if (isIncrementalPush) { + pushJobStatusStats.recordIncrementalPushFailureNotDueToUserErrorSensor(); + } else { + pushJobStatusStats.recordBatchPushFailureNotDueToUserErrorSensor(); + } + } + } else if (PushJobDetailsStatus.isSucceeded(overallStatus)) { + // Emit metrics for successful push jobs + if (isIncrementalPush) { + pushJobStatusStats.recordIncrementalPushSuccessSensor(); + } else { + pushJobStatusStats.recordBatchPushSuccessSensor(); + } + } + } + /** * Lazy initialize a Venice writer for an internal real time topic store of push job details records. * Use this writer to put a pair of push job detail record (key and value). @@ -1182,6 +1254,9 @@ private Integer fetchSystemStoreSchemaId(String clusterName, String storeName, S */ @Override public void sendPushJobDetails(PushJobStatusRecordKey key, PushJobDetails value) { + // Emit push job status metrics + emitPushJobDetailsMetrics(pushJobStatusStatsMap, value); + // Send push job details to the push job status system store if (pushJobStatusStoreClusterName.isEmpty()) { throw new VeniceException( ("Unable to send the push job details because " + ConfigKeys.PUSH_JOB_STATUS_STORE_CLUSTER_NAME) @@ -5755,6 +5830,10 @@ public DisabledPartitionStats getDisabledPartitionStats(String clusterName) { return disabledPartitionStatMap.get(clusterName); } + public PushJobStatusStats getPushJobStatusStats(String clusterName) { + return pushJobStatusStatsMap.get(clusterName); + } + /** * @return a map containing the storage node name and its connectivity status (InstanceStatus). */ diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/JobRoutes.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/JobRoutes.java index 31b36030c24..31f55e59873 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/JobRoutes.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/JobRoutes.java @@ -4,7 +4,6 @@ import static com.linkedin.venice.controllerapi.ControllerApiConstants.FABRIC; import static com.linkedin.venice.controllerapi.ControllerApiConstants.INCREMENTAL_PUSH_VERSION; import static com.linkedin.venice.controllerapi.ControllerApiConstants.NAME; -import static com.linkedin.venice.controllerapi.ControllerApiConstants.PUSH_JOB_DETAILS; import static com.linkedin.venice.controllerapi.ControllerApiConstants.TARGETED_REGIONS; import static com.linkedin.venice.controllerapi.ControllerApiConstants.TOPIC; import static com.linkedin.venice.controllerapi.ControllerApiConstants.VERSION; @@ -13,7 +12,6 @@ import static com.linkedin.venice.controllerapi.ControllerRoute.KILL_OFFLINE_PUSH_JOB; import static com.linkedin.venice.controllerapi.ControllerRoute.SEND_PUSH_JOB_DETAILS; -import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; import com.linkedin.venice.HttpConstants; import com.linkedin.venice.acl.DynamicAccessController; import com.linkedin.venice.controller.Admin; @@ -30,8 +28,6 @@ import com.linkedin.venice.utils.Utils; import java.util.Collections; import java.util.Optional; -import org.apache.avro.io.DatumReader; -import org.apache.avro.specific.SpecificDatumReader; import org.apache.http.HttpStatus; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -165,18 +161,7 @@ public Route sendPushJobDetails(Admin admin) { PushJobStatusRecordKey key = new PushJobStatusRecordKey(); key.storeName = storeName; key.versionNumber = versionNumber; - PushJobDetails pushJobDetails; - // TODO remove passing PushJobDetails as JSON string once all VPJ plugins are updated. - if (request.queryParams().contains(PUSH_JOB_DETAILS)) { - String pushJobDetailsString = request.queryParams(PUSH_JOB_DETAILS); - DatumReader reader = - new SpecificDatumReader<>(PushJobDetails.getClassSchema(), PushJobDetails.getClassSchema()); - pushJobDetails = reader.read( - null, - AvroCompatibilityHelper.newCompatibleJsonDecoder(PushJobDetails.getClassSchema(), pushJobDetailsString)); - } else { - pushJobDetails = pushJobDetailsSerializer.deserialize(null, request.bodyAsBytes()); - } + PushJobDetails pushJobDetails = pushJobDetailsSerializer.deserialize(null, request.bodyAsBytes()); admin.sendPushJobDetails(key, pushJobDetails); if (pushJobDetails.sendLivenessHeartbeatFailureDetails != null) { diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/stats/PushJobStatusStats.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/stats/PushJobStatusStats.java new file mode 100644 index 00000000000..3013e4c987f --- /dev/null +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/stats/PushJobStatusStats.java @@ -0,0 +1,60 @@ +package com.linkedin.venice.controller.stats; + +import com.linkedin.venice.stats.AbstractVeniceStats; +import io.tehuti.metrics.MetricsRepository; +import io.tehuti.metrics.Sensor; +import io.tehuti.metrics.stats.Count; +import io.tehuti.metrics.stats.CountSinceLastMeasurement; + + +public class PushJobStatusStats extends AbstractVeniceStats { + private final Sensor batchPushSuccessSensor; + private final Sensor batchPushFailureDueToUserErrorSensor; + private final Sensor batchPushFailureDueToNonUserErrorSensor; + private final Sensor incrementalPushSuccessSensor; + private final Sensor incrementalPushFailureDueToUserErrorSensor; + private final Sensor incrementalPushFailureDueToNonUserErrorSensor; + + public PushJobStatusStats(MetricsRepository metricsRepository, String name) { + super(metricsRepository, name); + batchPushSuccessSensor = + registerSensorIfAbsent("batch_push_job_success", new Count(), new CountSinceLastMeasurement()); + batchPushFailureDueToUserErrorSensor = + registerSensorIfAbsent("batch_push_job_failed_user_error", new Count(), new CountSinceLastMeasurement()); + batchPushFailureDueToNonUserErrorSensor = + registerSensorIfAbsent("batch_push_job_failed_non_user_error", new Count(), new CountSinceLastMeasurement()); + incrementalPushSuccessSensor = + registerSensorIfAbsent("incremental_push_job_success", new Count(), new CountSinceLastMeasurement()); + incrementalPushFailureDueToUserErrorSensor = + registerSensorIfAbsent("incremental_push_job_failed_user_error", new Count(), new CountSinceLastMeasurement()); + incrementalPushFailureDueToNonUserErrorSensor = registerSensorIfAbsent( + "incremental_push_job_failed_non_user_error", + new Count(), + new CountSinceLastMeasurement()); + } + + // record all metrics + public void recordBatchPushSuccessSensor() { + batchPushSuccessSensor.record(); + } + + public void recordBatchPushFailureDueToUserErrorSensor() { + batchPushFailureDueToUserErrorSensor.record(); + } + + public void recordBatchPushFailureNotDueToUserErrorSensor() { + batchPushFailureDueToNonUserErrorSensor.record(); + } + + public void recordIncrementalPushSuccessSensor() { + incrementalPushSuccessSensor.record(); + } + + public void recordIncrementalPushFailureDueToUserErrorSensor() { + incrementalPushFailureDueToUserErrorSensor.record(); + } + + public void recordIncrementalPushFailureNotDueToUserErrorSensor() { + incrementalPushFailureDueToNonUserErrorSensor.record(); + } +} diff --git a/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestPushJobStatusStats.java b/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestPushJobStatusStats.java new file mode 100644 index 00000000000..51f63d7dd4e --- /dev/null +++ b/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestPushJobStatusStats.java @@ -0,0 +1,127 @@ +package com.linkedin.venice.controller; + +import static com.linkedin.venice.controller.VeniceHelixAdmin.emitPushJobDetailsMetrics; +import static com.linkedin.venice.controller.VeniceHelixAdmin.isPushJobFailedUserError; +import static com.linkedin.venice.status.PushJobDetailsStatus.isFailed; +import static com.linkedin.venice.status.PushJobDetailsStatus.isSucceeded; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; + +import com.linkedin.venice.PushJobCheckpoints; +import com.linkedin.venice.controller.stats.PushJobStatusStats; +import com.linkedin.venice.status.PushJobDetailsStatus; +import com.linkedin.venice.status.protocol.PushJobDetails; +import com.linkedin.venice.status.protocol.PushJobDetailsStatusTuple; +import com.linkedin.venice.utils.DataProviderUtils; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.avro.util.Utf8; +import org.testng.annotations.Test; + + +public class TestPushJobStatusStats { + @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) + public void testEmitPushJobDetailsMetrics(boolean isIncrementalPush) { + PushJobDetails details = mock(PushJobDetails.class); + Map pushJobConfigs = new HashMap<>(); + pushJobConfigs.put(new Utf8("incremental.push"), isIncrementalPush ? "true" : "false"); + when(details.getPushJobConfigs()).thenReturn(pushJobConfigs); + + when(details.getClusterName()).thenReturn(new Utf8("cluster1")); + List statusTuples = new ArrayList<>(); + when(details.getOverallStatus()).thenReturn(statusTuples); + + Map pushJobStatusStatsMap = new HashMap<>(); + PushJobStatusStats stats = mock(PushJobStatusStats.class); + pushJobStatusStatsMap.put("cluster1", stats); + + int numberSuccess = 0; + int numberUserErrors = 0; + int numberNonUserErrors = 0; + + for (PushJobDetailsStatus status: PushJobDetailsStatus.values()) { + boolean recordMetrics = false; + if (isSucceeded(status) || isFailed(status)) { + recordMetrics = true; + } + + statusTuples.add(new PushJobDetailsStatusTuple(status.getValue(), 0L)); + + for (PushJobCheckpoints checkpoint: PushJobCheckpoints.values()) { + when(details.getPushJobLatestCheckpoint()).thenReturn(checkpoint.getValue()); + emitPushJobDetailsMetrics(pushJobStatusStatsMap, details); + switch (checkpoint) { + case QUOTA_EXCEEDED: + case WRITE_ACL_FAILED: + case DUP_KEY_WITH_DIFF_VALUE: + case INPUT_DATA_SCHEMA_VALIDATION_FAILED: + case EXTENDED_INPUT_DATA_SCHEMA_VALIDATION_FAILED: + case RECORD_TOO_LARGE_FAILED: + case CONCURRENT_BATCH_PUSH: + case DATASET_CHANGED: + case INVALID_INPUT_FILE: + case DVC_INGESTION_ERROR_DISK_FULL: + case DVC_INGESTION_ERROR_MEMORY_LIMIT_REACHED: + if (recordMetrics) { + if (isFailed(status)) { + assertTrue(isPushJobFailedUserError(status, checkpoint)); + numberUserErrors++; + if (isIncrementalPush) { + verify(stats, times(numberUserErrors)).recordIncrementalPushFailureDueToUserErrorSensor(); + } else { + verify(stats, times(numberUserErrors)).recordBatchPushFailureDueToUserErrorSensor(); + } + } else { + numberSuccess++; + if (isIncrementalPush) { + verify(stats, times(numberSuccess)).recordIncrementalPushSuccessSensor(); + } else { + verify(stats, times(numberSuccess)).recordBatchPushSuccessSensor(); + } + } + } + break; + + case INITIALIZE_PUSH_JOB: + case NEW_VERSION_CREATED: + case START_DATA_WRITER_JOB: + case DATA_WRITER_JOB_COMPLETED: + case START_JOB_STATUS_POLLING: + case JOB_STATUS_POLLING_COMPLETED: + case START_VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB: + case VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB_COMPLETED: + case ZSTD_DICTIONARY_CREATION_FAILED: + case DVC_INGESTION_ERROR_TOO_MANY_DEAD_INSTANCES: + case DVC_INGESTION_ERROR_OTHER: + if (recordMetrics) { + assertFalse(isPushJobFailedUserError(status, checkpoint)); + if (isFailed(status)) { + numberNonUserErrors++; + if (isIncrementalPush) { + verify(stats, times(numberNonUserErrors)).recordIncrementalPushFailureNotDueToUserErrorSensor(); + } else { + verify(stats, times(numberNonUserErrors)).recordBatchPushFailureNotDueToUserErrorSensor(); + } + } else { + numberSuccess++; + if (isIncrementalPush) { + verify(stats, times(numberSuccess)).recordIncrementalPushSuccessSensor(); + } else { + verify(stats, times(numberSuccess)).recordBatchPushSuccessSensor(); + } + } + } + break; + default: + throw new IllegalArgumentException("Unknown checkpoint: " + checkpoint); + } + } + } + } +} diff --git a/services/venice-router/src/main/java/com/linkedin/venice/router/MetaDataHandler.java b/services/venice-router/src/main/java/com/linkedin/venice/router/MetaDataHandler.java index 55a34650251..4c52b1e821d 100644 --- a/services/venice-router/src/main/java/com/linkedin/venice/router/MetaDataHandler.java +++ b/services/venice-router/src/main/java/com/linkedin/venice/router/MetaDataHandler.java @@ -311,11 +311,19 @@ private void handleValueSchemaLookup(ChannelHandlerContext ctx, VenicePathParser responseObject.setSuperSetSchemaId(superSetSchemaId); } Collection valueSchemaEntries = schemaRepo.getValueSchemas(storeName); - int schemaNum = valueSchemaEntries.size(); + int schemaNum = (int) valueSchemaEntries.stream().filter(schemaEntry -> schemaEntry.getId() > 0).count(); MultiSchemaResponse.Schema[] schemas = new MultiSchemaResponse.Schema[schemaNum]; int index = 0; for (SchemaEntry entry: valueSchemaEntries) { int schemaId = entry.getId(); + if (schemaId < 1) { + LOGGER.warn( + "Got an invalid schema id ({}) for store {} in handleValueSchemaLookup; will not include this in the {}.", + entry.getId(), + storeName, + responseObject.getClass().getSimpleName()); + continue; + } schemas[index] = new MultiSchemaResponse.Schema(); schemas[index].setId(schemaId); schemas[index].setSchemaStr(entry.getSchema().toString()); @@ -385,6 +393,14 @@ private void handleValueSchemaIdsLookup(ChannelHandlerContext ctx, VenicePathPar } Set schemaIdSet = new HashSet<>(); for (SchemaEntry entry: schemaRepo.getValueSchemas(storeName)) { + if (entry.getId() < 1) { + LOGGER.warn( + "Got an invalid schema id ({}) for store {} in handleValueSchemaIdsLookup; will not include this in the {}.", + entry.getId(), + storeName, + responseObject.getClass().getSimpleName()); + continue; + } schemaIdSet.add(entry.getId()); } responseObject.setSchemaIdSet(schemaIdSet); diff --git a/services/venice-router/src/main/java/com/linkedin/venice/router/RouterServer.java b/services/venice-router/src/main/java/com/linkedin/venice/router/RouterServer.java index e3d21e7966b..12a7cdfc3ec 100644 --- a/services/venice-router/src/main/java/com/linkedin/venice/router/RouterServer.java +++ b/services/venice-router/src/main/java/com/linkedin/venice/router/RouterServer.java @@ -17,7 +17,6 @@ import com.linkedin.d2.balancer.D2Client; import com.linkedin.venice.ConfigKeys; import com.linkedin.venice.acl.DynamicAccessController; -import com.linkedin.venice.acl.handler.StoreAclHandler; import com.linkedin.venice.authorization.IdentityParser; import com.linkedin.venice.compression.CompressorFactory; import com.linkedin.venice.exceptions.VeniceException; @@ -40,6 +39,7 @@ import com.linkedin.venice.meta.ReadOnlyStoreRepository; import com.linkedin.venice.pushstatushelper.PushStatusStoreReader; import com.linkedin.venice.read.RequestType; +import com.linkedin.venice.router.acl.RouterStoreAclHandler; import com.linkedin.venice.router.api.DictionaryRetrievalService; import com.linkedin.venice.router.api.MetaStoreShadowReader; import com.linkedin.venice.router.api.RouterExceptionAndTrackingUtils; @@ -686,8 +686,9 @@ public AsyncFuture getLongTailRetryMilliseconds( } RouterSslVerificationHandler routerSslVerificationHandler = new RouterSslVerificationHandler(securityStats); - StoreAclHandler aclHandler = - accessController.isPresent() ? new StoreAclHandler(accessController.get(), metadataRepository) : null; + RouterStoreAclHandler aclHandler = accessController.isPresent() + ? new RouterStoreAclHandler(identityParser, accessController.get(), metadataRepository) + : null; final SslInitializer sslInitializer; if (sslFactory.isPresent()) { sslInitializer = new SslInitializer(SslUtils.toAlpiniSSLFactory(sslFactory.get()), false); @@ -748,7 +749,7 @@ public AsyncFuture getLongTailRetryMilliseconds( pipeline.addLast("VerifySslHandler", routerSslVerificationHandler); pipeline.addLast("MetadataHandler", metaDataHandler); pipeline.addLast("AdminOperationsHandler", adminOperationsHandler); - pipeline.addLast("StoreAclHandler", aclHandler); + pipeline.addLast("RouterStoreAclHandler", aclHandler); pipeline.addLast("RouterThrottleHandler", routerThrottleHandler); addStreamingHandler(pipeline); addOptionalChannelHandlersToPipeline(pipeline); diff --git a/services/venice-router/src/main/java/com/linkedin/venice/router/acl/RouterStoreAclHandler.java b/services/venice-router/src/main/java/com/linkedin/venice/router/acl/RouterStoreAclHandler.java new file mode 100644 index 00000000000..ea51ee9dd80 --- /dev/null +++ b/services/venice-router/src/main/java/com/linkedin/venice/router/acl/RouterStoreAclHandler.java @@ -0,0 +1,68 @@ +package com.linkedin.venice.router.acl; + +import static com.linkedin.venice.router.api.RouterResourceType.TYPE_COMPUTE; +import static com.linkedin.venice.router.api.RouterResourceType.TYPE_INVALID; +import static com.linkedin.venice.router.api.RouterResourceType.TYPE_STORAGE; + +import com.linkedin.venice.acl.AclCreationDeletionListener; +import com.linkedin.venice.acl.DynamicAccessController; +import com.linkedin.venice.acl.handler.AbstractStoreAclHandler; +import com.linkedin.venice.authorization.IdentityParser; +import com.linkedin.venice.exceptions.VeniceUnsupportedOperationException; +import com.linkedin.venice.meta.ReadOnlyStoreRepository; +import com.linkedin.venice.router.api.RouterResourceType; +import io.netty.channel.ChannelHandler; + + +/** + * Store-level access control handler, which is being used by both Router and Server. + */ +@ChannelHandler.Sharable +public class RouterStoreAclHandler extends AbstractStoreAclHandler { + public RouterStoreAclHandler( + IdentityParser identityParser, + DynamicAccessController accessController, + ReadOnlyStoreRepository metadataRepository) { + super(identityParser, accessController, metadataRepository); + metadataRepository.registerStoreDataChangedListener(new AclCreationDeletionListener(accessController)); + } + + @Override + protected boolean needsAclValidation(RouterResourceType resourceType) { + if (resourceType == TYPE_STORAGE || resourceType == TYPE_COMPUTE) { + return true; + } + + if (resourceType == TYPE_INVALID) { + throw new VeniceUnsupportedOperationException(resourceType.name()); + } + + return false; + } + + /** + * Extract the store name from the incoming resource name. + */ + @Override + protected String extractStoreName(RouterResourceType resourceType, String[] requestParts) { + // In Routers, all requests that go through ACL checks have the 2nd part as the store name + return requestParts[2]; + } + + @Override + protected RouterResourceType validateRequest(String[] requestParts) { + int partsLength = requestParts.length; + + if (partsLength < 3) { + // In routers, all requests have at least the request type and store name + return null; + } else { // throw exception to retain current behavior for invalid query actions + String requestType = requestParts[1].toLowerCase(); + RouterResourceType resourceType = RouterResourceType.getTypeResourceType(requestType); + if (resourceType == TYPE_INVALID) { + return null; + } + return resourceType; + } + } +} diff --git a/services/venice-router/src/test/java/com/linkedin/venice/router/TestMetaDataHandler.java b/services/venice-router/src/test/java/com/linkedin/venice/router/TestMetaDataHandler.java index fb3276826b3..23cbf5c173e 100644 --- a/services/venice-router/src/test/java/com/linkedin/venice/router/TestMetaDataHandler.java +++ b/services/venice-router/src/test/java/com/linkedin/venice/router/TestMetaDataHandler.java @@ -348,7 +348,10 @@ public void testAllValueSchemaIdLookup() throws IOException { ReadOnlySchemaRepository schemaRepo = Mockito.mock(ReadOnlySchemaRepository.class); SchemaEntry valueSchemaEntry1 = new SchemaEntry(valueSchemaId1, valueSchemaStr1); SchemaEntry valueSchemaEntry2 = new SchemaEntry(valueSchemaId2, valueSchemaStr2); - Mockito.doReturn(Arrays.asList(valueSchemaEntry1, valueSchemaEntry2)).when(schemaRepo).getValueSchemas(storeName); + SchemaEntry valueSchemaEntry3 = new SchemaEntry(-1, valueSchemaStr2); + Mockito.doReturn(Arrays.asList(valueSchemaEntry1, valueSchemaEntry2, valueSchemaEntry3)) + .when(schemaRepo) + .getValueSchemas(storeName); FullHttpResponse response = passRequestToMetadataHandler( "http://myRouterHost:4567/all_value_schema_ids/" + storeName, null, @@ -412,7 +415,8 @@ public void testAllValueSchemaLookup() throws IOException { SchemaEntry valueSchemaEntry1 = new SchemaEntry(valueSchemaId1, valueSchemaStr1); SchemaEntry valueSchemaEntry2 = new SchemaEntry(valueSchemaId2, valueSchemaStr2); SchemaEntry valueSchemaEntry3 = new SchemaEntry(valueSchemaId3, valueSchemaStr3); - Mockito.doReturn(Arrays.asList(valueSchemaEntry1, valueSchemaEntry2, valueSchemaEntry3)) + SchemaEntry valueSchemaEntry4 = new SchemaEntry(-1, valueSchemaStr3); + Mockito.doReturn(Arrays.asList(valueSchemaEntry1, valueSchemaEntry2, valueSchemaEntry3, valueSchemaEntry4)) .when(schemaRepo) .getValueSchemas(storeName); diff --git a/services/venice-router/src/test/java/com/linkedin/venice/router/acl/RouterStoreAclHandlerTest.java b/services/venice-router/src/test/java/com/linkedin/venice/router/acl/RouterStoreAclHandlerTest.java new file mode 100644 index 00000000000..f6328519ace --- /dev/null +++ b/services/venice-router/src/test/java/com/linkedin/venice/router/acl/RouterStoreAclHandlerTest.java @@ -0,0 +1,409 @@ +package com.linkedin.venice.router.acl; + +import static com.linkedin.venice.router.api.VenicePathParser.TASK_READ_QUOTA_THROTTLE; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.argThat; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.clearInvocations; +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.assertFalse; +import static org.testng.Assert.assertNotNull; + +import com.linkedin.venice.HttpConstants; +import com.linkedin.venice.acl.AclException; +import com.linkedin.venice.acl.DynamicAccessController; +import com.linkedin.venice.acl.handler.AbstractStoreAclHandler; +import com.linkedin.venice.authorization.IdentityParser; +import com.linkedin.venice.helix.HelixReadOnlyStoreConfigRepository; +import com.linkedin.venice.helix.HelixReadOnlyStoreRepository; +import com.linkedin.venice.meta.ReadOnlyStoreRepository; +import com.linkedin.venice.meta.Store; +import com.linkedin.venice.meta.StoreConfig; +import com.linkedin.venice.meta.Version; +import com.linkedin.venice.router.api.RouterResourceType; +import io.netty.channel.Channel; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelPipeline; +import io.netty.handler.codec.http.FullHttpResponse; +import io.netty.handler.codec.http.HttpHeaders; +import io.netty.handler.codec.http.HttpMethod; +import io.netty.handler.codec.http.HttpRequest; +import io.netty.handler.codec.http.HttpResponseStatus; +import io.netty.handler.ssl.SslHandler; +import java.net.SocketAddress; +import java.security.cert.Certificate; +import java.security.cert.X509Certificate; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import javax.net.ssl.SSLEngine; +import javax.net.ssl.SSLPeerUnverifiedException; +import javax.net.ssl.SSLSession; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.mockito.ArgumentMatcher; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + + +public class RouterStoreAclHandlerTest { + private static final Logger LOGGER = LogManager.getLogger(RouterStoreAclHandlerTest.class); + private IdentityParser identityParser; + private DynamicAccessController accessController; + private HelixReadOnlyStoreRepository metadataRepo; + private ChannelHandlerContext ctx; + private HttpRequest req; + private String storeName; + private Store store; + private String clusterName; + private boolean[] hasStore = { false }; + private boolean[] isBadUri = { false }; + + private void resetAllConditions() { + hasStore[0] = false; + isBadUri[0] = false; + } + + @BeforeMethod + public void setUp() throws Exception { + clusterName = "testCluster"; + storeName = "testStore"; + identityParser = mock(IdentityParser.class); + accessController = mock(DynamicAccessController.class); + ctx = mock(ChannelHandlerContext.class); + req = mock(HttpRequest.class); + store = mock(Store.class); + + when(accessController.init(any())).thenReturn(accessController); + + // Certificate + ChannelPipeline pipe = mock(ChannelPipeline.class); + when(ctx.pipeline()).thenReturn(pipe); + SslHandler sslHandler = mock(SslHandler.class); + when(pipe.get(SslHandler.class)).thenReturn(sslHandler); + SSLEngine sslEngine = mock(SSLEngine.class); + when(sslHandler.engine()).thenReturn(sslEngine); + SSLSession sslSession = mock(SSLSession.class); + when(sslEngine.getSession()).thenReturn(sslSession); + X509Certificate cert = mock(X509Certificate.class); + when(sslSession.getPeerCertificates()).thenReturn(new Certificate[] { cert }); + + // Host + Channel channel = mock(Channel.class); + when(ctx.channel()).thenReturn(channel); + SocketAddress address = mock(SocketAddress.class); + when(channel.remoteAddress()).thenReturn(address); + + when(req.method()).thenReturn(HttpMethod.GET); + + HttpHeaders headers = mock(HttpHeaders.class); + when(headers.contains(HttpConstants.VENICE_ALLOW_REDIRECT)).thenReturn(true); + + when(req.headers()).thenReturn(headers); + } + + @Test + public void storeMissing() throws Exception { + hasStore[0] = false; + enumerate(); + + verify(ctx, times(1)).fireChannelRead(req); + verify(ctx, never()).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.FORBIDDEN))); + verify(ctx, never()).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.UNAUTHORIZED))); + verify(ctx, never()).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.BAD_REQUEST))); + } + + @Test + public void isBadUri() throws Exception { + isBadUri[0] = true; + enumerate(hasStore); + + // should fail every time for BAD_REQUEST + verify(ctx, times(2)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.BAD_REQUEST))); + } + + @Test + public void testAllCases() throws Exception { + enumerate(hasStore, isBadUri); + + // !isBadUri && hasStore = 1 times + // !isBadUri && !hasStore = 1 times + verify(ctx, times(2)).fireChannelRead(req); + + // isBadUri = 2 times + verify(ctx, times(2)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.BAD_REQUEST))); + + verify(ctx, never()).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.FORBIDDEN))); + verify(ctx, never()).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.UNAUTHORIZED))); + } + + private void update() throws Exception { + when(accessController.hasAccess(any(), any(), any())).thenReturn(true); + when(accessController.hasAcl(any())).thenReturn(true); + when(accessController.isFailOpen()).thenReturn(true); + when(metadataRepo.hasStore(any())).thenReturn(hasStore[0]); + if (hasStore[0]) { + when(metadataRepo.getStore(any())).thenReturn(store); + } else { + when(metadataRepo.getStore(any())).thenReturn(null); + } + + String storeNameInRequest = storeName; + when(store.isSystemStore()).thenReturn(false); + if (isBadUri[0]) { + when(req.uri()).thenReturn("/badUri"); + } else { + when(req.uri()).thenReturn(String.format("/storage/%s/random", storeNameInRequest)); + } + } + + /** + * Generate every possible combination for a given list of booleans based on variables passed + * to boolean[]... conditions. If all variables (8 in count) are passed, then there will be 256 + * combinations: + * + * for (int i = 0; i < 256; i++) { | i= 0 1 2 3 4 ... + * _hasAccess= (i>>0) % 2 == 1| F T F T F ... + * _hasAcl= (i>>1) % 2 == 1| F F T T F ... + * _hasStore= (i>>2) % 2 == 1| F F F F T ... + * _isAccessControlled= (i>>3) % 2 == 1| F F F F F ... + * _isFailOpen= (i>>4) % 2 == 1| F F F F F ... + * _isMetadata= (i>>5) % 2 == 1| F F F F F ... + * _isHealthCheck= (i>>6) % 2 == 1| F F F F F ... + * _isBadUri= (i>>7) % 2 == 1| F F F F F ... + * } + */ + private void enumerate(boolean[]... conditions) throws Exception { + // enumerate for all possible combinations + int len = conditions.length; + for (int i = 0; i < Math.pow(2, len); i++) { + for (int j = 0; j < len; j++) { + conditions[j][0] = ((i >> j) & 1) == 1; + } + // New metadataRepo mock and aclHandler every update since thenThrow cannot be re-mocked. + metadataRepo = mock(HelixReadOnlyStoreRepository.class); + AbstractStoreAclHandler aclHandler = + spy(new RouterStoreAclHandler(identityParser, accessController, metadataRepo)); + update(); + LOGGER.info("hasStore: {}, isBadUri: {}", hasStore[0], isBadUri[0]); + aclHandler.channelRead0(ctx, req); + } + + // reset all supported conditions to the default to remove changes from this test + resetAllConditions(); + } + + private static class ContextMatcher implements ArgumentMatcher { + private HttpResponseStatus status; + + public ContextMatcher(HttpResponseStatus status) { + this.status = status; + } + + @Override + public boolean matches(FullHttpResponse argument) { + return argument.status().equals(status); + } + } + + /** + * Mock access controller to verify basic request parsing and handling for {@link RouterStoreAclHandler} + */ + private static class MockAccessController implements DynamicAccessController { + private RouterResourceType resourceType; + + public MockAccessController(RouterResourceType resourceType) { + this.resourceType = resourceType; + } + + @Override + public boolean hasAccessToTopic(X509Certificate clientCert, String resource, String method) throws AclException { + assertNotNull(clientCert, resourceType.toString()); + validateStringArg(resource, "resource"); + validateStringArg(method, "method"); + return true; + } + + @Override + public boolean hasAccessToAdminOperation(X509Certificate clientCert, String operation) throws AclException { + assertNotNull(clientCert, resourceType.toString()); + validateStringArg(operation, "operation"); + return true; + } + + @Override + public boolean isAllowlistUsers(X509Certificate clientCert, String resource, String method) { + assertNotNull(clientCert, resourceType.toString()); + validateStringArg(resource, "resource"); + validateStringArg(method, "method"); + return true; + } + + @Override + public String getPrincipalId(X509Certificate clientCert) { + assertNotNull(clientCert, resourceType.toString()); + return "testPrincipalId"; + } + + @Override + public DynamicAccessController init(List resources) { + return this; + } + + @Override + public boolean hasAccess(X509Certificate clientCert, String resource, String method) throws AclException { + assertNotNull(clientCert); + validateStringArg(resource, "resource"); + validateStringArg(method, "method"); + return true; + } + + @Override + public boolean hasAcl(String resource) throws AclException { + validateStringArg(resource, "resource"); + return true; + } + + @Override + public void addAcl(String resource) throws AclException { + validateStringArg(resource, "resource"); + } + + @Override + public void removeAcl(String resource) throws AclException { + validateStringArg(resource, "resource"); + } + + @Override + public Set getAccessControlledResources() { + return null; + } + + @Override + public boolean isFailOpen() { + return false; + } + + private void validateStringArg(String arg, String argName) { + assertNotNull(arg, argName + " should not be null for resource type " + resourceType.toString()); + assertFalse(arg.isEmpty(), argName + " should not be empty string for resource type " + resourceType.toString()); + } + } + + @Test + public void testAllRequestTypes() throws SSLPeerUnverifiedException, AclException { + Store store = mock(Store.class); + ReadOnlyStoreRepository metadataRepo = mock(ReadOnlyStoreRepository.class); + when(metadataRepo.getStore(storeName)).thenReturn(store); + HelixReadOnlyStoreConfigRepository storeConfigRepository = mock(HelixReadOnlyStoreConfigRepository.class); + StoreConfig storeConfig = mock(StoreConfig.class); + when(storeConfig.getCluster()).thenReturn(clusterName); + when(storeConfigRepository.getStoreConfig(storeName)).thenReturn(Optional.of(storeConfig)); + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + HttpRequest request = mock(HttpRequest.class); + Channel channel = mock(Channel.class); + SocketAddress socketAddress = mock(SocketAddress.class); + doReturn("testRemoteHost").when(socketAddress).toString(); + doReturn(socketAddress).when(channel).remoteAddress(); + doReturn(channel).when(ctx).channel(); + SslHandler sslHandler = mock(SslHandler.class); + ChannelPipeline channelPipeline = mock(ChannelPipeline.class); + doReturn(sslHandler).when(channelPipeline).get(SslHandler.class); + SSLEngine sslEngine = mock(SSLEngine.class); + SSLSession sslSession = mock(SSLSession.class); + X509Certificate certificate = mock(X509Certificate.class); + Certificate[] certificates = new Certificate[1]; + certificates[0] = certificate; + doReturn(certificates).when(sslSession).getPeerCertificates(); + doReturn(sslSession).when(sslEngine).getSession(); + doReturn(sslEngine).when(sslHandler).engine(); + doReturn(channelPipeline).when(ctx).pipeline(); + doReturn(HttpMethod.GET).when(request).method(); + IdentityParser identityParser = mock(IdentityParser.class); + doReturn("testPrincipalId").when(identityParser).parseIdentityFromCert(certificate); + for (RouterResourceType resourceType: RouterResourceType.values()) { + clearInvocations(ctx); + MockAccessController mockAccessController = new MockAccessController(resourceType); + MockAccessController spyMockAccessController = spy(mockAccessController); + RouterStoreAclHandler storeAclHandler = + new RouterStoreAclHandler(identityParser, spyMockAccessController, metadataRepo); + doReturn(buildTestURI(resourceType)).when(request).uri(); + storeAclHandler.channelRead0(ctx, request); + + LOGGER.info("Testing {} resource type", resourceType); + switch (resourceType) { + case TYPE_LEADER_CONTROLLER: + case TYPE_LEADER_CONTROLLER_LEGACY: + case TYPE_KEY_SCHEMA: + case TYPE_VALUE_SCHEMA: + case TYPE_LATEST_VALUE_SCHEMA: + case TYPE_GET_UPDATE_SCHEMA: + case TYPE_ALL_VALUE_SCHEMA_IDS: + case TYPE_CLUSTER_DISCOVERY: + case TYPE_STREAM_HYBRID_STORE_QUOTA: + case TYPE_STREAM_REPROCESSING_HYBRID_STORE_QUOTA: + case TYPE_STORE_STATE: + case TYPE_PUSH_STATUS: + case TYPE_ADMIN: + case TYPE_RESOURCE_STATE: + case TYPE_CURRENT_VERSION: + case TYPE_BLOB_DISCOVERY: + case TYPE_REQUEST_TOPIC: + verify(spyMockAccessController, never()).hasAccess(any(), any(), any()); + break; + case TYPE_STORAGE: + case TYPE_COMPUTE: + verify(spyMockAccessController).hasAccess(any(), eq(storeName), any()); + break; + case TYPE_INVALID: + verify(spyMockAccessController, never()).hasAccess(any(), any(), any()); + verify(ctx, times(1)).writeAndFlush(argThat(new ContextMatcher(HttpResponseStatus.BAD_REQUEST))); + break; + default: + throw new IllegalArgumentException("Invalid resource type: " + resourceType); + } + } + } + + private String buildTestURI(RouterResourceType resourceType) { + switch (resourceType) { + case TYPE_LEADER_CONTROLLER: + case TYPE_LEADER_CONTROLLER_LEGACY: + case TYPE_RESOURCE_STATE: + return "/" + resourceType.toString().toLowerCase(); + case TYPE_KEY_SCHEMA: + case TYPE_VALUE_SCHEMA: + case TYPE_LATEST_VALUE_SCHEMA: + case TYPE_GET_UPDATE_SCHEMA: + case TYPE_ALL_VALUE_SCHEMA_IDS: + case TYPE_STORE_STATE: + case TYPE_CLUSTER_DISCOVERY: + case TYPE_STREAM_HYBRID_STORE_QUOTA: + case TYPE_CURRENT_VERSION: + case TYPE_REQUEST_TOPIC: + return "/" + resourceType.toString().toLowerCase() + "/" + storeName; + case TYPE_STREAM_REPROCESSING_HYBRID_STORE_QUOTA: + case TYPE_PUSH_STATUS: + String topicName = Version.composeKafkaTopic(storeName, 1); + return "/" + resourceType.toString().toLowerCase() + "/" + topicName; + case TYPE_ADMIN: + return "/" + resourceType.toString().toLowerCase() + "/" + TASK_READ_QUOTA_THROTTLE; + case TYPE_STORAGE: + case TYPE_COMPUTE: + return "/" + resourceType.toString().toLowerCase() + "/" + storeName + "/ABCDEFG"; + case TYPE_BLOB_DISCOVERY: + return "/" + resourceType.toString().toLowerCase() + "?store=" + storeName + + "&store_version=1&store_partition=2"; + case TYPE_INVALID: + return "/invalid"; + default: + throw new IllegalArgumentException("Invalid resource type: " + resourceType); + } + } +} diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/HttpChannelInitializer.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/HttpChannelInitializer.java index f118c58faff..50aa4342645 100644 --- a/services/venice-server/src/main/java/com/linkedin/venice/listener/HttpChannelInitializer.java +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/HttpChannelInitializer.java @@ -110,8 +110,12 @@ public HttpChannelInitializer( this.sslFactory = sslFactory; this.sslHandshakeExecutor = sslHandshakeExecutor; + + Class identityParserClass = ReflectUtils.loadClass(serverConfig.getIdentityParserClassName()); + this.identityParser = ReflectUtils.callConstructor(identityParserClass, new Class[0], new Object[0]); + this.storeAclHandler = storeAccessController.isPresent() - ? Optional.of(new ServerStoreAclHandler(storeAccessController.get(), storeMetadataRepository)) + ? Optional.of(new ServerStoreAclHandler(identityParser, storeAccessController.get(), storeMetadataRepository)) : Optional.empty(); /** * If the store-level access handler is present, we don't want to fail fast if the access gets denied by {@link ServerAclHandler}. @@ -146,9 +150,6 @@ public HttpChannelInitializer( this.http2PipelineInitializerBuilder = new VeniceHttp2PipelineInitializerBuilder(serverConfig); serverConnectionStats = new ServerConnectionStats(metricsRepository, "server_connection_stats"); - - Class identityParserClass = ReflectUtils.loadClass(serverConfig.getIdentityParserClassName()); - this.identityParser = ReflectUtils.callConstructor(identityParserClass, new Class[0], new Object[0]); } /* diff --git a/services/venice-server/src/main/java/com/linkedin/venice/listener/ServerStoreAclHandler.java b/services/venice-server/src/main/java/com/linkedin/venice/listener/ServerStoreAclHandler.java index f482b1219db..b5344aa264c 100644 --- a/services/venice-server/src/main/java/com/linkedin/venice/listener/ServerStoreAclHandler.java +++ b/services/venice-server/src/main/java/com/linkedin/venice/listener/ServerStoreAclHandler.java @@ -1,18 +1,32 @@ package com.linkedin.venice.listener; +import static com.linkedin.venice.grpc.GrpcUtils.accessResultToGrpcStatus; +import static com.linkedin.venice.grpc.GrpcUtils.extractGrpcClientCert; + import com.linkedin.venice.acl.DynamicAccessController; -import com.linkedin.venice.acl.handler.StoreAclHandler; +import com.linkedin.venice.acl.handler.AbstractStoreAclHandler; +import com.linkedin.venice.acl.handler.AccessResult; +import com.linkedin.venice.authorization.IdentityParser; +import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.meta.QueryAction; import com.linkedin.venice.meta.ReadOnlyStoreRepository; import com.linkedin.venice.meta.Version; +import com.linkedin.venice.protocols.VeniceClientRequest; +import io.grpc.ForwardingServerCallListener; import io.grpc.Metadata; import io.grpc.ServerCall; import io.grpc.ServerCallHandler; +import io.grpc.ServerInterceptor; +import io.grpc.Status; import io.netty.channel.ChannelHandlerContext; -import io.netty.handler.codec.http.HttpRequest; import io.netty.util.Attribute; -import io.netty.util.ReferenceCountUtil; +import java.security.cert.X509Certificate; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; +import java.util.function.Consumer; import javax.net.ssl.SSLPeerUnverifiedException; +import org.apache.commons.lang.StringUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -24,24 +38,36 @@ * validate the request in store-level, which is exactly same as the access control behavior in Router. * If both of them fail, the request will be rejected. */ -public class ServerStoreAclHandler extends StoreAclHandler { +public class ServerStoreAclHandler extends AbstractStoreAclHandler implements ServerInterceptor { private final static Logger LOGGER = LogManager.getLogger(ServerStoreAclHandler.class); - public ServerStoreAclHandler(DynamicAccessController accessController, ReadOnlyStoreRepository metadataRepository) { - super(accessController, metadataRepository); + /** + * Skip ACL for requests to /metadata, /admin, /current_version, /health and /topic_partition_ingestion_context + * as there's no sensitive information in the response. + */ + private static final Set QUERIES_TO_SKIP_ACL = new HashSet<>( + Arrays.asList( + QueryAction.METADATA, + QueryAction.ADMIN, + QueryAction.HEALTH, + QueryAction.CURRENT_VERSION, + QueryAction.TOPIC_PARTITION_INGESTION_CONTEXT)); + + public ServerStoreAclHandler( + IdentityParser identityParser, + DynamicAccessController accessController, + ReadOnlyStoreRepository metadataRepository) { + super(identityParser, accessController, metadataRepository); } @Override - public void channelRead0(ChannelHandlerContext ctx, HttpRequest req) throws SSLPeerUnverifiedException { - if (checkWhetherAccessHasAlreadyApproved(ctx)) { - /** - * Access has been approved by {@link ServerAclHandler}. - */ - ReferenceCountUtil.retain(req); - ctx.fireChannelRead(req); - } else { - super.channelRead0(ctx, req); - } + protected boolean needsAclValidation(QueryAction queryAction) { + /* + * Skip request uri validations for store name and certificates due to special actions + * TODO: Identify validations for each query actions and have a flow to perform validations and actions based on + * query actions + */ + return !QUERIES_TO_SKIP_ACL.contains(queryAction); } @Override @@ -53,16 +79,72 @@ public ServerCall.Listener interceptCall( LOGGER.debug("Access already approved by ServerAclHandler"); return next.startCall(call, headers); } else { - LOGGER.debug("Delegating access check to StoreAclHandler"); - return super.interceptCall(call, headers, next); + return new ForwardingServerCallListener.SimpleForwardingServerCallListener(next.startCall(call, headers)) { + @Override + public void onMessage(ReqT message) { + validateStoreAclForGRPC(super::onMessage, message, call, headers); + } + }; } } + // Visible for testing + void validateStoreAclForGRPC( + Consumer onAuthenticated, + ReqT message, + ServerCall call, + Metadata headers) { + VeniceClientRequest request = (VeniceClientRequest) message; + // For now, GRPC only supports STORAGE query + String resourceName = request.getResourceName(); + String storeName; + try { + storeName = extractStoreName(resourceName, QueryAction.STORAGE); + } catch (IllegalArgumentException e) { + LOGGER.error("Invalid store name in resource '{}'", resourceName); + call.close(Status.INVALID_ARGUMENT.withDescription("Invalid request"), headers); + return; + } + String method = request.getMethod(); + + if (StringUtils.isEmpty(method)) { + LOGGER.error("Invalid method {}", method); + call.close(Status.INVALID_ARGUMENT.withDescription("Invalid request"), headers); + return; + } + + try { + X509Certificate clientCert = extractGrpcClientCert(call); + AccessResult accessResult = checkAccess(call.getAuthority(), clientCert, storeName, method); + switch (accessResult) { + case GRANTED: + onAuthenticated.accept(message); + break; + case UNAUTHORIZED: + case FORBIDDEN: + case ERROR_FORBIDDEN: + call.close(accessResultToGrpcStatus(accessResult), headers); + break; + } + } catch (SSLPeerUnverifiedException e) { + LOGGER.error("Cannot verify the certificate.", e); + call.close(Status.UNAUTHENTICATED.withDescription("Invalid certificate"), headers); + } catch (VeniceException e) { + LOGGER.error("Cannot process request successfully due to", e); + call.close(Status.INTERNAL.withDescription(e.getMessage()), headers); + } + } + + @Override + protected String extractStoreName(QueryAction queryAction, String[] requestParts) { + String resourceName = requestParts[2]; + return extractStoreName(resourceName, queryAction); + } + /** * In Venice Server, the resource name is actually a Kafka topic name for STORAGE/COMPUTE but store name for DICTIONARY. */ - @Override - protected String extractStoreName(String resourceName, QueryAction queryAction) { + private String extractStoreName(String resourceName, QueryAction queryAction) { switch (queryAction) { case STORAGE: case COMPUTE: @@ -75,12 +157,37 @@ protected String extractStoreName(String resourceName, QueryAction queryAction) } } - protected static boolean checkWhetherAccessHasAlreadyApproved(ChannelHandlerContext ctx) { + @Override + protected QueryAction validateRequest(String[] requestParts) { + int partsLength = requestParts.length; + // Only for HEALTH queries, parts length can be 2 + if (partsLength == 2) { + if (requestParts[1].equalsIgnoreCase(QueryAction.HEALTH.name())) { + return QueryAction.HEALTH; + } else { + return null; + } + } else if (partsLength < 3) { // invalid request if parts length < 3 except health queries + return null; + } else { // throw exception to retain current behavior for invalid query actions + try { + return QueryAction.valueOf(requestParts[1].toUpperCase()); + } catch (IllegalArgumentException exception) { + return null; + } + } + } + + @Override + protected boolean isAccessAlreadyApproved(ChannelHandlerContext ctx) { + /** + * Access has been approved by {@link ServerAclHandler}. + */ Attribute serverAclApprovedAttr = ctx.channel().attr(ServerAclHandler.SERVER_ACL_APPROVED_ATTRIBUTE_KEY); return Boolean.TRUE.equals(serverAclApprovedAttr.get()); } - protected static boolean checkWhetherAccessHasAlreadyApproved(Metadata headers) { + static boolean checkWhetherAccessHasAlreadyApproved(Metadata headers) { return Boolean.parseBoolean( headers.get(Metadata.Key.of(ServerAclHandler.SERVER_ACL_APPROVED, Metadata.ASCII_STRING_MARSHALLER))); } diff --git a/services/venice-server/src/test/java/com/linkedin/venice/listener/ServerStoreAclHandlerTest.java b/services/venice-server/src/test/java/com/linkedin/venice/listener/ServerStoreAclHandlerTest.java index f6b5d83785b..d5976bfff42 100644 --- a/services/venice-server/src/test/java/com/linkedin/venice/listener/ServerStoreAclHandlerTest.java +++ b/services/venice-server/src/test/java/com/linkedin/venice/listener/ServerStoreAclHandlerTest.java @@ -1,24 +1,34 @@ package com.linkedin.venice.listener; +import static io.grpc.Status.Code.INVALID_ARGUMENT; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.clearInvocations; 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.assertFalse; import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import com.linkedin.venice.acl.AclException; import com.linkedin.venice.acl.DynamicAccessController; -import com.linkedin.venice.acl.handler.StoreAclHandler; +import com.linkedin.venice.authorization.IdentityParser; +import com.linkedin.venice.common.VeniceSystemStoreType; import com.linkedin.venice.meta.QueryAction; import com.linkedin.venice.meta.ReadOnlyStoreRepository; import com.linkedin.venice.meta.ServerAdminAction; +import com.linkedin.venice.meta.Store; import com.linkedin.venice.meta.Version; +import com.linkedin.venice.protocols.VeniceClientRequest; +import io.grpc.Attributes; +import io.grpc.Grpc; import io.grpc.Metadata; import io.grpc.ServerCall; import io.grpc.ServerCallHandler; @@ -34,13 +44,17 @@ import java.security.cert.X509Certificate; import java.util.List; import java.util.Set; +import java.util.function.Consumer; import javax.net.ssl.SSLEngine; import javax.net.ssl.SSLPeerUnverifiedException; import javax.net.ssl.SSLSession; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.testng.annotations.Test; public class ServerStoreAclHandlerTest { + private static final Logger LOGGER = LogManager.getLogger(ServerStoreAclHandlerTest.class); // Store name can be in a version topic format private static final String TEST_STORE_NAME = "testStore_v1"; private static final String TEST_STORE_VERSION = Version.composeKafkaTopic(TEST_STORE_NAME, 1); @@ -48,7 +62,7 @@ public class ServerStoreAclHandlerTest { /** * Mock access controller to verify basic request parsing and handling for {@link ServerStoreAclHandler} */ - private class MockAccessController implements DynamicAccessController { + private static class MockAccessController implements DynamicAccessController { private QueryAction queryAction; public MockAccessController(QueryAction queryAction) { @@ -138,18 +152,21 @@ public void testCheckWhetherAccessHasAlreadyApproved() { doReturn(true).when(accessAttr).get(); doReturn(accessAttr).when(channel).attr(ServerAclHandler.SERVER_ACL_APPROVED_ATTRIBUTE_KEY); + ServerStoreAclHandler handler = new ServerStoreAclHandler( + mock(IdentityParser.class), + mock(DynamicAccessController.class), + mock(ReadOnlyStoreRepository.class)); + assertTrue( - ServerStoreAclHandler.checkWhetherAccessHasAlreadyApproved(ctx), + handler.isAccessAlreadyApproved(ctx), "Should return true if it is already approved by previous acl handler"); doReturn(false).when(accessAttr).get(); assertFalse( - ServerStoreAclHandler.checkWhetherAccessHasAlreadyApproved(ctx), + handler.isAccessAlreadyApproved(ctx), "Should return false if it is already denied by previous acl handler"); doReturn(null).when(accessAttr).get(); - assertFalse( - ServerStoreAclHandler.checkWhetherAccessHasAlreadyApproved(ctx), - "Should return false if it hasn't been processed by acl handler"); + assertFalse(handler.isAccessAlreadyApproved(ctx), "Should return false if it hasn't been processed by acl handler"); } @Test @@ -170,8 +187,10 @@ public void testInterceptor() { Metadata falseHeaders = new Metadata(); falseHeaders.put(Metadata.Key.of(ServerAclHandler.SERVER_ACL_APPROVED, Metadata.ASCII_STRING_MARSHALLER), "false"); - ServerStoreAclHandler handler = - new ServerStoreAclHandler(mock(DynamicAccessController.class), mock(ReadOnlyStoreRepository.class)); + ServerStoreAclHandler handler = new ServerStoreAclHandler( + mock(IdentityParser.class), + mock(DynamicAccessController.class), + mock(ReadOnlyStoreRepository.class)); // next.intercept call should have been invoked handler.interceptCall(call, falseHeaders, next); @@ -187,7 +206,9 @@ public void testInterceptor() { @Test public void testAllRequestTypes() throws SSLPeerUnverifiedException, AclException { + Store store = mock(Store.class); ReadOnlyStoreRepository metadataRepo = mock(ReadOnlyStoreRepository.class); + when(metadataRepo.getStore(TEST_STORE_NAME)).thenReturn(store); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); HttpRequest request = mock(HttpRequest.class); Channel channel = mock(Channel.class); @@ -203,7 +224,7 @@ public void testAllRequestTypes() throws SSLPeerUnverifiedException, AclExceptio doReturn(sslHandler).when(channelPipeline).get(SslHandler.class); SSLEngine sslEngine = mock(SSLEngine.class); SSLSession sslSession = mock(SSLSession.class); - Certificate certificate = mock(X509Certificate.class); + X509Certificate certificate = mock(X509Certificate.class); Certificate[] certificates = new Certificate[1]; certificates[0] = certificate; doReturn(certificates).when(sslSession).getPeerCertificates(); @@ -211,12 +232,17 @@ public void testAllRequestTypes() throws SSLPeerUnverifiedException, AclExceptio doReturn(sslEngine).when(sslHandler).engine(); doReturn(channelPipeline).when(ctx).pipeline(); doReturn(HttpMethod.GET).when(request).method(); + IdentityParser identityParser = mock(IdentityParser.class); + doReturn("testPrincipalId").when(identityParser).parseIdentityFromCert(certificate); for (QueryAction queryAction: QueryAction.values()) { MockAccessController mockAccessController = new MockAccessController(queryAction); MockAccessController spyMockAccessController = spy(mockAccessController); - StoreAclHandler storeAclHandler = new ServerStoreAclHandler(spyMockAccessController, metadataRepo); + ServerStoreAclHandler storeAclHandler = + new ServerStoreAclHandler(identityParser, spyMockAccessController, metadataRepo); doReturn(buildTestURI(queryAction)).when(request).uri(); storeAclHandler.channelRead0(ctx, request); + + LOGGER.info("Testing {} query action", queryAction); switch (queryAction) { case ADMIN: case CURRENT_VERSION: @@ -260,4 +286,81 @@ private String buildTestURI(QueryAction queryAction) { throw new IllegalArgumentException("Invalid query action: " + queryAction); } } + + @Test + public void testInvalidRequest() { + ServerStoreAclHandler handler = new ServerStoreAclHandler( + mock(IdentityParser.class), + mock(DynamicAccessController.class), + mock(ReadOnlyStoreRepository.class)); + + // Happy path is tested in "testAllRequestTypes". Only test the invalid paths + + // #parts == 2 but != HEALTH request + assertNull(handler.validateRequest(new String[] { "", "invalid" })); + + // #parts == 1 (if request is made to "/") + assertNull(handler.validateRequest(new String[] { "" })); + + // #parts == 1 (if request is made without "/". Not sure if this is possible too. But testing for completeness) + assertNull(handler.validateRequest(new String[] { "invalid" })); + + // #parts >= 3, but invalid QueryAction + assertNull(handler.validateRequest(new String[] { "", "invalid", "whatever" })); + } + + @Test + public void testValidateStoreAclForGRPC() throws SSLPeerUnverifiedException, AclException { + Consumer onAuthenticatedConsumer = spy(Consumer.class); + ServerCall serverCall = spy(ServerCall.class); + Metadata headers = new Metadata(); + + IdentityParser identityParser = mock(IdentityParser.class); + MockAccessController accessController = new MockAccessController(QueryAction.STORAGE); + ReadOnlyStoreRepository metadataRepository = mock(ReadOnlyStoreRepository.class); + + SSLSession sslSession = mock(SSLSession.class); + Attributes attributes = Attributes.newBuilder().set(Grpc.TRANSPORT_ATTR_SSL_SESSION, sslSession).build(); + doReturn(attributes).when(serverCall).getAttributes(); + + X509Certificate certificate = mock(X509Certificate.class); + Certificate[] certificates = new Certificate[] { certificate }; + doReturn(certificates).when(sslSession).getPeerCertificates(); + doReturn("identity").when(identityParser).parseIdentityFromCert(certificate); + + ServerStoreAclHandler handler = new ServerStoreAclHandler(identityParser, accessController, metadataRepository); + + // Empty store name + VeniceClientRequest emptyStoreRequest = VeniceClientRequest.newBuilder().build(); + handler.validateStoreAclForGRPC(onAuthenticatedConsumer, emptyStoreRequest, serverCall, headers); + verify(serverCall, times(1)).close( + argThat((status) -> status.getCode() == INVALID_ARGUMENT && status.getDescription().equals("Invalid request")), + eq(headers)); + clearInvocations(serverCall); + + // Empty method + VeniceClientRequest emptyMethodRequest = + VeniceClientRequest.newBuilder().setResourceName(TEST_STORE_VERSION).build(); + handler.validateStoreAclForGRPC(onAuthenticatedConsumer, emptyMethodRequest, serverCall, headers); + verify(serverCall, times(1)).close( + argThat((status) -> status.getCode() == INVALID_ARGUMENT && status.getDescription().equals("Invalid request")), + eq(headers)); + clearInvocations(serverCall); + + // System store + String systemStoreName = VeniceSystemStoreType.DAVINCI_PUSH_STATUS_STORE.getSystemStoreName(TEST_STORE_NAME); + String storeVersion = Version.composeKafkaTopic(systemStoreName, 1); + VeniceClientRequest systemStoreRequest = + VeniceClientRequest.newBuilder().setResourceName(storeVersion).setMethod(HttpMethod.GET.name()).build(); + handler.validateStoreAclForGRPC(onAuthenticatedConsumer, systemStoreRequest, serverCall, headers); + verify(onAuthenticatedConsumer, times(1)).accept(eq(systemStoreRequest)); + clearInvocations(onAuthenticatedConsumer); + + // Authenticated + VeniceClientRequest authenticatedRequest = + VeniceClientRequest.newBuilder().setResourceName(TEST_STORE_VERSION).setMethod(HttpMethod.GET.name()).build(); + handler.validateStoreAclForGRPC(onAuthenticatedConsumer, authenticatedRequest, serverCall, headers); + verify(onAuthenticatedConsumer, times(1)).accept(eq(authenticatedRequest)); + clearInvocations(onAuthenticatedConsumer); + } }