From 88e0f5b97e1927ca39da724f4ff713f9906f77dd Mon Sep 17 00:00:00 2001 From: eldernewborn Date: Wed, 18 Sep 2024 09:29:28 -0700 Subject: [PATCH 01/16] [spark] [hadoop] [vpj] Refactor to make spark a first class citizen of com.linkedin.venice (#1183) * Refactor to make spark a first class citizen of com.linkedin.venice * refactor tests in the same fashion --- .../{hadoop => }/spark/SparkConstants.java | 2 +- .../jobs/AbstractDataWriterSparkJob.java | 40 +++++++++---------- .../datawriter/jobs/DataWriterSparkJob.java | 4 +- .../datawriter/partition/PartitionSorter.java | 5 ++- .../partition/VeniceSparkPartitioner.java | 2 +- .../SparkInputRecordProcessor.java | 19 ++++----- .../SparkInputRecordProcessorFactory.java | 4 +- .../task/DataWriterAccumulators.java | 2 +- .../task/SparkDataWriterTaskTracker.java | 2 +- .../writer/SparkPartitionWriter.java | 12 +++--- .../writer/SparkPartitionWriterFactory.java | 4 +- .../engine/SparkEngineTaskConfigProvider.java | 4 +- .../input/VeniceAbstractPartitionReader.java | 2 +- .../input/hdfs/VeniceHdfsInputPartition.java | 2 +- .../hdfs/VeniceHdfsInputPartitionReader.java | 4 +- ...VeniceHdfsInputPartitionReaderFactory.java | 2 +- .../spark/input/hdfs/VeniceHdfsInputScan.java | 4 +- .../hdfs/VeniceHdfsInputScanBuilder.java | 2 +- .../input/hdfs/VeniceHdfsInputTable.java | 4 +- .../spark/input/hdfs/VeniceHdfsSource.java | 4 +- .../spark/utils/SparkPartitionUtils.java | 2 +- .../spark/utils/SparkScalaUtils.java | 2 +- .../jobs/AbstractDataWriterSparkJobTest.java | 12 +++--- .../partition/PartitionSorterTest.java | 2 +- .../partition/VeniceSparkPartitionerTest.java | 4 +- .../task/SparkDataWriterTaskTrackerTest.java | 4 +- .../SparkEngineTaskConfigProviderTest.java | 4 +- .../input/hdfs/TestSparkInputFromHdfs.java | 6 +-- .../spark/utils/SparkScalaUtilsTest.java | 6 +-- .../venice/endToEnd/PartialUpdateTest.java | 2 +- .../linkedin/venice/endToEnd/TestBatch.java | 2 +- .../venice/endToEnd/TestVsonStoreBatch.java | 2 +- 32 files changed, 85 insertions(+), 87 deletions(-) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/spark/SparkConstants.java (98%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/spark/datawriter/jobs/AbstractDataWriterSparkJob.java (93%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/spark/datawriter/jobs/DataWriterSparkJob.java (95%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/spark/datawriter/partition/PartitionSorter.java (78%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/spark/datawriter/partition/VeniceSparkPartitioner.java (97%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/spark/datawriter/recordprocessor/SparkInputRecordProcessor.java (73%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/spark/datawriter/recordprocessor/SparkInputRecordProcessorFactory.java (88%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/spark/datawriter/task/DataWriterAccumulators.java (97%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/spark/datawriter/task/SparkDataWriterTaskTracker.java (98%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/spark/datawriter/writer/SparkPartitionWriter.java (78%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/spark/datawriter/writer/SparkPartitionWriterFactory.java (86%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/spark/engine/SparkEngineTaskConfigProvider.java (85%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/spark/input/VeniceAbstractPartitionReader.java (96%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/spark/input/hdfs/VeniceHdfsInputPartition.java (93%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/spark/input/hdfs/VeniceHdfsInputPartitionReader.java (93%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/spark/input/hdfs/VeniceHdfsInputPartitionReaderFactory.java (94%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/spark/input/hdfs/VeniceHdfsInputScan.java (95%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/spark/input/hdfs/VeniceHdfsInputScanBuilder.java (89%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/spark/input/hdfs/VeniceHdfsInputTable.java (91%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/spark/input/hdfs/VeniceHdfsSource.java (89%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/spark/utils/SparkPartitionUtils.java (96%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/spark/utils/SparkScalaUtils.java (94%) rename clients/venice-push-job/src/test/java/com/linkedin/venice/{hadoop => }/spark/datawriter/jobs/AbstractDataWriterSparkJobTest.java (96%) rename clients/venice-push-job/src/test/java/com/linkedin/venice/{hadoop => }/spark/datawriter/partition/PartitionSorterTest.java (95%) rename clients/venice-push-job/src/test/java/com/linkedin/venice/{hadoop => }/spark/datawriter/partition/VeniceSparkPartitionerTest.java (96%) rename clients/venice-push-job/src/test/java/com/linkedin/venice/{hadoop => }/spark/datawriter/task/SparkDataWriterTaskTrackerTest.java (98%) rename clients/venice-push-job/src/test/java/com/linkedin/venice/{hadoop => }/spark/engine/SparkEngineTaskConfigProviderTest.java (95%) rename clients/venice-push-job/src/test/java/com/linkedin/venice/{hadoop => }/spark/input/hdfs/TestSparkInputFromHdfs.java (98%) rename clients/venice-push-job/src/test/java/com/linkedin/venice/{hadoop => }/spark/utils/SparkScalaUtilsTest.java (80%) diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/SparkConstants.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/SparkConstants.java similarity index 98% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/SparkConstants.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/spark/SparkConstants.java index 657c8d0e79..e8ba243b85 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/SparkConstants.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/SparkConstants.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.spark; +package com.linkedin.venice.spark; import static org.apache.spark.sql.types.DataTypes.BinaryType; import static org.apache.spark.sql.types.DataTypes.IntegerType; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/jobs/AbstractDataWriterSparkJob.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/jobs/AbstractDataWriterSparkJob.java similarity index 93% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/jobs/AbstractDataWriterSparkJob.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/jobs/AbstractDataWriterSparkJob.java index 5587c86fbc..769e5ee3a7 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/jobs/AbstractDataWriterSparkJob.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/jobs/AbstractDataWriterSparkJob.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.spark.datawriter.jobs; +package com.linkedin.venice.spark.datawriter.jobs; import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; import static com.linkedin.venice.ConfigKeys.KAFKA_PRODUCER_DELIVERY_TIMEOUT_MS; @@ -36,17 +36,17 @@ import static com.linkedin.venice.hadoop.VenicePushJobConstants.ZSTD_COMPRESSION_LEVEL; import static com.linkedin.venice.hadoop.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_REQUIRED; import static com.linkedin.venice.hadoop.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_SUCCESS; -import static com.linkedin.venice.hadoop.spark.SparkConstants.DEFAULT_SCHEMA; -import static com.linkedin.venice.hadoop.spark.SparkConstants.DEFAULT_SCHEMA_WITH_PARTITION; -import static com.linkedin.venice.hadoop.spark.SparkConstants.DEFAULT_SPARK_CLUSTER; -import static com.linkedin.venice.hadoop.spark.SparkConstants.KEY_COLUMN_NAME; -import static com.linkedin.venice.hadoop.spark.SparkConstants.PARTITION_COLUMN_NAME; -import static com.linkedin.venice.hadoop.spark.SparkConstants.SPARK_CASE_SENSITIVE_CONFIG; -import static com.linkedin.venice.hadoop.spark.SparkConstants.SPARK_CLUSTER_CONFIG; -import static com.linkedin.venice.hadoop.spark.SparkConstants.SPARK_DATA_WRITER_CONF_PREFIX; -import static com.linkedin.venice.hadoop.spark.SparkConstants.SPARK_LEADER_CONFIG; -import static com.linkedin.venice.hadoop.spark.SparkConstants.SPARK_SESSION_CONF_PREFIX; -import static com.linkedin.venice.hadoop.spark.SparkConstants.VALUE_COLUMN_NAME; +import static com.linkedin.venice.spark.SparkConstants.DEFAULT_SCHEMA; +import static com.linkedin.venice.spark.SparkConstants.DEFAULT_SCHEMA_WITH_PARTITION; +import static com.linkedin.venice.spark.SparkConstants.DEFAULT_SPARK_CLUSTER; +import static com.linkedin.venice.spark.SparkConstants.KEY_COLUMN_NAME; +import static com.linkedin.venice.spark.SparkConstants.PARTITION_COLUMN_NAME; +import static com.linkedin.venice.spark.SparkConstants.SPARK_CASE_SENSITIVE_CONFIG; +import static com.linkedin.venice.spark.SparkConstants.SPARK_CLUSTER_CONFIG; +import static com.linkedin.venice.spark.SparkConstants.SPARK_DATA_WRITER_CONF_PREFIX; +import static com.linkedin.venice.spark.SparkConstants.SPARK_LEADER_CONFIG; +import static com.linkedin.venice.spark.SparkConstants.SPARK_SESSION_CONF_PREFIX; +import static com.linkedin.venice.spark.SparkConstants.VALUE_COLUMN_NAME; import com.github.luben.zstd.Zstd; import com.linkedin.venice.compression.CompressionStrategy; @@ -55,16 +55,16 @@ import com.linkedin.venice.hadoop.PushJobSetting; import com.linkedin.venice.hadoop.input.kafka.ttl.TTLResolutionPolicy; import com.linkedin.venice.hadoop.jobs.DataWriterComputeJob; -import com.linkedin.venice.hadoop.spark.datawriter.partition.PartitionSorter; -import com.linkedin.venice.hadoop.spark.datawriter.partition.VeniceSparkPartitioner; -import com.linkedin.venice.hadoop.spark.datawriter.recordprocessor.SparkInputRecordProcessorFactory; -import com.linkedin.venice.hadoop.spark.datawriter.task.DataWriterAccumulators; -import com.linkedin.venice.hadoop.spark.datawriter.task.SparkDataWriterTaskTracker; -import com.linkedin.venice.hadoop.spark.datawriter.writer.SparkPartitionWriterFactory; -import com.linkedin.venice.hadoop.spark.utils.SparkPartitionUtils; -import com.linkedin.venice.hadoop.spark.utils.SparkScalaUtils; import com.linkedin.venice.hadoop.ssl.TempFileSSLConfigurator; import com.linkedin.venice.hadoop.task.datawriter.DataWriterTaskTracker; +import com.linkedin.venice.spark.datawriter.partition.PartitionSorter; +import com.linkedin.venice.spark.datawriter.partition.VeniceSparkPartitioner; +import com.linkedin.venice.spark.datawriter.recordprocessor.SparkInputRecordProcessorFactory; +import com.linkedin.venice.spark.datawriter.task.DataWriterAccumulators; +import com.linkedin.venice.spark.datawriter.task.SparkDataWriterTaskTracker; +import com.linkedin.venice.spark.datawriter.writer.SparkPartitionWriterFactory; +import com.linkedin.venice.spark.utils.SparkPartitionUtils; +import com.linkedin.venice.spark.utils.SparkScalaUtils; import com.linkedin.venice.utils.VeniceProperties; import com.linkedin.venice.writer.VeniceWriter; import java.io.IOException; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/jobs/DataWriterSparkJob.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/jobs/DataWriterSparkJob.java similarity index 95% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/jobs/DataWriterSparkJob.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/jobs/DataWriterSparkJob.java index 5cd2e171a1..5bf7b5b82b 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/jobs/DataWriterSparkJob.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/jobs/DataWriterSparkJob.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.spark.datawriter.jobs; +package com.linkedin.venice.spark.datawriter.jobs; import static com.linkedin.venice.hadoop.VenicePushJobConstants.ETL_VALUE_SCHEMA_TRANSFORMATION; import static com.linkedin.venice.hadoop.VenicePushJobConstants.FILE_KEY_SCHEMA; @@ -12,7 +12,7 @@ import static com.linkedin.venice.hadoop.VenicePushJobConstants.VSON_PUSH; import com.linkedin.venice.hadoop.PushJobSetting; -import com.linkedin.venice.hadoop.spark.input.hdfs.VeniceHdfsSource; +import com.linkedin.venice.spark.input.hdfs.VeniceHdfsSource; import org.apache.hadoop.fs.Path; import org.apache.spark.sql.DataFrameReader; import org.apache.spark.sql.Dataset; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/partition/PartitionSorter.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/partition/PartitionSorter.java similarity index 78% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/partition/PartitionSorter.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/partition/PartitionSorter.java index 61ca53d2d0..f4b591ce69 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/partition/PartitionSorter.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/partition/PartitionSorter.java @@ -1,5 +1,6 @@ -package com.linkedin.venice.hadoop.spark.datawriter.partition; +package com.linkedin.venice.spark.datawriter.partition; +import com.linkedin.venice.spark.datawriter.writer.SparkPartitionWriter; import com.linkedin.venice.utils.ArrayUtils; import java.io.Serializable; import java.util.Comparator; @@ -10,7 +11,7 @@ * Sort the rows based on the key and value in ascending order using unsigned byte comparison. * */ public class PartitionSorter implements Comparator, Serializable { diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/partition/VeniceSparkPartitioner.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/partition/VeniceSparkPartitioner.java similarity index 97% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/partition/VeniceSparkPartitioner.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/partition/VeniceSparkPartitioner.java index d7caa4a2e4..ebd9727ac2 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/partition/VeniceSparkPartitioner.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/partition/VeniceSparkPartitioner.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.spark.datawriter.partition; +package com.linkedin.venice.spark.datawriter.partition; import static com.linkedin.venice.hadoop.VenicePushJobConstants.PARTITION_COUNT; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/recordprocessor/SparkInputRecordProcessor.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/recordprocessor/SparkInputRecordProcessor.java similarity index 73% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/recordprocessor/SparkInputRecordProcessor.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/recordprocessor/SparkInputRecordProcessor.java index 256f8d7678..54c683461e 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/recordprocessor/SparkInputRecordProcessor.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/recordprocessor/SparkInputRecordProcessor.java @@ -1,16 +1,13 @@ -package com.linkedin.venice.hadoop.spark.datawriter.recordprocessor; - -import static com.linkedin.venice.hadoop.spark.SparkConstants.DEFAULT_SCHEMA; -import static com.linkedin.venice.hadoop.spark.SparkConstants.KEY_COLUMN_NAME; -import static com.linkedin.venice.hadoop.spark.SparkConstants.VALUE_COLUMN_NAME; +package com.linkedin.venice.spark.datawriter.recordprocessor; import com.linkedin.venice.hadoop.input.recordreader.AbstractVeniceRecordReader; import com.linkedin.venice.hadoop.input.recordreader.avro.IdentityVeniceRecordReader; -import com.linkedin.venice.hadoop.spark.datawriter.task.DataWriterAccumulators; -import com.linkedin.venice.hadoop.spark.datawriter.task.SparkDataWriterTaskTracker; -import com.linkedin.venice.hadoop.spark.engine.SparkEngineTaskConfigProvider; import com.linkedin.venice.hadoop.task.datawriter.AbstractInputRecordProcessor; import com.linkedin.venice.hadoop.task.datawriter.DataWriterTaskTracker; +import com.linkedin.venice.spark.SparkConstants; +import com.linkedin.venice.spark.datawriter.task.DataWriterAccumulators; +import com.linkedin.venice.spark.datawriter.task.SparkDataWriterTaskTracker; +import com.linkedin.venice.spark.engine.SparkEngineTaskConfigProvider; import com.linkedin.venice.utils.VeniceProperties; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -36,8 +33,8 @@ public SparkInputRecordProcessor(Properties jobProperties, DataWriterAccumulator public Iterator processRecord(Row record) { List outputRows = new ArrayList<>(); - ByteBuffer keyBB = ByteBuffer.wrap(record.getAs(KEY_COLUMN_NAME)); - byte[] value = record.getAs(VALUE_COLUMN_NAME); + ByteBuffer keyBB = ByteBuffer.wrap(record.getAs(SparkConstants.KEY_COLUMN_NAME)); + byte[] value = record.getAs(SparkConstants.VALUE_COLUMN_NAME); ByteBuffer valueBB = value == null ? null : ByteBuffer.wrap(value); super.processRecord(keyBB, valueBB, getRecordEmitter(outputRows), dataWriterTaskTracker); return outputRows.iterator(); @@ -50,7 +47,7 @@ protected AbstractVeniceRecordReader getRecordReader(Ven private BiConsumer getRecordEmitter(List rows) { return (key, value) -> { - rows.add(new GenericRowWithSchema(new Object[] { key, value }, DEFAULT_SCHEMA)); + rows.add(new GenericRowWithSchema(new Object[] { key, value }, SparkConstants.DEFAULT_SCHEMA)); }; } } diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/recordprocessor/SparkInputRecordProcessorFactory.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/recordprocessor/SparkInputRecordProcessorFactory.java similarity index 88% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/recordprocessor/SparkInputRecordProcessorFactory.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/recordprocessor/SparkInputRecordProcessorFactory.java index ba8bb24570..d07d7af04f 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/recordprocessor/SparkInputRecordProcessorFactory.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/recordprocessor/SparkInputRecordProcessorFactory.java @@ -1,6 +1,6 @@ -package com.linkedin.venice.hadoop.spark.datawriter.recordprocessor; +package com.linkedin.venice.spark.datawriter.recordprocessor; -import com.linkedin.venice.hadoop.spark.datawriter.task.DataWriterAccumulators; +import com.linkedin.venice.spark.datawriter.task.DataWriterAccumulators; import java.util.Iterator; import java.util.Properties; import org.apache.spark.api.java.function.FlatMapFunction; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/task/DataWriterAccumulators.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/task/DataWriterAccumulators.java similarity index 97% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/task/DataWriterAccumulators.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/task/DataWriterAccumulators.java index aa5252adfa..6e3477e5e9 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/task/DataWriterAccumulators.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/task/DataWriterAccumulators.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.spark.datawriter.task; +package com.linkedin.venice.spark.datawriter.task; import java.io.Serializable; import org.apache.spark.SparkContext; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/task/SparkDataWriterTaskTracker.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/task/SparkDataWriterTaskTracker.java similarity index 98% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/task/SparkDataWriterTaskTracker.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/task/SparkDataWriterTaskTracker.java index d6bc19ae75..debfe71de4 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/task/SparkDataWriterTaskTracker.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/task/SparkDataWriterTaskTracker.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.spark.datawriter.task; +package com.linkedin.venice.spark.datawriter.task; import com.linkedin.venice.hadoop.task.datawriter.DataWriterTaskTracker; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/writer/SparkPartitionWriter.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/writer/SparkPartitionWriter.java similarity index 78% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/writer/SparkPartitionWriter.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/writer/SparkPartitionWriter.java index 61bb0a3237..271e811391 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/writer/SparkPartitionWriter.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/writer/SparkPartitionWriter.java @@ -1,12 +1,12 @@ -package com.linkedin.venice.hadoop.spark.datawriter.writer; +package com.linkedin.venice.spark.datawriter.writer; -import static com.linkedin.venice.hadoop.spark.SparkConstants.KEY_COLUMN_NAME; -import static com.linkedin.venice.hadoop.spark.SparkConstants.VALUE_COLUMN_NAME; +import static com.linkedin.venice.spark.SparkConstants.KEY_COLUMN_NAME; +import static com.linkedin.venice.spark.SparkConstants.VALUE_COLUMN_NAME; -import com.linkedin.venice.hadoop.spark.datawriter.task.DataWriterAccumulators; -import com.linkedin.venice.hadoop.spark.datawriter.task.SparkDataWriterTaskTracker; -import com.linkedin.venice.hadoop.spark.engine.SparkEngineTaskConfigProvider; import com.linkedin.venice.hadoop.task.datawriter.AbstractPartitionWriter; +import com.linkedin.venice.spark.datawriter.task.DataWriterAccumulators; +import com.linkedin.venice.spark.datawriter.task.SparkDataWriterTaskTracker; +import com.linkedin.venice.spark.engine.SparkEngineTaskConfigProvider; import java.util.ArrayList; import java.util.Arrays; import java.util.Iterator; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/writer/SparkPartitionWriterFactory.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/writer/SparkPartitionWriterFactory.java similarity index 86% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/writer/SparkPartitionWriterFactory.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/writer/SparkPartitionWriterFactory.java index a5f2108fed..6f6f689c93 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/writer/SparkPartitionWriterFactory.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/writer/SparkPartitionWriterFactory.java @@ -1,6 +1,6 @@ -package com.linkedin.venice.hadoop.spark.datawriter.writer; +package com.linkedin.venice.spark.datawriter.writer; -import com.linkedin.venice.hadoop.spark.datawriter.task.DataWriterAccumulators; +import com.linkedin.venice.spark.datawriter.task.DataWriterAccumulators; import java.util.Iterator; import java.util.Properties; import org.apache.spark.api.java.function.MapPartitionsFunction; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/engine/SparkEngineTaskConfigProvider.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/engine/SparkEngineTaskConfigProvider.java similarity index 85% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/engine/SparkEngineTaskConfigProvider.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/spark/engine/SparkEngineTaskConfigProvider.java index 6adb45660b..cb59990161 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/engine/SparkEngineTaskConfigProvider.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/engine/SparkEngineTaskConfigProvider.java @@ -1,6 +1,6 @@ -package com.linkedin.venice.hadoop.spark.engine; +package com.linkedin.venice.spark.engine; -import static com.linkedin.venice.hadoop.spark.SparkConstants.SPARK_APP_NAME_CONFIG; +import static com.linkedin.venice.spark.SparkConstants.SPARK_APP_NAME_CONFIG; import com.linkedin.venice.hadoop.engine.EngineTaskConfigProvider; import java.util.Properties; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/VeniceAbstractPartitionReader.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/VeniceAbstractPartitionReader.java similarity index 96% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/VeniceAbstractPartitionReader.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/VeniceAbstractPartitionReader.java index 43a31eae8a..1b2d1ddb9e 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/VeniceAbstractPartitionReader.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/VeniceAbstractPartitionReader.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.spark.input; +package com.linkedin.venice.spark.input; import com.linkedin.venice.hadoop.input.recordreader.VeniceRecordIterator; import com.linkedin.venice.utils.Utils; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputPartition.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputPartition.java similarity index 93% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputPartition.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputPartition.java index 48325ecc34..981ebe8d59 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputPartition.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputPartition.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.spark.input.hdfs; +package com.linkedin.venice.spark.input.hdfs; import java.net.URI; import org.apache.hadoop.fs.Path; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputPartitionReader.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputPartitionReader.java similarity index 93% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputPartitionReader.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputPartitionReader.java index 3bacdfd35f..c48ff758a5 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputPartitionReader.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputPartitionReader.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.spark.input.hdfs; +package com.linkedin.venice.spark.input.hdfs; import static com.linkedin.venice.hadoop.VenicePushJobConstants.VSON_PUSH; @@ -8,7 +8,7 @@ import com.linkedin.venice.hadoop.input.recordreader.avro.VeniceAvroRecordReader; import com.linkedin.venice.hadoop.input.recordreader.vson.VeniceVsonFileIterator; import com.linkedin.venice.hadoop.input.recordreader.vson.VeniceVsonRecordReader; -import com.linkedin.venice.hadoop.spark.input.VeniceAbstractPartitionReader; +import com.linkedin.venice.spark.input.VeniceAbstractPartitionReader; import com.linkedin.venice.utils.VeniceProperties; import java.io.IOException; import org.apache.hadoop.conf.Configuration; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputPartitionReaderFactory.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputPartitionReaderFactory.java similarity index 94% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputPartitionReaderFactory.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputPartitionReaderFactory.java index 49d9b58d42..01c62b3938 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputPartitionReaderFactory.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputPartitionReaderFactory.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.spark.input.hdfs; +package com.linkedin.venice.spark.input.hdfs; import com.linkedin.venice.utils.VeniceProperties; import org.apache.spark.sql.catalyst.InternalRow; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputScan.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputScan.java similarity index 95% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputScan.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputScan.java index b24b111a77..7dcb56369c 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputScan.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputScan.java @@ -1,10 +1,10 @@ -package com.linkedin.venice.hadoop.spark.input.hdfs; +package com.linkedin.venice.spark.input.hdfs; import static com.linkedin.venice.hadoop.VenicePushJobConstants.INPUT_PATH_PROP; import static com.linkedin.venice.hadoop.VenicePushJobConstants.PATH_FILTER; import com.linkedin.venice.exceptions.VeniceException; -import com.linkedin.venice.hadoop.spark.SparkConstants; +import com.linkedin.venice.spark.SparkConstants; import com.linkedin.venice.utils.VeniceProperties; import java.io.IOException; import java.util.ArrayList; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputScanBuilder.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputScanBuilder.java similarity index 89% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputScanBuilder.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputScanBuilder.java index 42e900e12e..b35058062e 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputScanBuilder.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputScanBuilder.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.spark.input.hdfs; +package com.linkedin.venice.spark.input.hdfs; import com.linkedin.venice.utils.VeniceProperties; import org.apache.spark.sql.connector.read.Scan; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputTable.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputTable.java similarity index 91% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputTable.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputTable.java index 5b8b3bbca4..14fe5c6bcd 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputTable.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputTable.java @@ -1,6 +1,6 @@ -package com.linkedin.venice.hadoop.spark.input.hdfs; +package com.linkedin.venice.spark.input.hdfs; -import static com.linkedin.venice.hadoop.spark.SparkConstants.DEFAULT_SCHEMA; +import static com.linkedin.venice.spark.SparkConstants.DEFAULT_SCHEMA; import com.linkedin.venice.utils.VeniceProperties; import java.util.Collections; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsSource.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsSource.java similarity index 89% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsSource.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsSource.java index 9e76f4f3c1..2c2fc90445 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsSource.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsSource.java @@ -1,6 +1,6 @@ -package com.linkedin.venice.hadoop.spark.input.hdfs; +package com.linkedin.venice.spark.input.hdfs; -import static com.linkedin.venice.hadoop.spark.SparkConstants.DEFAULT_SCHEMA; +import static com.linkedin.venice.spark.SparkConstants.DEFAULT_SCHEMA; import com.linkedin.venice.utils.VeniceProperties; import java.util.Map; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/utils/SparkPartitionUtils.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/utils/SparkPartitionUtils.java similarity index 96% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/utils/SparkPartitionUtils.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/spark/utils/SparkPartitionUtils.java index 9fb1758a33..0b429c84a3 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/utils/SparkPartitionUtils.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/utils/SparkPartitionUtils.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.spark.utils; +package com.linkedin.venice.spark.utils; import java.util.Comparator; import org.apache.spark.Partitioner; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/utils/SparkScalaUtils.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/utils/SparkScalaUtils.java similarity index 94% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/utils/SparkScalaUtils.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/spark/utils/SparkScalaUtils.java index 33e3ac1ba4..ce42a0be16 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/utils/SparkScalaUtils.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/utils/SparkScalaUtils.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.spark.utils; +package com.linkedin.venice.spark.utils; import org.apache.spark.sql.types.StructType; import scala.Int; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/datawriter/jobs/AbstractDataWriterSparkJobTest.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/datawriter/jobs/AbstractDataWriterSparkJobTest.java similarity index 96% rename from clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/datawriter/jobs/AbstractDataWriterSparkJobTest.java rename to clients/venice-push-job/src/test/java/com/linkedin/venice/spark/datawriter/jobs/AbstractDataWriterSparkJobTest.java index 5c2b198ff1..a6e6628988 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/datawriter/jobs/AbstractDataWriterSparkJobTest.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/datawriter/jobs/AbstractDataWriterSparkJobTest.java @@ -1,14 +1,14 @@ -package com.linkedin.venice.hadoop.spark.datawriter.jobs; +package com.linkedin.venice.spark.datawriter.jobs; import static com.linkedin.venice.ConfigKeys.KAFKA_CONFIG_PREFIX; 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.spark.SparkConstants.KEY_COLUMN_NAME; -import static com.linkedin.venice.hadoop.spark.SparkConstants.SPARK_APP_NAME_CONFIG; -import static com.linkedin.venice.hadoop.spark.SparkConstants.SPARK_DATA_WRITER_CONF_PREFIX; -import static com.linkedin.venice.hadoop.spark.SparkConstants.SPARK_SESSION_CONF_PREFIX; -import static com.linkedin.venice.hadoop.spark.SparkConstants.VALUE_COLUMN_NAME; import static com.linkedin.venice.meta.Store.UNLIMITED_STORAGE_QUOTA; +import static com.linkedin.venice.spark.SparkConstants.KEY_COLUMN_NAME; +import static com.linkedin.venice.spark.SparkConstants.SPARK_APP_NAME_CONFIG; +import static com.linkedin.venice.spark.SparkConstants.SPARK_DATA_WRITER_CONF_PREFIX; +import static com.linkedin.venice.spark.SparkConstants.SPARK_SESSION_CONF_PREFIX; +import static com.linkedin.venice.spark.SparkConstants.VALUE_COLUMN_NAME; import static org.apache.spark.sql.types.DataTypes.BinaryType; import static org.apache.spark.sql.types.DataTypes.StringType; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/datawriter/partition/PartitionSorterTest.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/datawriter/partition/PartitionSorterTest.java similarity index 95% rename from clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/datawriter/partition/PartitionSorterTest.java rename to clients/venice-push-job/src/test/java/com/linkedin/venice/spark/datawriter/partition/PartitionSorterTest.java index 289adee8dd..727c6418da 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/datawriter/partition/PartitionSorterTest.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/datawriter/partition/PartitionSorterTest.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.spark.datawriter.partition; +package com.linkedin.venice.spark.datawriter.partition; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/datawriter/partition/VeniceSparkPartitionerTest.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/datawriter/partition/VeniceSparkPartitionerTest.java similarity index 96% rename from clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/datawriter/partition/VeniceSparkPartitionerTest.java rename to clients/venice-push-job/src/test/java/com/linkedin/venice/spark/datawriter/partition/VeniceSparkPartitionerTest.java index 320ad8d69c..b7d78153e2 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/datawriter/partition/VeniceSparkPartitionerTest.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/datawriter/partition/VeniceSparkPartitionerTest.java @@ -1,11 +1,11 @@ -package com.linkedin.venice.hadoop.spark.datawriter.partition; +package com.linkedin.venice.spark.datawriter.partition; import static com.linkedin.venice.hadoop.VenicePushJobConstants.PARTITION_COUNT; import com.linkedin.venice.exceptions.VeniceException; -import com.linkedin.venice.hadoop.spark.SparkConstants; import com.linkedin.venice.partitioner.DefaultVenicePartitioner; import com.linkedin.venice.partitioner.VenicePartitioner; +import com.linkedin.venice.spark.SparkConstants; import com.linkedin.venice.utils.ByteUtils; import com.linkedin.venice.utils.VeniceProperties; import java.util.Properties; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/datawriter/task/SparkDataWriterTaskTrackerTest.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/datawriter/task/SparkDataWriterTaskTrackerTest.java similarity index 98% rename from clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/datawriter/task/SparkDataWriterTaskTrackerTest.java rename to clients/venice-push-job/src/test/java/com/linkedin/venice/spark/datawriter/task/SparkDataWriterTaskTrackerTest.java index c036f0eb4d..b0f39d5537 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/datawriter/task/SparkDataWriterTaskTrackerTest.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/datawriter/task/SparkDataWriterTaskTrackerTest.java @@ -1,6 +1,6 @@ -package com.linkedin.venice.hadoop.spark.datawriter.task; +package com.linkedin.venice.spark.datawriter.task; -import com.linkedin.venice.hadoop.spark.SparkConstants; +import com.linkedin.venice.spark.SparkConstants; import org.apache.spark.sql.SparkSession; import org.testng.Assert; import org.testng.annotations.AfterClass; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/engine/SparkEngineTaskConfigProviderTest.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/engine/SparkEngineTaskConfigProviderTest.java similarity index 95% rename from clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/engine/SparkEngineTaskConfigProviderTest.java rename to clients/venice-push-job/src/test/java/com/linkedin/venice/spark/engine/SparkEngineTaskConfigProviderTest.java index cb244cd670..0bbacb2862 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/engine/SparkEngineTaskConfigProviderTest.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/engine/SparkEngineTaskConfigProviderTest.java @@ -1,6 +1,6 @@ -package com.linkedin.venice.hadoop.spark.engine; +package com.linkedin.venice.spark.engine; -import com.linkedin.venice.hadoop.spark.SparkConstants; +import com.linkedin.venice.spark.SparkConstants; import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/input/hdfs/TestSparkInputFromHdfs.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/input/hdfs/TestSparkInputFromHdfs.java similarity index 98% rename from clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/input/hdfs/TestSparkInputFromHdfs.java rename to clients/venice-push-job/src/test/java/com/linkedin/venice/spark/input/hdfs/TestSparkInputFromHdfs.java index a7b7237366..c47def0872 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/input/hdfs/TestSparkInputFromHdfs.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/input/hdfs/TestSparkInputFromHdfs.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.spark.input.hdfs; +package com.linkedin.venice.spark.input.hdfs; import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; @@ -11,8 +11,8 @@ import static com.linkedin.venice.hadoop.VenicePushJobConstants.UPDATE_SCHEMA_STRING_PROP; import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_FIELD_PROP; import static com.linkedin.venice.hadoop.VenicePushJobConstants.VSON_PUSH; -import static com.linkedin.venice.hadoop.spark.SparkConstants.DEFAULT_SCHEMA; -import static com.linkedin.venice.hadoop.spark.input.hdfs.VeniceHdfsInputTable.INPUT_TABLE_NAME; +import static com.linkedin.venice.spark.SparkConstants.DEFAULT_SCHEMA; +import static com.linkedin.venice.spark.input.hdfs.VeniceHdfsInputTable.INPUT_TABLE_NAME; import static com.linkedin.venice.utils.TestWriteUtils.DEFAULT_USER_DATA_VALUE_PREFIX; import static com.linkedin.venice.utils.TestWriteUtils.NAME_RECORD_V2_SCHEMA; import static com.linkedin.venice.utils.TestWriteUtils.STRING_TO_STRING_SCHEMA; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/utils/SparkScalaUtilsTest.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/utils/SparkScalaUtilsTest.java similarity index 80% rename from clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/utils/SparkScalaUtilsTest.java rename to clients/venice-push-job/src/test/java/com/linkedin/venice/spark/utils/SparkScalaUtilsTest.java index 7081393d26..badf34aea8 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/utils/SparkScalaUtilsTest.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/utils/SparkScalaUtilsTest.java @@ -1,7 +1,7 @@ -package com.linkedin.venice.hadoop.spark.utils; +package com.linkedin.venice.spark.utils; -import static com.linkedin.venice.hadoop.spark.SparkConstants.KEY_COLUMN_NAME; -import static com.linkedin.venice.hadoop.spark.SparkConstants.VALUE_COLUMN_NAME; +import static com.linkedin.venice.spark.SparkConstants.KEY_COLUMN_NAME; +import static com.linkedin.venice.spark.SparkConstants.VALUE_COLUMN_NAME; import static org.apache.spark.sql.types.DataTypes.BinaryType; import org.apache.spark.sql.types.Metadata; 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 8f7fb3a627..d242d7a60b 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 @@ -70,7 +70,6 @@ import com.linkedin.venice.controllerapi.VersionCreationResponse; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.hadoop.VenicePushJob; -import com.linkedin.venice.hadoop.spark.datawriter.jobs.DataWriterSparkJob; import com.linkedin.venice.helix.VeniceJsonSerializer; import com.linkedin.venice.integration.utils.ServiceFactory; import com.linkedin.venice.integration.utils.VeniceClusterWrapper; @@ -93,6 +92,7 @@ import com.linkedin.venice.schema.writecompute.WriteComputeSchemaConverter; import com.linkedin.venice.serialization.avro.AvroProtocolDefinition; import com.linkedin.venice.serialization.avro.ChunkedValueManifestSerializer; +import com.linkedin.venice.spark.datawriter.jobs.DataWriterSparkJob; import com.linkedin.venice.stats.AbstractVeniceStats; import com.linkedin.venice.storage.protocol.ChunkedValueManifest; import com.linkedin.venice.tehuti.MetricsUtils; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestBatch.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestBatch.java index 9667f2aae5..833d1d11a4 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestBatch.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestBatch.java @@ -62,11 +62,11 @@ import com.linkedin.venice.controllerapi.UpdateStoreQueryParams; import com.linkedin.venice.controllerapi.VersionCreationResponse; import com.linkedin.venice.exceptions.VeniceException; -import com.linkedin.venice.hadoop.spark.datawriter.jobs.DataWriterSparkJob; import com.linkedin.venice.integration.utils.VeniceClusterWrapper; import com.linkedin.venice.meta.BackupStrategy; import com.linkedin.venice.meta.Version; import com.linkedin.venice.read.RequestType; +import com.linkedin.venice.spark.datawriter.jobs.DataWriterSparkJob; import com.linkedin.venice.stats.AbstractVeniceStats; import com.linkedin.venice.system.store.MetaStoreDataType; import com.linkedin.venice.systemstore.schemas.StoreMetaKey; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestVsonStoreBatch.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestVsonStoreBatch.java index 932e5ee8d5..952c8aedc9 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestVsonStoreBatch.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestVsonStoreBatch.java @@ -23,12 +23,12 @@ import com.linkedin.venice.compression.CompressionStrategy; import com.linkedin.venice.controllerapi.ControllerClient; import com.linkedin.venice.controllerapi.UpdateStoreQueryParams; -import com.linkedin.venice.hadoop.spark.datawriter.jobs.DataWriterSparkJob; import com.linkedin.venice.integration.utils.ServiceFactory; import com.linkedin.venice.integration.utils.VeniceClusterWrapper; import com.linkedin.venice.meta.Version; import com.linkedin.venice.schema.vson.VsonAvroSchemaAdapter; import com.linkedin.venice.schema.vson.VsonSchema; +import com.linkedin.venice.spark.datawriter.jobs.DataWriterSparkJob; import com.linkedin.venice.utils.KeyAndValueSchemas; import com.linkedin.venice.utils.Pair; import com.linkedin.venice.utils.TestWriteUtils; From 2b0ae246b1078a182fc0fc629e8e06b857657465 Mon Sep 17 00:00:00 2001 From: eldernewborn Date: Thu, 19 Sep 2024 13:45:51 -0700 Subject: [PATCH 02/16] [VPJ] More reorg for the venice-push-job module (#1186) * Brought heartbeat, jobs packages into the main folder since none of them are Hadoop specific. updated relevant tests. * adjusted method signature to allow for testing on DefaultPushJobHeartbeatSender::getVeniceWriter --- .../hadoop/DefaultInputDataInfoProvider.java | 16 +- .../venice/hadoop/PushJobSetting.java | 3 +- .../venice/hadoop/PushJobZstdConfig.java | 6 +- .../ValidateSchemaAndBuildDictMapper.java | 26 +-- ...alidateSchemaAndBuildDictOutputFormat.java | 2 +- .../venice/hadoop/VeniceFileInputFormat.java | 2 +- .../hadoop/VeniceFileInputRecordReader.java | 2 +- .../linkedin/venice/hadoop/VenicePushJob.java | 162 +++++++++--------- .../input/kafka/KafkaInputDictTrainer.java | 12 +- .../hadoop/input/kafka/KafkaInputFormat.java | 6 +- .../input/kafka/KafkaInputFormatCombiner.java | 2 +- .../input/kafka/KafkaInputRecordReader.java | 4 +- .../hadoop/input/kafka/KafkaInputUtils.java | 10 +- .../input/kafka/VeniceKafkaInputMapper.java | 2 +- .../input/kafka/VeniceKafkaInputReducer.java | 12 +- .../input/kafka/ttl/VeniceRmdTTLFilter.java | 14 +- .../avro/VeniceAvroRecordReader.java | 16 +- .../vson/VeniceVsonRecordReader.java | 8 +- .../hadoop/mapreduce/common/JobUtils.java | 12 +- .../datawriter/jobs/DataWriterMRJob.java | 80 ++++----- .../hadoop/ssl/TempFileSSLConfigurator.java | 8 +- .../datawriter/AbstractDataWriterTask.java | 4 +- .../AbstractInputRecordProcessor.java | 14 +- .../datawriter/AbstractPartitionWriter.java | 18 +- .../venice/hadoop/utils/VPJSSLUtils.java | 14 +- .../DefaultPushJobHeartbeatSender.java | 2 +- .../DefaultPushJobHeartbeatSenderFactory.java | 4 +- .../heartbeat/NoOpPushJobHeartbeatSender.java | 2 +- .../NoOpPushJobHeartbeatSenderFactory.java | 2 +- .../heartbeat/PushJobHeartbeatSender.java | 2 +- .../PushJobHeartbeatSenderFactory.java | 2 +- .../venice/{hadoop => }/jobs/ComputeJob.java | 2 +- .../jobs/DataWriterComputeJob.java | 2 +- .../jobs/AbstractDataWriterSparkJob.java | 60 +++---- .../datawriter/jobs/DataWriterSparkJob.java | 20 +-- .../partition/VeniceSparkPartitioner.java | 2 +- .../hdfs/VeniceHdfsInputPartitionReader.java | 2 +- .../spark/input/hdfs/VeniceHdfsInputScan.java | 4 +- .../VenicePushJobConstants.java | 6 +- .../TestKafkaFormatTopicAutoDiscover.java | 24 +-- .../TestValidateSchemaAndBuildDictMapper.java | 16 +- .../hadoop/TestVenicePushJobCheckpoints.java | 34 ++-- .../venice/hadoop/VenicePushJobTest.java | 48 +++--- .../kafka/KafkaInputRecordReaderTest.java | 6 +- .../kafka/TestVeniceKafkaInputMapper.java | 16 +- .../kafka/TestVeniceKafkaInputReducer.java | 22 +-- .../TestVeniceChunkedPayloadTTLFilter.java | 18 +- .../ttl/TestVeniceKafkaInputTTLFilter.java | 18 +- .../mapreduce/AbstractTestVeniceMR.java | 34 ++-- .../datawriter/map/TestVeniceAvroMapper.java | 14 +- .../datawriter/map/TestVeniceVsonMapper.java | 8 +- .../datawriter/reduce/TestVeniceReducer.java | 10 +- .../avro/TestVeniceAvroRecordReader.java | 4 +- .../ssl/TestTempFileSSLConfigurator.java | 8 +- .../venice/hadoop/utils/TestVPJSSLUtils.java | 8 +- .../heartbeat/TestPushJobHeartbeatSender.java | 2 +- .../jobs/DataWriterComputeJobTest.java | 4 +- .../jobs/AbstractDataWriterSparkJobTest.java | 6 +- .../partition/VeniceSparkPartitionerTest.java | 2 +- .../input/hdfs/TestSparkInputFromHdfs.java | 22 +-- .../linkedin/venice/zstd/TestZstdLibrary.java | 8 +- ...nsumerIntegrationTestWithSchemaReader.java | 8 +- .../TestBootstrappingChangelogConsumer.java | 4 +- .../consumer/TestChangelogConsumer.java | 14 +- ...stParentControllerWithMultiDataCenter.java | 2 +- .../endToEnd/DaVinciClientDiskFullTest.java | 2 +- .../venice/endToEnd/DaVinciClientTest.java | 2 +- .../venice/endToEnd/PartialUpdateTest.java | 24 +-- .../venice/endToEnd/PushJobDetailsTest.java | 6 +- .../venice/endToEnd/PushStatusStoreTest.java | 4 +- .../endToEnd/TestActiveActiveIngestion.java | 16 +- ...TestActiveActiveReplicationForIncPush.java | 10 +- ...TestBackupVersionDatabaseOptimization.java | 2 +- .../linkedin/venice/endToEnd/TestBatch.java | 32 ++-- .../TestBatchReportIncrementalPush.java | 6 +- .../venice/endToEnd/TestEmptyPush.java | 4 +- .../linkedin/venice/endToEnd/TestHybrid.java | 2 +- .../endToEnd/TestLeaderReplicaFailover.java | 2 +- .../endToEnd/TestPushJobVersionCleanup.java | 6 +- ...JobWithEmergencySourceRegionSelection.java | 6 +- .../TestPushJobWithNativeReplication.java | 24 +-- ...tPushJobWithSourceGridFabricSelection.java | 8 +- .../TestStoreBackupVersionDeletion.java | 2 +- .../venice/endToEnd/TestStoreMigration.java | 6 +- .../TestSuperSetSchemaRegistration.java | 4 +- ...TopicWiseSharedConsumerPoolResilience.java | 2 +- .../venice/endToEnd/TestVsonStoreBatch.java | 14 +- .../venice/hadoop/TestVenicePushJob.java | 32 ++-- .../input/kafka/TestKafkaInputFormat.java | 6 +- .../kafka/TestKafkaInputRecordReader.java | 6 +- .../IngestionHeartBeatTest.java | 6 +- .../kafka/EndToEndKafkaWithSASLTest.java | 2 +- .../venice/kafka/ssl/TestProduceWithSSL.java | 14 +- .../TestMetadataOperationInMultiCluster.java | 6 +- ...tingSstFilesWithActiveActiveIngestion.java | 4 +- .../TestRestartServerDuringIngestion.java | 4 +- .../utils/IntegrationTestPushUtils.java | 18 +- .../venice/utils/KeyAndValueSchemas.java | 4 +- .../venice/utils/PushInputSchemaBuilder.java | 4 +- .../linkedin/venice/utils/TestWriteUtils.java | 34 ++-- 100 files changed, 627 insertions(+), 624 deletions(-) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/heartbeat/DefaultPushJobHeartbeatSender.java (99%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/heartbeat/DefaultPushJobHeartbeatSenderFactory.java (97%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/heartbeat/NoOpPushJobHeartbeatSender.java (93%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/heartbeat/NoOpPushJobHeartbeatSenderFactory.java (91%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/heartbeat/PushJobHeartbeatSender.java (93%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/heartbeat/PushJobHeartbeatSenderFactory.java (92%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/jobs/ComputeJob.java (96%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => }/jobs/DataWriterComputeJob.java (99%) rename clients/venice-push-job/src/main/java/com/linkedin/venice/{hadoop => vpj}/VenicePushJobConstants.java (98%) rename clients/venice-push-job/src/test/java/com/linkedin/venice/{hadoop => }/heartbeat/TestPushJobHeartbeatSender.java (98%) rename clients/venice-push-job/src/test/java/com/linkedin/venice/{hadoop => }/jobs/DataWriterComputeJobTest.java (87%) diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/DefaultInputDataInfoProvider.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/DefaultInputDataInfoProvider.java index 6d5a28bf5f..e3f44f6df6 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/DefaultInputDataInfoProvider.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/DefaultInputDataInfoProvider.java @@ -1,13 +1,13 @@ package com.linkedin.venice.hadoop; -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.FILE_KEY_SCHEMA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.FILE_VALUE_SCHEMA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KEY_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.MINIMUM_NUMBER_OF_SAMPLES_REQUIRED_TO_BUILD_ZSTD_DICTIONARY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PATH_FILTER; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.FILE_KEY_SCHEMA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.FILE_VALUE_SCHEMA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.MINIMUM_NUMBER_OF_SAMPLES_REQUIRED_TO_BUILD_ZSTD_DICTIONARY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PATH_FILTER; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_FIELD_PROP; import com.linkedin.venice.compression.ZstdWithDictCompressor; import com.linkedin.venice.exceptions.VeniceException; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/PushJobSetting.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/PushJobSetting.java index 906917adbe..487a899ca7 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/PushJobSetting.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/PushJobSetting.java @@ -4,11 +4,12 @@ import com.linkedin.venice.controllerapi.RepushInfoResponse; import com.linkedin.venice.controllerapi.StoreResponse; import com.linkedin.venice.etl.ETLValueSchemaTransformation; -import com.linkedin.venice.hadoop.jobs.DataWriterComputeJob; +import com.linkedin.venice.jobs.DataWriterComputeJob; import com.linkedin.venice.meta.BufferReplayPolicy; import com.linkedin.venice.meta.HybridStoreConfig; import com.linkedin.venice.meta.Version; import com.linkedin.venice.schema.vson.VsonSchema; +import com.linkedin.venice.vpj.VenicePushJobConstants; import java.io.Serializable; import java.util.Map; import org.apache.avro.Schema; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/PushJobZstdConfig.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/PushJobZstdConfig.java index 27cd89981c..1279f4ecf7 100755 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/PushJobZstdConfig.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/PushJobZstdConfig.java @@ -1,8 +1,8 @@ package com.linkedin.venice.hadoop; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_DICTIONARY_SAMPLE_SIZE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_DICTIONARY_SIZE_LIMIT; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_COMPRESSION_DICTIONARY_SAMPLE_SIZE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_DICTIONARY_SAMPLE_SIZE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_DICTIONARY_SIZE_LIMIT; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_COMPRESSION_DICTIONARY_SAMPLE_SIZE; import com.github.luben.zstd.ZstdDictTrainer; import com.linkedin.venice.utils.VeniceProperties; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/ValidateSchemaAndBuildDictMapper.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/ValidateSchemaAndBuildDictMapper.java index 1553f91c7a..f619d854ec 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/ValidateSchemaAndBuildDictMapper.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/ValidateSchemaAndBuildDictMapper.java @@ -1,18 +1,18 @@ package com.linkedin.venice.hadoop; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_METRIC_COLLECTION_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_STRATEGY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ETL_VALUE_SCHEMA_TRANSFORMATION; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.INCREMENTAL_PUSH; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.INPUT_PATH_LAST_MODIFIED_TIME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.INPUT_PATH_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KEY_INPUT_FILE_DATA_SIZE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KEY_ZSTD_COMPRESSION_DICTIONARY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.MINIMUM_NUMBER_OF_SAMPLES_REQUIRED_TO_BUILD_ZSTD_DICTIONARY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PATH_FILTER; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.USE_MAPPER_TO_BUILD_DICTIONARY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_STORE_NAME_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_REQUIRED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_METRIC_COLLECTION_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_STRATEGY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ETL_VALUE_SCHEMA_TRANSFORMATION; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INCREMENTAL_PUSH; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INPUT_PATH_LAST_MODIFIED_TIME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INPUT_PATH_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KEY_INPUT_FILE_DATA_SIZE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KEY_ZSTD_COMPRESSION_DICTIONARY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.MINIMUM_NUMBER_OF_SAMPLES_REQUIRED_TO_BUILD_ZSTD_DICTIONARY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PATH_FILTER; +import static com.linkedin.venice.vpj.VenicePushJobConstants.USE_MAPPER_TO_BUILD_DICTIONARY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_REQUIRED; import com.linkedin.venice.compression.CompressionStrategy; import com.linkedin.venice.etl.ETLValueSchemaTransformation; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/ValidateSchemaAndBuildDictOutputFormat.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/ValidateSchemaAndBuildDictOutputFormat.java index f37fe28c0b..2069f4ba22 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/ValidateSchemaAndBuildDictOutputFormat.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/ValidateSchemaAndBuildDictOutputFormat.java @@ -1,7 +1,7 @@ package com.linkedin.venice.hadoop; import static com.linkedin.venice.hadoop.VenicePushJob.getValidateSchemaAndBuildDictionaryOutputFileNameNoExtension; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALIDATE_SCHEMA_AND_BUILD_DICT_MAPPER_OUTPUT_DIRECTORY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALIDATE_SCHEMA_AND_BUILD_DICT_MAPPER_OUTPUT_DIRECTORY; import static org.apache.hadoop.mapreduce.MRJobConfig.ID; import java.io.IOException; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/VeniceFileInputFormat.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/VeniceFileInputFormat.java index 7b2dbf1775..814cd7d605 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/VeniceFileInputFormat.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/VeniceFileInputFormat.java @@ -1,6 +1,6 @@ package com.linkedin.venice.hadoop; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.INPUT_PATH_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INPUT_PATH_PROP; import java.io.IOException; import org.apache.hadoop.io.IntWritable; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/VeniceFileInputRecordReader.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/VeniceFileInputRecordReader.java index d4214cb98b..8ab8854918 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/VeniceFileInputRecordReader.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/VeniceFileInputRecordReader.java @@ -1,6 +1,6 @@ package com.linkedin.venice.hadoop; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PATH_FILTER; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PATH_FILTER; import java.io.IOException; import org.apache.hadoop.conf.Configuration; 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 f4f9343145..36eaae8def 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 @@ -8,83 +8,83 @@ import static com.linkedin.venice.ConfigKeys.MULTI_REGION; import static com.linkedin.venice.ConfigKeys.VENICE_PARTITIONERS; import static com.linkedin.venice.VeniceConstants.DEFAULT_SSL_FACTORY_CLASS_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ALLOW_DUPLICATE_KEY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.BATCH_NUM_BYTES_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_DICTIONARY_SAMPLE_SIZE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_DICTIONARY_SIZE_LIMIT; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_METRIC_COLLECTION_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_STRATEGY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.CONTROLLER_REQUEST_RETRY_ATTEMPTS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.D2_ZK_HOSTS_PREFIX; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DATA_WRITER_COMPUTE_JOB_CLASS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_BATCH_BYTES_SIZE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_COMPRESSION_DICTIONARY_SAMPLE_SIZE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_COMPRESSION_METRIC_COLLECTION_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_EXTENDED_SCHEMA_VALIDITY_CHECK_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_JOB_STATUS_IN_UNKNOWN_STATE_TIMEOUT_MS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_POLL_STATUS_INTERVAL_MS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_RE_PUSH_REWIND_IN_SECONDS_OVERRIDE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_SSL_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_USE_MAPPER_TO_BUILD_DICTIONARY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFER_VERSION_SWAP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ENABLE_SSL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ENABLE_WRITE_COMPUTE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ETL_VALUE_SCHEMA_TRANSFORMATION; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.EXTENDED_SCHEMA_VALIDITY_CHECK_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.HADOOP_TMP_DIR; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.HADOOP_VALIDATE_SCHEMA_AND_BUILD_DICT_PREFIX; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.INCREMENTAL_PUSH; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.INPUT_PATH_LAST_MODIFIED_TIME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.INPUT_PATH_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.JOB_EXEC_ID; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.JOB_EXEC_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.JOB_SERVER_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.JOB_STATUS_IN_UNKNOWN_STATE_TIMEOUT_MS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_COMBINER_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_COMPRESSION_BUILD_NEW_DICT_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_FABRIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_TOPIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KEY_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.LEGACY_AVRO_KEY_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.LEGACY_AVRO_VALUE_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.NON_CRITICAL_EXCEPTION; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.NOT_SET; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PARENT_CONTROLLER_REGION_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PARTITION_COUNT; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PATH_FILTER; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PERMISSION_700; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PERMISSION_777; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.POLL_JOB_STATUS_INTERVAL_MS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.POLL_STATUS_RETRY_ATTEMPTS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PUSH_JOB_STATUS_UPLOAD_ENABLE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_ENABLE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_SECONDS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_START_TIMESTAMP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REWIND_EPOCH_TIME_BUFFER_IN_SECONDS_OVERRIDE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REWIND_EPOCH_TIME_IN_SECONDS_OVERRIDE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REWIND_TIME_IN_SECONDS_OVERRIDE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SEND_CONTROL_MESSAGES_DIRECTLY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SORTED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_ETL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_GRID_FABRIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_KAFKA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SUPPRESS_END_OF_PUSH_MESSAGE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SYSTEM_SCHEMA_READER_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.TARGETED_REGION_PUSH_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.TARGETED_REGION_PUSH_LIST; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.TEMP_DIR_PREFIX; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.UNCREATED_VERSION_NUMBER; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.USE_MAPPER_TO_BUILD_DICTIONARY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALIDATE_SCHEMA_AND_BUILD_DICTIONARY_MAPPER_OUTPUT_FILE_EXTENSION; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALIDATE_SCHEMA_AND_BUILD_DICTIONARY_MAPPER_OUTPUT_FILE_PREFIX; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALIDATE_SCHEMA_AND_BUILD_DICT_MAPPER_OUTPUT_DIRECTORY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_DISCOVER_URL_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static com.linkedin.venice.status.BatchJobHeartbeatConfigs.HEARTBEAT_ENABLED_CONFIG; import static com.linkedin.venice.utils.AvroSupersetSchemaUtils.validateSubsetValueSchema; import static com.linkedin.venice.utils.ByteUtils.generateHumanReadableByteCountString; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ALLOW_DUPLICATE_KEY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.BATCH_NUM_BYTES_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_DICTIONARY_SAMPLE_SIZE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_DICTIONARY_SIZE_LIMIT; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_METRIC_COLLECTION_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_STRATEGY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.CONTROLLER_REQUEST_RETRY_ATTEMPTS; +import static com.linkedin.venice.vpj.VenicePushJobConstants.D2_ZK_HOSTS_PREFIX; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DATA_WRITER_COMPUTE_JOB_CLASS; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_BATCH_BYTES_SIZE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_COMPRESSION_DICTIONARY_SAMPLE_SIZE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_COMPRESSION_METRIC_COLLECTION_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_EXTENDED_SCHEMA_VALIDITY_CHECK_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_JOB_STATUS_IN_UNKNOWN_STATE_TIMEOUT_MS; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_POLL_STATUS_INTERVAL_MS; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_RE_PUSH_REWIND_IN_SECONDS_OVERRIDE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_SSL_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_USE_MAPPER_TO_BUILD_DICTIONARY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFER_VERSION_SWAP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ENABLE_SSL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ENABLE_WRITE_COMPUTE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ETL_VALUE_SCHEMA_TRANSFORMATION; +import static com.linkedin.venice.vpj.VenicePushJobConstants.EXTENDED_SCHEMA_VALIDITY_CHECK_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.HADOOP_TMP_DIR; +import static com.linkedin.venice.vpj.VenicePushJobConstants.HADOOP_VALIDATE_SCHEMA_AND_BUILD_DICT_PREFIX; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INCREMENTAL_PUSH; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INPUT_PATH_LAST_MODIFIED_TIME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INPUT_PATH_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.JOB_EXEC_ID; +import static com.linkedin.venice.vpj.VenicePushJobConstants.JOB_EXEC_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.JOB_SERVER_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.JOB_STATUS_IN_UNKNOWN_STATE_TIMEOUT_MS; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_COMBINER_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_COMPRESSION_BUILD_NEW_DICT_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_FABRIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_TOPIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.LEGACY_AVRO_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.LEGACY_AVRO_VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.NON_CRITICAL_EXCEPTION; +import static com.linkedin.venice.vpj.VenicePushJobConstants.NOT_SET; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PARENT_CONTROLLER_REGION_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PARTITION_COUNT; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PATH_FILTER; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PERMISSION_700; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PERMISSION_777; +import static com.linkedin.venice.vpj.VenicePushJobConstants.POLL_JOB_STATUS_INTERVAL_MS; +import static com.linkedin.venice.vpj.VenicePushJobConstants.POLL_STATUS_RETRY_ATTEMPTS; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PUSH_JOB_STATUS_UPLOAD_ENABLE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_ENABLE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_SECONDS; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_START_TIMESTAMP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REWIND_EPOCH_TIME_BUFFER_IN_SECONDS_OVERRIDE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REWIND_EPOCH_TIME_IN_SECONDS_OVERRIDE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REWIND_TIME_IN_SECONDS_OVERRIDE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SEND_CONTROL_MESSAGES_DIRECTLY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SORTED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_ETL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_GRID_FABRIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_KAFKA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SUPPRESS_END_OF_PUSH_MESSAGE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SYSTEM_SCHEMA_READER_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.TARGETED_REGION_PUSH_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.TARGETED_REGION_PUSH_LIST; +import static com.linkedin.venice.vpj.VenicePushJobConstants.TEMP_DIR_PREFIX; +import static com.linkedin.venice.vpj.VenicePushJobConstants.UNCREATED_VERSION_NUMBER; +import static com.linkedin.venice.vpj.VenicePushJobConstants.USE_MAPPER_TO_BUILD_DICTIONARY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALIDATE_SCHEMA_AND_BUILD_DICTIONARY_MAPPER_OUTPUT_FILE_EXTENSION; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALIDATE_SCHEMA_AND_BUILD_DICTIONARY_MAPPER_OUTPUT_FILE_PREFIX; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALIDATE_SCHEMA_AND_BUILD_DICT_MAPPER_OUTPUT_DIRECTORY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_DISCOVER_URL_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; import com.linkedin.venice.compression.CompressionStrategy; @@ -104,14 +104,7 @@ import com.linkedin.venice.exceptions.ErrorType; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.exceptions.VeniceResourceAccessException; -import com.linkedin.venice.hadoop.heartbeat.DefaultPushJobHeartbeatSenderFactory; -import com.linkedin.venice.hadoop.heartbeat.NoOpPushJobHeartbeatSender; -import com.linkedin.venice.hadoop.heartbeat.NoOpPushJobHeartbeatSenderFactory; -import com.linkedin.venice.hadoop.heartbeat.PushJobHeartbeatSender; -import com.linkedin.venice.hadoop.heartbeat.PushJobHeartbeatSenderFactory; import com.linkedin.venice.hadoop.input.kafka.KafkaInputDictTrainer; -import com.linkedin.venice.hadoop.jobs.ComputeJob; -import com.linkedin.venice.hadoop.jobs.DataWriterComputeJob; import com.linkedin.venice.hadoop.mapreduce.common.JobUtils; import com.linkedin.venice.hadoop.mapreduce.counter.MRJobCounterHelper; import com.linkedin.venice.hadoop.mapreduce.datawriter.jobs.DataWriterMRJob; @@ -123,6 +116,13 @@ import com.linkedin.venice.hadoop.utils.VPJSSLUtils; import com.linkedin.venice.hadoop.validation.NoOpValidator; import com.linkedin.venice.hadoop.validation.Validator; +import com.linkedin.venice.heartbeat.DefaultPushJobHeartbeatSenderFactory; +import com.linkedin.venice.heartbeat.NoOpPushJobHeartbeatSender; +import com.linkedin.venice.heartbeat.NoOpPushJobHeartbeatSenderFactory; +import com.linkedin.venice.heartbeat.PushJobHeartbeatSender; +import com.linkedin.venice.heartbeat.PushJobHeartbeatSenderFactory; +import com.linkedin.venice.jobs.ComputeJob; +import com.linkedin.venice.jobs.DataWriterComputeJob; import com.linkedin.venice.message.KafkaKey; import com.linkedin.venice.meta.BufferReplayPolicy; import com.linkedin.venice.meta.HybridStoreConfig; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputDictTrainer.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputDictTrainer.java index fad3f0b9b0..0de8397a10 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputDictTrainer.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputDictTrainer.java @@ -1,11 +1,11 @@ package com.linkedin.venice.hadoop.input.kafka; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_DICTIONARY_SAMPLE_SIZE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_DICTIONARY_SIZE_LIMIT; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_SOURCE_TOPIC_CHUNKING_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_TOPIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_SOURCE_KEY_SCHEMA_STRING_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_DICTIONARY_SAMPLE_SIZE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_DICTIONARY_SIZE_LIMIT; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_SOURCE_TOPIC_CHUNKING_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_TOPIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_SOURCE_KEY_SCHEMA_STRING_PROP; import com.github.luben.zstd.ZstdDictTrainer; import com.linkedin.venice.compression.CompressionStrategy; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputFormat.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputFormat.java index 63889814a9..ef2a996df4 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputFormat.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputFormat.java @@ -1,8 +1,8 @@ package com.linkedin.venice.hadoop.input.kafka; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_TOPIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_TOPIC; import com.linkedin.venice.hadoop.input.kafka.avro.KafkaInputMapperKey; import com.linkedin.venice.hadoop.input.kafka.avro.KafkaInputMapperValue; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputFormatCombiner.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputFormatCombiner.java index 90a6e1c12d..9af9742116 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputFormatCombiner.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputFormatCombiner.java @@ -1,7 +1,6 @@ package com.linkedin.venice.hadoop.input.kafka; import com.linkedin.venice.exceptions.VeniceException; -import com.linkedin.venice.hadoop.VenicePushJobConstants; import com.linkedin.venice.hadoop.input.kafka.avro.KafkaInputMapperValue; import com.linkedin.venice.hadoop.input.kafka.avro.MapperValueType; import com.linkedin.venice.hadoop.mapreduce.datawriter.partition.VeniceMRPartitioner; @@ -11,6 +10,7 @@ import com.linkedin.venice.serializer.RecordDeserializer; import com.linkedin.venice.utils.ByteUtils; import com.linkedin.venice.utils.VeniceProperties; +import com.linkedin.venice.vpj.VenicePushJobConstants; import java.io.IOException; import java.util.Iterator; import java.util.Optional; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputRecordReader.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputRecordReader.java index 8250c1d8a7..0e8aa3e9ff 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputRecordReader.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputRecordReader.java @@ -1,7 +1,7 @@ package com.linkedin.venice.hadoop.input.kafka; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_SOURCE_TOPIC_CHUNKING_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_SOURCE_KEY_SCHEMA_STRING_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_SOURCE_TOPIC_CHUNKING_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_SOURCE_KEY_SCHEMA_STRING_PROP; import com.linkedin.venice.chunking.ChunkKeyValueTransformer; import com.linkedin.venice.chunking.ChunkKeyValueTransformerImpl; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputUtils.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputUtils.java index dab08a2a94..c9506eb76a 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputUtils.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputUtils.java @@ -2,11 +2,11 @@ import static com.linkedin.venice.CommonConfigKeys.SSL_FACTORY_CLASS_NAME; import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_CONFIGURATOR_CLASS_CONFIG; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SYSTEM_SCHEMA_CLUSTER_D2_SERVICE_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SYSTEM_SCHEMA_CLUSTER_D2_ZK_HOST; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SYSTEM_SCHEMA_READER_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_CONFIGURATOR_CLASS_CONFIG; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SYSTEM_SCHEMA_CLUSTER_D2_SERVICE_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SYSTEM_SCHEMA_CLUSTER_D2_ZK_HOST; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SYSTEM_SCHEMA_READER_ENABLED; import com.linkedin.d2.balancer.D2Client; import com.linkedin.d2.balancer.D2ClientBuilder; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/VeniceKafkaInputMapper.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/VeniceKafkaInputMapper.java index cd88fe6233..b85558c151 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/VeniceKafkaInputMapper.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/VeniceKafkaInputMapper.java @@ -1,6 +1,6 @@ package com.linkedin.venice.hadoop.input.kafka; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_ENABLE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_ENABLE; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.hadoop.FilterChain; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/VeniceKafkaInputReducer.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/VeniceKafkaInputReducer.java index 1474f411d4..a48ad378fc 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/VeniceKafkaInputReducer.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/VeniceKafkaInputReducer.java @@ -1,12 +1,12 @@ package com.linkedin.venice.hadoop.input.kafka; import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_STRATEGY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_SOURCE_COMPRESSION_STRATEGY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_TOPIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_ENABLE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.TOPIC_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_STRATEGY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_SOURCE_COMPRESSION_STRATEGY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_TOPIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_ENABLE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.TOPIC_PROP; import com.linkedin.venice.compression.CompressionStrategy; import com.linkedin.venice.compression.CompressorFactory; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/ttl/VeniceRmdTTLFilter.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/ttl/VeniceRmdTTLFilter.java index 56ad71081b..80ebc0217b 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/ttl/VeniceRmdTTLFilter.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/kafka/ttl/VeniceRmdTTLFilter.java @@ -1,13 +1,13 @@ package com.linkedin.venice.hadoop.input.kafka.ttl; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_SOURCE_COMPRESSION_STRATEGY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_TOPIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_POLICY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_START_TIMESTAMP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.RMD_SCHEMA_DIR; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_SCHEMA_DIR; import static com.linkedin.venice.schema.rmd.RmdConstants.TIMESTAMP_FIELD_POS; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_SOURCE_COMPRESSION_STRATEGY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_TOPIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_POLICY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_START_TIMESTAMP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.RMD_SCHEMA_DIR; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_SCHEMA_DIR; import com.linkedin.davinci.schema.merge.CollectionTimestampMergeRecordHelper; import com.linkedin.davinci.schema.merge.MergeRecordHelper; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/recordreader/avro/VeniceAvroRecordReader.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/recordreader/avro/VeniceAvroRecordReader.java index bf86d7c873..3bd78cb3a0 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/recordreader/avro/VeniceAvroRecordReader.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/recordreader/avro/VeniceAvroRecordReader.java @@ -1,13 +1,13 @@ package com.linkedin.venice.hadoop.input.recordreader.avro; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_EXTENDED_SCHEMA_VALIDITY_CHECK_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ETL_VALUE_SCHEMA_TRANSFORMATION; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.EXTENDED_SCHEMA_VALIDITY_CHECK_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.GENERATE_PARTIAL_UPDATE_RECORD_FROM_INPUT; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KEY_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SCHEMA_STRING_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.UPDATE_SCHEMA_STRING_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_EXTENDED_SCHEMA_VALIDITY_CHECK_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ETL_VALUE_SCHEMA_TRANSFORMATION; +import static com.linkedin.venice.vpj.VenicePushJobConstants.EXTENDED_SCHEMA_VALIDITY_CHECK_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.GENERATE_PARTIAL_UPDATE_RECORD_FROM_INPUT; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SCHEMA_STRING_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.UPDATE_SCHEMA_STRING_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_FIELD_PROP; import com.linkedin.venice.etl.ETLValueSchemaTransformation; import com.linkedin.venice.schema.AvroSchemaParseUtils; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/recordreader/vson/VeniceVsonRecordReader.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/recordreader/vson/VeniceVsonRecordReader.java index 567532bf8d..ad0a81c408 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/recordreader/vson/VeniceVsonRecordReader.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/recordreader/vson/VeniceVsonRecordReader.java @@ -1,9 +1,9 @@ package com.linkedin.venice.hadoop.input.recordreader.vson; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.FILE_KEY_SCHEMA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.FILE_VALUE_SCHEMA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KEY_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.FILE_KEY_SCHEMA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.FILE_VALUE_SCHEMA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_FIELD_PROP; import com.linkedin.venice.hadoop.exceptions.VeniceSchemaFieldNotFoundException; import com.linkedin.venice.hadoop.input.recordreader.AbstractVeniceRecordReader; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/mapreduce/common/JobUtils.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/mapreduce/common/JobUtils.java index 63fd937554..9d43cc79db 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/mapreduce/common/JobUtils.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/mapreduce/common/JobUtils.java @@ -1,11 +1,11 @@ package com.linkedin.venice.hadoop.mapreduce.common; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_METRIC_COLLECTION_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_CONFIGURATOR_CLASS_CONFIG; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_PASSWORD_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_STORE_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_TRUST_STORE_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_REQUIRED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_METRIC_COLLECTION_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_CONFIGURATOR_CLASS_CONFIG; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_PASSWORD_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_STORE_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_TRUST_STORE_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_REQUIRED; import static org.apache.hadoop.mapreduce.MRJobConfig.MAPREDUCE_JOB_CLASSLOADER; import static org.apache.hadoop.mapreduce.MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY; import static org.apache.hadoop.security.UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/mapreduce/datawriter/jobs/DataWriterMRJob.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/mapreduce/datawriter/jobs/DataWriterMRJob.java index 50889ed089..4a4e37dc8b 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/mapreduce/datawriter/jobs/DataWriterMRJob.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/mapreduce/datawriter/jobs/DataWriterMRJob.java @@ -9,45 +9,45 @@ import static com.linkedin.venice.ConfigKeys.PUSH_JOB_GUID_LEAST_SIGNIFICANT_BITS; import static com.linkedin.venice.ConfigKeys.PUSH_JOB_GUID_MOST_SIGNIFICANT_BITS; import static com.linkedin.venice.VeniceConstants.DEFAULT_SSL_FACTORY_CLASS_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ALLOW_DUPLICATE_KEY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.BATCH_NUM_BYTES_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_STRATEGY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DERIVED_SCHEMA_ID_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ENABLE_WRITE_COMPUTE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ETL_VALUE_SCHEMA_TRANSFORMATION; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.EXTENDED_SCHEMA_VALIDITY_CHECK_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.FILE_KEY_SCHEMA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.FILE_VALUE_SCHEMA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.GENERATE_PARTIAL_UPDATE_RECORD_FROM_INPUT; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_SOURCE_COMPRESSION_STRATEGY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_SOURCE_TOPIC_CHUNKING_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_TOPIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_SECURITY_PROTOCOL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_SOURCE_KEY_SCHEMA_STRING_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KEY_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.MAP_REDUCE_PARTITIONER_CLASS_CONFIG; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PARTITION_COUNT; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REDUCER_SPECULATIVE_EXECUTION_ENABLE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_ENABLE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_POLICY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_START_TIMESTAMP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.RMD_SCHEMA_DIR; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SCHEMA_STRING_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_PREFIX; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.STORAGE_QUOTA_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SYSTEM_SCHEMA_CLUSTER_D2_SERVICE_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SYSTEM_SCHEMA_CLUSTER_D2_ZK_HOST; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SYSTEM_SCHEMA_READER_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.TELEMETRY_MESSAGE_INTERVAL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.TOPIC_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.UPDATE_SCHEMA_STRING_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_SCHEMA_DIR; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_SCHEMA_ID_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VSON_PUSH; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ZSTD_COMPRESSION_LEVEL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_SUCCESS; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ALLOW_DUPLICATE_KEY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.BATCH_NUM_BYTES_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_STRATEGY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DERIVED_SCHEMA_ID_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ENABLE_WRITE_COMPUTE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ETL_VALUE_SCHEMA_TRANSFORMATION; +import static com.linkedin.venice.vpj.VenicePushJobConstants.EXTENDED_SCHEMA_VALIDITY_CHECK_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.FILE_KEY_SCHEMA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.FILE_VALUE_SCHEMA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.GENERATE_PARTIAL_UPDATE_RECORD_FROM_INPUT; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_SOURCE_COMPRESSION_STRATEGY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_SOURCE_TOPIC_CHUNKING_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_TOPIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_SECURITY_PROTOCOL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_SOURCE_KEY_SCHEMA_STRING_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.MAP_REDUCE_PARTITIONER_CLASS_CONFIG; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PARTITION_COUNT; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REDUCER_SPECULATIVE_EXECUTION_ENABLE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_ENABLE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_POLICY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_START_TIMESTAMP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.RMD_SCHEMA_DIR; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SCHEMA_STRING_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_PREFIX; +import static com.linkedin.venice.vpj.VenicePushJobConstants.STORAGE_QUOTA_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SYSTEM_SCHEMA_CLUSTER_D2_SERVICE_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SYSTEM_SCHEMA_CLUSTER_D2_ZK_HOST; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SYSTEM_SCHEMA_READER_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.TELEMETRY_MESSAGE_INTERVAL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.TOPIC_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.UPDATE_SCHEMA_STRING_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_SCHEMA_DIR; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_SCHEMA_ID_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VSON_PUSH; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ZSTD_COMPRESSION_LEVEL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_SUCCESS; import com.github.luben.zstd.Zstd; import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; @@ -64,7 +64,6 @@ import com.linkedin.venice.hadoop.input.kafka.VeniceKafkaInputMapper; import com.linkedin.venice.hadoop.input.kafka.VeniceKafkaInputReducer; import com.linkedin.venice.hadoop.input.kafka.ttl.TTLResolutionPolicy; -import com.linkedin.venice.hadoop.jobs.DataWriterComputeJob; import com.linkedin.venice.hadoop.mapreduce.common.JobUtils; import com.linkedin.venice.hadoop.mapreduce.datawriter.map.VeniceAvroMapper; import com.linkedin.venice.hadoop.mapreduce.datawriter.map.VeniceVsonMapper; @@ -72,6 +71,7 @@ import com.linkedin.venice.hadoop.mapreduce.datawriter.reduce.VeniceReducer; import com.linkedin.venice.hadoop.mapreduce.datawriter.task.CounterBackedMapReduceDataWriterTaskTracker; import com.linkedin.venice.hadoop.task.datawriter.DataWriterTaskTracker; +import com.linkedin.venice.jobs.DataWriterComputeJob; import com.linkedin.venice.utils.ReflectUtils; import com.linkedin.venice.utils.VeniceProperties; import com.linkedin.venice.writer.VeniceWriter; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/ssl/TempFileSSLConfigurator.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/ssl/TempFileSSLConfigurator.java index ecfc798307..279555a395 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/ssl/TempFileSSLConfigurator.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/ssl/TempFileSSLConfigurator.java @@ -11,10 +11,10 @@ import static com.linkedin.venice.CommonConfigKeys.SSL_TRUSTSTORE_PASSWORD; import static com.linkedin.venice.CommonConfigKeys.SSL_TRUSTSTORE_TYPE; import static com.linkedin.venice.ConfigKeys.KAFKA_SECURITY_PROTOCOL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_PASSWORD_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_STORE_PASSWORD_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_STORE_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_TRUST_STORE_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_PASSWORD_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_STORE_PASSWORD_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_STORE_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_TRUST_STORE_PROPERTY_NAME; import com.linkedin.venice.exceptions.VeniceException; import java.io.ByteArrayInputStream; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/task/datawriter/AbstractDataWriterTask.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/task/datawriter/AbstractDataWriterTask.java index c2934ea750..7109634539 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/task/datawriter/AbstractDataWriterTask.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/task/datawriter/AbstractDataWriterTask.java @@ -1,7 +1,7 @@ package com.linkedin.venice.hadoop.task.datawriter; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PARTITION_COUNT; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_CONFIGURATOR_CLASS_CONFIG; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PARTITION_COUNT; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_CONFIGURATOR_CLASS_CONFIG; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.hadoop.ValidateSchemaAndBuildDictMapper; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/task/datawriter/AbstractInputRecordProcessor.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/task/datawriter/AbstractInputRecordProcessor.java index ffaf2bcfd4..e49de59fdc 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/task/datawriter/AbstractInputRecordProcessor.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/task/datawriter/AbstractInputRecordProcessor.java @@ -2,12 +2,12 @@ import static com.linkedin.venice.compression.CompressionStrategy.NO_OP; import static com.linkedin.venice.compression.CompressionStrategy.ZSTD_WITH_DICT; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_METRIC_COLLECTION_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_STRATEGY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.TOPIC_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ZSTD_COMPRESSION_LEVEL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_REQUIRED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_SUCCESS; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_METRIC_COLLECTION_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_STRATEGY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.TOPIC_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ZSTD_COMPRESSION_LEVEL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_REQUIRED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_SUCCESS; import com.github.luben.zstd.Zstd; import com.linkedin.venice.compression.CompressionStrategy; @@ -15,12 +15,12 @@ import com.linkedin.venice.compression.VeniceCompressor; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.hadoop.VenicePushJob; -import com.linkedin.venice.hadoop.VenicePushJobConstants; import com.linkedin.venice.hadoop.input.recordreader.AbstractVeniceRecordReader; import com.linkedin.venice.utils.ByteUtils; import com.linkedin.venice.utils.DictionaryUtils; import com.linkedin.venice.utils.Utils; import com.linkedin.venice.utils.VeniceProperties; +import com.linkedin.venice.vpj.VenicePushJobConstants; import java.io.Closeable; import java.io.IOException; import java.nio.ByteBuffer; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/task/datawriter/AbstractPartitionWriter.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/task/datawriter/AbstractPartitionWriter.java index 9146ec22fd..f8df8c2575 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/task/datawriter/AbstractPartitionWriter.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/task/datawriter/AbstractPartitionWriter.java @@ -2,15 +2,15 @@ import static com.linkedin.venice.ConfigKeys.PUSH_JOB_GUID_LEAST_SIGNIFICANT_BITS; import static com.linkedin.venice.ConfigKeys.PUSH_JOB_GUID_MOST_SIGNIFICANT_BITS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ALLOW_DUPLICATE_KEY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_IS_DUPLICATED_KEY_ALLOWED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DERIVED_SCHEMA_ID_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ENABLE_WRITE_COMPUTE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.STORAGE_QUOTA_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.TELEMETRY_MESSAGE_INTERVAL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.TOPIC_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_SCHEMA_ID_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VSON_PUSH; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ALLOW_DUPLICATE_KEY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_IS_DUPLICATED_KEY_ALLOWED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DERIVED_SCHEMA_ID_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ENABLE_WRITE_COMPUTE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.STORAGE_QUOTA_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.TELEMETRY_MESSAGE_INTERVAL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.TOPIC_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_SCHEMA_ID_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VSON_PUSH; import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; import com.linkedin.venice.ConfigKeys; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/utils/VPJSSLUtils.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/utils/VPJSSLUtils.java index 8c00ab5508..53181deedc 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/utils/VPJSSLUtils.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/utils/VPJSSLUtils.java @@ -1,13 +1,13 @@ package com.linkedin.venice.hadoop.utils; import static com.linkedin.venice.CommonConfigKeys.SSL_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_SECURITY_PROTOCOL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_CONFIGURATOR_CLASS_CONFIG; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_PASSWORD_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_STORE_PASSWORD_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_STORE_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_PREFIX; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_TRUST_STORE_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_SECURITY_PROTOCOL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_CONFIGURATOR_CLASS_CONFIG; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_PASSWORD_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_STORE_PASSWORD_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_STORE_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_PREFIX; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_TRUST_STORE_PROPERTY_NAME; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.hadoop.ssl.SSLConfigurator; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/heartbeat/DefaultPushJobHeartbeatSender.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/heartbeat/DefaultPushJobHeartbeatSender.java similarity index 99% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/heartbeat/DefaultPushJobHeartbeatSender.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/heartbeat/DefaultPushJobHeartbeatSender.java index c7153cbc01..6e86c40680 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/heartbeat/DefaultPushJobHeartbeatSender.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/heartbeat/DefaultPushJobHeartbeatSender.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.heartbeat; +package com.linkedin.venice.heartbeat; import com.linkedin.venice.pubsub.api.PubSubProduceResult; import com.linkedin.venice.pubsub.api.PubSubProducerCallback; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/heartbeat/DefaultPushJobHeartbeatSenderFactory.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/heartbeat/DefaultPushJobHeartbeatSenderFactory.java similarity index 97% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/heartbeat/DefaultPushJobHeartbeatSenderFactory.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/heartbeat/DefaultPushJobHeartbeatSenderFactory.java index 3414e9afb0..87a42be413 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/heartbeat/DefaultPushJobHeartbeatSenderFactory.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/heartbeat/DefaultPushJobHeartbeatSenderFactory.java @@ -1,10 +1,10 @@ -package com.linkedin.venice.hadoop.heartbeat; +package com.linkedin.venice.heartbeat; import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.CONTROLLER_REQUEST_RETRY_ATTEMPTS; import static com.linkedin.venice.status.BatchJobHeartbeatConfigs.HEARTBEAT_INITIAL_DELAY_CONFIG; import static com.linkedin.venice.status.BatchJobHeartbeatConfigs.HEARTBEAT_INTERVAL_CONFIG; import static com.linkedin.venice.status.BatchJobHeartbeatConfigs.HEARTBEAT_LAST_HEARTBEAT_IS_DELETE_CONFIG; +import static com.linkedin.venice.vpj.VenicePushJobConstants.CONTROLLER_REQUEST_RETRY_ATTEMPTS; import com.linkedin.venice.controllerapi.ControllerClient; import com.linkedin.venice.controllerapi.MultiSchemaResponse; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/heartbeat/NoOpPushJobHeartbeatSender.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/heartbeat/NoOpPushJobHeartbeatSender.java similarity index 93% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/heartbeat/NoOpPushJobHeartbeatSender.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/heartbeat/NoOpPushJobHeartbeatSender.java index 3bc5d3a351..02f91a1ec2 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/heartbeat/NoOpPushJobHeartbeatSender.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/heartbeat/NoOpPushJobHeartbeatSender.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.heartbeat; +package com.linkedin.venice.heartbeat; import java.time.Duration; import java.util.Optional; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/heartbeat/NoOpPushJobHeartbeatSenderFactory.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/heartbeat/NoOpPushJobHeartbeatSenderFactory.java similarity index 91% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/heartbeat/NoOpPushJobHeartbeatSenderFactory.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/heartbeat/NoOpPushJobHeartbeatSenderFactory.java index 4c6e4ff08a..5a41dd6460 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/heartbeat/NoOpPushJobHeartbeatSenderFactory.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/heartbeat/NoOpPushJobHeartbeatSenderFactory.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.heartbeat; +package com.linkedin.venice.heartbeat; import com.linkedin.venice.controllerapi.ControllerClient; import com.linkedin.venice.utils.VeniceProperties; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/heartbeat/PushJobHeartbeatSender.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/heartbeat/PushJobHeartbeatSender.java similarity index 93% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/heartbeat/PushJobHeartbeatSender.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/heartbeat/PushJobHeartbeatSender.java index 15cefe5eff..b4f235c937 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/heartbeat/PushJobHeartbeatSender.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/heartbeat/PushJobHeartbeatSender.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.heartbeat; +package com.linkedin.venice.heartbeat; import java.time.Duration; import java.util.Optional; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/heartbeat/PushJobHeartbeatSenderFactory.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/heartbeat/PushJobHeartbeatSenderFactory.java similarity index 92% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/heartbeat/PushJobHeartbeatSenderFactory.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/heartbeat/PushJobHeartbeatSenderFactory.java index 84361d2d3c..92c7062c8b 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/heartbeat/PushJobHeartbeatSenderFactory.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/heartbeat/PushJobHeartbeatSenderFactory.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.heartbeat; +package com.linkedin.venice.heartbeat; import com.linkedin.venice.controllerapi.ControllerClient; import com.linkedin.venice.utils.VeniceProperties; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/jobs/ComputeJob.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/jobs/ComputeJob.java similarity index 96% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/jobs/ComputeJob.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/jobs/ComputeJob.java index 53e98f7470..c389220fdc 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/jobs/ComputeJob.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/jobs/ComputeJob.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.jobs; +package com.linkedin.venice.jobs; import com.linkedin.venice.utils.VeniceProperties; import java.io.Closeable; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/jobs/DataWriterComputeJob.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/jobs/DataWriterComputeJob.java similarity index 99% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/jobs/DataWriterComputeJob.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/jobs/DataWriterComputeJob.java index b9db043521..045d895574 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/jobs/DataWriterComputeJob.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/jobs/DataWriterComputeJob.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.jobs; +package com.linkedin.venice.jobs; import com.linkedin.venice.ConfigKeys; import com.linkedin.venice.exceptions.VeniceException; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/jobs/AbstractDataWriterSparkJob.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/jobs/AbstractDataWriterSparkJob.java index 769e5ee3a7..893b971fe1 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/jobs/AbstractDataWriterSparkJob.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/jobs/AbstractDataWriterSparkJob.java @@ -7,35 +7,6 @@ import static com.linkedin.venice.ConfigKeys.PARTITIONER_CLASS; import static com.linkedin.venice.ConfigKeys.PUSH_JOB_GUID_LEAST_SIGNIFICANT_BITS; import static com.linkedin.venice.ConfigKeys.PUSH_JOB_GUID_MOST_SIGNIFICANT_BITS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ALLOW_DUPLICATE_KEY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.BATCH_NUM_BYTES_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_METRIC_COLLECTION_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_STRATEGY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DERIVED_SCHEMA_ID_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ENABLE_WRITE_COMPUTE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.EXTENDED_SCHEMA_VALIDITY_CHECK_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_SOURCE_COMPRESSION_STRATEGY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_SOURCE_TOPIC_CHUNKING_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_TOPIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_SECURITY_PROTOCOL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PARTITION_COUNT; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_ENABLE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_POLICY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_START_TIMESTAMP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.RMD_SCHEMA_DIR; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_CONFIGURATOR_CLASS_CONFIG; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_PASSWORD_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_STORE_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_PREFIX; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_TRUST_STORE_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.STORAGE_QUOTA_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.TELEMETRY_MESSAGE_INTERVAL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.TOPIC_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_SCHEMA_ID_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ZSTD_COMPRESSION_LEVEL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_REQUIRED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_SUCCESS; import static com.linkedin.venice.spark.SparkConstants.DEFAULT_SCHEMA; import static com.linkedin.venice.spark.SparkConstants.DEFAULT_SCHEMA_WITH_PARTITION; import static com.linkedin.venice.spark.SparkConstants.DEFAULT_SPARK_CLUSTER; @@ -47,6 +18,35 @@ import static com.linkedin.venice.spark.SparkConstants.SPARK_LEADER_CONFIG; import static com.linkedin.venice.spark.SparkConstants.SPARK_SESSION_CONF_PREFIX; import static com.linkedin.venice.spark.SparkConstants.VALUE_COLUMN_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ALLOW_DUPLICATE_KEY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.BATCH_NUM_BYTES_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_METRIC_COLLECTION_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_STRATEGY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DERIVED_SCHEMA_ID_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ENABLE_WRITE_COMPUTE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.EXTENDED_SCHEMA_VALIDITY_CHECK_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_SOURCE_COMPRESSION_STRATEGY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_SOURCE_TOPIC_CHUNKING_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_TOPIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_SECURITY_PROTOCOL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PARTITION_COUNT; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_ENABLE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_POLICY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_START_TIMESTAMP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.RMD_SCHEMA_DIR; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_CONFIGURATOR_CLASS_CONFIG; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_PASSWORD_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_STORE_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_PREFIX; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_TRUST_STORE_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.STORAGE_QUOTA_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.TELEMETRY_MESSAGE_INTERVAL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.TOPIC_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_SCHEMA_ID_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ZSTD_COMPRESSION_LEVEL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_REQUIRED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_SUCCESS; import com.github.luben.zstd.Zstd; import com.linkedin.venice.compression.CompressionStrategy; @@ -54,9 +54,9 @@ import com.linkedin.venice.exceptions.VeniceUnsupportedOperationException; import com.linkedin.venice.hadoop.PushJobSetting; import com.linkedin.venice.hadoop.input.kafka.ttl.TTLResolutionPolicy; -import com.linkedin.venice.hadoop.jobs.DataWriterComputeJob; import com.linkedin.venice.hadoop.ssl.TempFileSSLConfigurator; import com.linkedin.venice.hadoop.task.datawriter.DataWriterTaskTracker; +import com.linkedin.venice.jobs.DataWriterComputeJob; import com.linkedin.venice.spark.datawriter.partition.PartitionSorter; import com.linkedin.venice.spark.datawriter.partition.VeniceSparkPartitioner; import com.linkedin.venice.spark.datawriter.recordprocessor.SparkInputRecordProcessorFactory; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/jobs/DataWriterSparkJob.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/jobs/DataWriterSparkJob.java index 5bf7b5b82b..2288aa370b 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/jobs/DataWriterSparkJob.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/jobs/DataWriterSparkJob.java @@ -1,15 +1,15 @@ package com.linkedin.venice.spark.datawriter.jobs; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ETL_VALUE_SCHEMA_TRANSFORMATION; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.FILE_KEY_SCHEMA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.FILE_VALUE_SCHEMA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.GENERATE_PARTIAL_UPDATE_RECORD_FROM_INPUT; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.INPUT_PATH_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KEY_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SCHEMA_STRING_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.UPDATE_SCHEMA_STRING_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VSON_PUSH; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ETL_VALUE_SCHEMA_TRANSFORMATION; +import static com.linkedin.venice.vpj.VenicePushJobConstants.FILE_KEY_SCHEMA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.FILE_VALUE_SCHEMA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.GENERATE_PARTIAL_UPDATE_RECORD_FROM_INPUT; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INPUT_PATH_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SCHEMA_STRING_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.UPDATE_SCHEMA_STRING_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VSON_PUSH; import com.linkedin.venice.hadoop.PushJobSetting; import com.linkedin.venice.spark.input.hdfs.VeniceHdfsSource; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/partition/VeniceSparkPartitioner.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/partition/VeniceSparkPartitioner.java index ebd9727ac2..afb8c935fc 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/partition/VeniceSparkPartitioner.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/partition/VeniceSparkPartitioner.java @@ -1,6 +1,6 @@ package com.linkedin.venice.spark.datawriter.partition; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PARTITION_COUNT; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PARTITION_COUNT; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.partitioner.VenicePartitioner; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputPartitionReader.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputPartitionReader.java index c48ff758a5..f65672ccef 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputPartitionReader.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputPartitionReader.java @@ -1,6 +1,6 @@ package com.linkedin.venice.spark.input.hdfs; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VSON_PUSH; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VSON_PUSH; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.hadoop.input.recordreader.VeniceRecordIterator; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputScan.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputScan.java index 7dcb56369c..0693b3ca20 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputScan.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/input/hdfs/VeniceHdfsInputScan.java @@ -1,7 +1,7 @@ package com.linkedin.venice.spark.input.hdfs; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.INPUT_PATH_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PATH_FILTER; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INPUT_PATH_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PATH_FILTER; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.spark.SparkConstants; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/VenicePushJobConstants.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/vpj/VenicePushJobConstants.java similarity index 98% rename from clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/VenicePushJobConstants.java rename to clients/venice-push-job/src/main/java/com/linkedin/venice/vpj/VenicePushJobConstants.java index e68d490f41..c6f62f4ee2 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/VenicePushJobConstants.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/vpj/VenicePushJobConstants.java @@ -1,11 +1,13 @@ -package com.linkedin.venice.hadoop; +package com.linkedin.venice.vpj; import static com.linkedin.venice.utils.ByteUtils.BYTES_PER_MB; import com.github.luben.zstd.ZstdDictTrainer; import com.linkedin.venice.compression.CompressionStrategy; -import com.linkedin.venice.hadoop.jobs.DataWriterComputeJob; +import com.linkedin.venice.hadoop.ValidateSchemaAndBuildDictMapper; +import com.linkedin.venice.hadoop.VenicePushJob; import com.linkedin.venice.hadoop.mapreduce.datawriter.map.AbstractVeniceMapper; +import com.linkedin.venice.jobs.DataWriterComputeJob; import com.linkedin.venice.meta.StoreInfo; import com.linkedin.venice.utils.Time; import org.apache.hadoop.fs.PathFilter; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/TestKafkaFormatTopicAutoDiscover.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/TestKafkaFormatTopicAutoDiscover.java index 07d23703c8..c3d84c01ad 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/TestKafkaFormatTopicAutoDiscover.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/TestKafkaFormatTopicAutoDiscover.java @@ -1,18 +1,18 @@ package com.linkedin.venice.hadoop; import static com.linkedin.venice.ConfigKeys.MULTI_REGION; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.D2_ZK_HOSTS_PREFIX; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_TOPIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REWIND_EPOCH_TIME_IN_SECONDS_OVERRIDE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_GRID_FABRIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_KAFKA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_PASSWORD_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_STORE_PASSWORD_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_STORE_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_TRUST_STORE_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_DISCOVER_URL_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_STORE_NAME_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.D2_ZK_HOSTS_PREFIX; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_TOPIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REWIND_EPOCH_TIME_IN_SECONDS_OVERRIDE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_GRID_FABRIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_KAFKA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_PASSWORD_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_STORE_PASSWORD_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_STORE_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_TRUST_STORE_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_DISCOVER_URL_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/TestValidateSchemaAndBuildDictMapper.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/TestValidateSchemaAndBuildDictMapper.java index 02d8f5513b..54cb86d189 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/TestValidateSchemaAndBuildDictMapper.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/TestValidateSchemaAndBuildDictMapper.java @@ -1,13 +1,13 @@ package com.linkedin.venice.hadoop; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_DICTIONARY_SAMPLE_SIZE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ETL_VALUE_SCHEMA_TRANSFORMATION; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.INCREMENTAL_PUSH; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.INPUT_PATH_LAST_MODIFIED_TIME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.INPUT_PATH_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.USE_MAPPER_TO_BUILD_DICTIONARY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_STORE_NAME_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_REQUIRED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_DICTIONARY_SAMPLE_SIZE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ETL_VALUE_SCHEMA_TRANSFORMATION; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INCREMENTAL_PUSH; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INPUT_PATH_LAST_MODIFIED_TIME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INPUT_PATH_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.USE_MAPPER_TO_BUILD_DICTIONARY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_REQUIRED; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; 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 cb6f295b54..3561508cd6 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 @@ -1,23 +1,23 @@ package com.linkedin.venice.hadoop; import static com.linkedin.venice.ConfigKeys.MULTI_REGION; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_METRIC_COLLECTION_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_STRATEGY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.D2_ZK_HOSTS_PREFIX; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.INPUT_PATH_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KEY_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.POLL_JOB_STATUS_INTERVAL_MS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PUSH_JOB_STATUS_UPLOAD_ENABLE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_GRID_FABRIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_PASSWORD_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_STORE_PASSWORD_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_STORE_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_TRUST_STORE_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.USE_MAPPER_TO_BUILD_DICTIONARY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_DISCOVER_URL_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static com.linkedin.venice.utils.DataProviderUtils.allPermutationGenerator; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_METRIC_COLLECTION_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_STRATEGY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.D2_ZK_HOSTS_PREFIX; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INPUT_PATH_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.POLL_JOB_STATUS_INTERVAL_MS; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PUSH_JOB_STATUS_UPLOAD_ENABLE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_GRID_FABRIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_PASSWORD_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_STORE_PASSWORD_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_STORE_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_TRUST_STORE_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.USE_MAPPER_TO_BUILD_DICTIONARY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_DISCOVER_URL_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyBoolean; import static org.mockito.Mockito.anyInt; @@ -40,11 +40,11 @@ import com.linkedin.venice.controllerapi.StoreResponse; import com.linkedin.venice.controllerapi.VersionCreationResponse; import com.linkedin.venice.exceptions.VeniceException; -import com.linkedin.venice.hadoop.jobs.DataWriterComputeJob; import com.linkedin.venice.hadoop.mapreduce.counter.MRJobCounterHelper; import com.linkedin.venice.hadoop.mapreduce.datawriter.task.CounterBackedMapReduceDataWriterTaskTracker; import com.linkedin.venice.hadoop.output.avro.ValidateSchemaAndBuildDictMapperOutput; import com.linkedin.venice.hadoop.task.datawriter.DataWriterTaskTracker; +import com.linkedin.venice.jobs.DataWriterComputeJob; import com.linkedin.venice.message.KafkaKey; import com.linkedin.venice.meta.StoreInfo; import com.linkedin.venice.partitioner.DefaultVenicePartitioner; 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 2a0295fc17..9de5bd827f 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 @@ -2,34 +2,34 @@ import static com.linkedin.venice.ConfigKeys.MULTI_REGION; import static com.linkedin.venice.hadoop.VenicePushJob.getExecutionStatusFromControllerResponse; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.CONTROLLER_REQUEST_RETRY_ATTEMPTS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.D2_ZK_HOSTS_PREFIX; -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.INPUT_PATH_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_TOPIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KEY_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.LEGACY_AVRO_KEY_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.LEGACY_AVRO_VALUE_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PARENT_CONTROLLER_REGION_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_ENABLE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_SECONDS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_START_TIMESTAMP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_ETL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_KAFKA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SYSTEM_SCHEMA_READER_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.TARGETED_REGION_PUSH_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.TARGETED_REGION_PUSH_LIST; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_DISCOVER_URL_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static com.linkedin.venice.status.BatchJobHeartbeatConfigs.HEARTBEAT_ENABLED_CONFIG; import static com.linkedin.venice.utils.ByteUtils.BYTES_PER_MB; import static com.linkedin.venice.utils.TestWriteUtils.NAME_RECORD_V1_SCHEMA; import static com.linkedin.venice.utils.TestWriteUtils.NAME_RECORD_V1_UPDATE_SCHEMA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.CONTROLLER_REQUEST_RETRY_ATTEMPTS; +import static com.linkedin.venice.vpj.VenicePushJobConstants.D2_ZK_HOSTS_PREFIX; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INCREMENTAL_PUSH; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INPUT_PATH_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_TOPIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.LEGACY_AVRO_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.LEGACY_AVRO_VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PARENT_CONTROLLER_REGION_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_ENABLE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_SECONDS; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_START_TIMESTAMP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_ETL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_KAFKA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SYSTEM_SCHEMA_READER_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.TARGETED_REGION_PUSH_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.TARGETED_REGION_PUSH_LIST; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_DISCOVER_URL_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyInt; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputRecordReaderTest.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputRecordReaderTest.java index 37315ae3d8..690b40b5ba 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputRecordReaderTest.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/KafkaInputRecordReaderTest.java @@ -1,9 +1,9 @@ package com.linkedin.venice.hadoop.input.kafka; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_TOPIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_SOURCE_KEY_SCHEMA_STRING_PROP; import static com.linkedin.venice.kafka.protocol.enums.MessageType.PUT; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_TOPIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_SOURCE_KEY_SCHEMA_STRING_PROP; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/TestVeniceKafkaInputMapper.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/TestVeniceKafkaInputMapper.java index 04dff2d4b8..b5a0c34a2c 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/TestVeniceKafkaInputMapper.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/TestVeniceKafkaInputMapper.java @@ -1,13 +1,13 @@ package com.linkedin.venice.hadoop.input.kafka; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_SOURCE_COMPRESSION_STRATEGY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_TOPIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_ENABLE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_POLICY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_START_TIMESTAMP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.RMD_SCHEMA_DIR; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_SCHEMA_DIR; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_SOURCE_COMPRESSION_STRATEGY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_TOPIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_ENABLE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_POLICY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_START_TIMESTAMP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.RMD_SCHEMA_DIR; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_SCHEMA_DIR; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/TestVeniceKafkaInputReducer.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/TestVeniceKafkaInputReducer.java index 1d9a21f5b9..08c489fecb 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/TestVeniceKafkaInputReducer.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/TestVeniceKafkaInputReducer.java @@ -1,18 +1,18 @@ package com.linkedin.venice.hadoop.input.kafka; import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_STRATEGY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_SOURCE_COMPRESSION_STRATEGY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_TOPIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_ENABLE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_POLICY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_START_TIMESTAMP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.RMD_SCHEMA_DIR; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.TOPIC_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_SCHEMA_DIR; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_SCHEMA_ID_PROP; import static com.linkedin.venice.hadoop.mapreduce.datawriter.reduce.VeniceReducer.MAP_REDUCE_JOB_ID_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_STRATEGY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_SOURCE_COMPRESSION_STRATEGY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_TOPIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_ENABLE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_POLICY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_START_TIMESTAMP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.RMD_SCHEMA_DIR; +import static com.linkedin.venice.vpj.VenicePushJobConstants.TOPIC_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_SCHEMA_DIR; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_SCHEMA_ID_PROP; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/ttl/TestVeniceChunkedPayloadTTLFilter.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/ttl/TestVeniceChunkedPayloadTTLFilter.java index 4f0360ebd9..b072c63411 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/ttl/TestVeniceChunkedPayloadTTLFilter.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/ttl/TestVeniceChunkedPayloadTTLFilter.java @@ -1,19 +1,19 @@ package com.linkedin.venice.hadoop.input.kafka.ttl; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_SOURCE_COMPRESSION_STRATEGY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_TOPIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_ENABLE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_POLICY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_START_TIMESTAMP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.RMD_SCHEMA_DIR; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_SCHEMA_DIR; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static com.linkedin.venice.schema.rmd.v1.CollectionRmdTimestamp.PUT_ONLY_PART_LENGTH_FIELD_NAME; import static com.linkedin.venice.schema.rmd.v1.CollectionRmdTimestamp.TOP_LEVEL_COLO_ID_FIELD_NAME; import static com.linkedin.venice.schema.rmd.v1.CollectionRmdTimestamp.TOP_LEVEL_TS_FIELD_NAME; import static com.linkedin.venice.utils.ChunkingTestUtils.createChunkBytes; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_SOURCE_COMPRESSION_STRATEGY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_TOPIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_ENABLE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_POLICY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_START_TIMESTAMP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.RMD_SCHEMA_DIR; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_SCHEMA_DIR; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/ttl/TestVeniceKafkaInputTTLFilter.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/ttl/TestVeniceKafkaInputTTLFilter.java index 0e6613c27f..00471e7867 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/ttl/TestVeniceKafkaInputTTLFilter.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/kafka/ttl/TestVeniceKafkaInputTTLFilter.java @@ -1,17 +1,17 @@ package com.linkedin.venice.hadoop.input.kafka.ttl; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_SOURCE_COMPRESSION_STRATEGY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_TOPIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_ENABLE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_POLICY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_START_TIMESTAMP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.RMD_SCHEMA_DIR; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_SCHEMA_DIR; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static com.linkedin.venice.schema.rmd.RmdConstants.REPLICATION_CHECKPOINT_VECTOR_FIELD_NAME; import static com.linkedin.venice.schema.rmd.RmdConstants.TIMESTAMP_FIELD_NAME; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_SOURCE_COMPRESSION_STRATEGY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_TOPIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_ENABLE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_POLICY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_START_TIMESTAMP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.RMD_SCHEMA_DIR; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_SCHEMA_DIR; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/mapreduce/AbstractTestVeniceMR.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/mapreduce/AbstractTestVeniceMR.java index b5d6d9dd5c..f3b5878c33 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/mapreduce/AbstractTestVeniceMR.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/mapreduce/AbstractTestVeniceMR.java @@ -1,22 +1,22 @@ package com.linkedin.venice.hadoop.mapreduce; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ALLOW_DUPLICATE_KEY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_METRIC_COLLECTION_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_STRATEGY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KEY_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PARTITION_COUNT; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SCHEMA_STRING_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_CONFIGURATOR_CLASS_CONFIG; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_STORE_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_TRUST_STORE_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.STORAGE_ENGINE_OVERHEAD_RATIO; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.STORAGE_QUOTA_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.TOPIC_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.USE_MAPPER_TO_BUILD_DICTIONARY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_SCHEMA_ID_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_REQUIRED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_SUCCESS; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ALLOW_DUPLICATE_KEY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_METRIC_COLLECTION_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_STRATEGY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PARTITION_COUNT; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SCHEMA_STRING_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_CONFIGURATOR_CLASS_CONFIG; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_STORE_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_TRUST_STORE_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.STORAGE_ENGINE_OVERHEAD_RATIO; +import static com.linkedin.venice.vpj.VenicePushJobConstants.STORAGE_QUOTA_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.TOPIC_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.USE_MAPPER_TO_BUILD_DICTIONARY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_SCHEMA_ID_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_REQUIRED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_SUCCESS; import com.linkedin.venice.compression.CompressionStrategy; import com.linkedin.venice.hadoop.mapreduce.datawriter.reduce.VeniceReducer; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/mapreduce/datawriter/map/TestVeniceAvroMapper.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/mapreduce/datawriter/map/TestVeniceAvroMapper.java index f6bfcd9ea1..f59b01b8c6 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/mapreduce/datawriter/map/TestVeniceAvroMapper.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/mapreduce/datawriter/map/TestVeniceAvroMapper.java @@ -1,12 +1,12 @@ package com.linkedin.venice.hadoop.mapreduce.datawriter.map; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_METRIC_COLLECTION_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_STRATEGY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.STORAGE_ENGINE_OVERHEAD_RATIO; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.STORAGE_QUOTA_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.TOPIC_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_REQUIRED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_SUCCESS; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_METRIC_COLLECTION_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_STRATEGY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.STORAGE_ENGINE_OVERHEAD_RATIO; +import static com.linkedin.venice.vpj.VenicePushJobConstants.STORAGE_QUOTA_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.TOPIC_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_REQUIRED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ZSTD_DICTIONARY_CREATION_SUCCESS; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyLong; import static org.mockito.Mockito.anyString; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/mapreduce/datawriter/map/TestVeniceVsonMapper.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/mapreduce/datawriter/map/TestVeniceVsonMapper.java index 540bd50fcf..d2b1ca2051 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/mapreduce/datawriter/map/TestVeniceVsonMapper.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/mapreduce/datawriter/map/TestVeniceVsonMapper.java @@ -1,9 +1,9 @@ package com.linkedin.venice.hadoop.mapreduce.datawriter.map; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.FILE_KEY_SCHEMA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.FILE_VALUE_SCHEMA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KEY_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.FILE_KEY_SCHEMA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.FILE_VALUE_SCHEMA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_FIELD_PROP; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/mapreduce/datawriter/reduce/TestVeniceReducer.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/mapreduce/datawriter/reduce/TestVeniceReducer.java index 66dd49bdfc..1e2d1e0298 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/mapreduce/datawriter/reduce/TestVeniceReducer.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/mapreduce/datawriter/reduce/TestVeniceReducer.java @@ -1,12 +1,12 @@ package com.linkedin.venice.hadoop.mapreduce.datawriter.reduce; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ALLOW_DUPLICATE_KEY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DERIVED_SCHEMA_ID_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ENABLE_WRITE_COMPUTE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.STORAGE_QUOTA_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.TELEMETRY_MESSAGE_INTERVAL; import static com.linkedin.venice.hadoop.mapreduce.counter.MRJobCounterHelper.TOTAL_KEY_SIZE_GROUP_COUNTER_NAME; import static com.linkedin.venice.hadoop.mapreduce.counter.MRJobCounterHelper.TOTAL_VALUE_SIZE_GROUP_COUNTER_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ALLOW_DUPLICATE_KEY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DERIVED_SCHEMA_ID_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ENABLE_WRITE_COMPUTE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.STORAGE_QUOTA_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.TELEMETRY_MESSAGE_INTERVAL; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyInt; import static org.mockito.Mockito.anyLong; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/recordreader/avro/TestVeniceAvroRecordReader.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/recordreader/avro/TestVeniceAvroRecordReader.java index ab2e4787eb..4fc3473c26 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/recordreader/avro/TestVeniceAvroRecordReader.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/recordreader/avro/TestVeniceAvroRecordReader.java @@ -3,13 +3,13 @@ import static com.linkedin.venice.etl.ETLValueSchemaTransformation.ADD_NULL_TO_UNION; import static com.linkedin.venice.etl.ETLValueSchemaTransformation.NONE; import static com.linkedin.venice.etl.ETLValueSchemaTransformation.UNIONIZE_WITH_NULL; -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.utils.DataProviderUtils.BOOLEAN; import static com.linkedin.venice.utils.TestWriteUtils.INT_SCHEMA; import static com.linkedin.venice.utils.TestWriteUtils.NAME_RECORD_V2_SCHEMA; import static com.linkedin.venice.utils.TestWriteUtils.STRING_SCHEMA; import static com.linkedin.venice.utils.TestWriteUtils.STRING_TO_NAME_RECORD_V1_SCHEMA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; import com.linkedin.avroutil1.compatibility.RandomRecordGenerator; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/ssl/TestTempFileSSLConfigurator.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/ssl/TestTempFileSSLConfigurator.java index 00289f5305..b4e3fc86a3 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/ssl/TestTempFileSSLConfigurator.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/ssl/TestTempFileSSLConfigurator.java @@ -1,9 +1,9 @@ package com.linkedin.venice.hadoop.ssl; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_PASSWORD_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_STORE_PASSWORD_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_STORE_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_TRUST_STORE_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_PASSWORD_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_STORE_PASSWORD_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_STORE_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_TRUST_STORE_PROPERTY_NAME; import com.linkedin.venice.CommonConfigKeys; import com.linkedin.venice.ConfigKeys; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/utils/TestVPJSSLUtils.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/utils/TestVPJSSLUtils.java index 9ee542c4d1..35071c7964 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/utils/TestVPJSSLUtils.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/utils/TestVPJSSLUtils.java @@ -1,9 +1,9 @@ package com.linkedin.venice.hadoop.utils; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_PASSWORD_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_STORE_PASSWORD_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_STORE_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_TRUST_STORE_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_PASSWORD_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_STORE_PASSWORD_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_STORE_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_TRUST_STORE_PROPERTY_NAME; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.utils.VeniceProperties; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/heartbeat/TestPushJobHeartbeatSender.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/heartbeat/TestPushJobHeartbeatSender.java similarity index 98% rename from clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/heartbeat/TestPushJobHeartbeatSender.java rename to clients/venice-push-job/src/test/java/com/linkedin/venice/heartbeat/TestPushJobHeartbeatSender.java index 1a4a273e4e..a69967c731 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/heartbeat/TestPushJobHeartbeatSender.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/heartbeat/TestPushJobHeartbeatSender.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.hadoop.heartbeat; +package com.linkedin.venice.heartbeat; import static org.mockito.Mockito.*; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/jobs/DataWriterComputeJobTest.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/jobs/DataWriterComputeJobTest.java similarity index 87% rename from clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/jobs/DataWriterComputeJobTest.java rename to clients/venice-push-job/src/test/java/com/linkedin/venice/jobs/DataWriterComputeJobTest.java index 58a9bcc3ef..9056f35a73 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/jobs/DataWriterComputeJobTest.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/jobs/DataWriterComputeJobTest.java @@ -1,6 +1,6 @@ -package com.linkedin.venice.hadoop.jobs; +package com.linkedin.venice.jobs; -import static com.linkedin.venice.hadoop.jobs.DataWriterComputeJob.PASS_THROUGH_CONFIG_PREFIXES; +import static com.linkedin.venice.jobs.DataWriterComputeJob.PASS_THROUGH_CONFIG_PREFIXES; import com.linkedin.venice.exceptions.VeniceException; import org.testng.annotations.Test; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/datawriter/jobs/AbstractDataWriterSparkJobTest.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/datawriter/jobs/AbstractDataWriterSparkJobTest.java index a6e6628988..e371038f0c 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/datawriter/jobs/AbstractDataWriterSparkJobTest.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/datawriter/jobs/AbstractDataWriterSparkJobTest.java @@ -1,14 +1,14 @@ package com.linkedin.venice.spark.datawriter.jobs; import static com.linkedin.venice.ConfigKeys.KAFKA_CONFIG_PREFIX; -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.meta.Store.UNLIMITED_STORAGE_QUOTA; import static com.linkedin.venice.spark.SparkConstants.KEY_COLUMN_NAME; import static com.linkedin.venice.spark.SparkConstants.SPARK_APP_NAME_CONFIG; import static com.linkedin.venice.spark.SparkConstants.SPARK_DATA_WRITER_CONF_PREFIX; import static com.linkedin.venice.spark.SparkConstants.SPARK_SESSION_CONF_PREFIX; import static com.linkedin.venice.spark.SparkConstants.VALUE_COLUMN_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; import static org.apache.spark.sql.types.DataTypes.BinaryType; import static org.apache.spark.sql.types.DataTypes.StringType; @@ -16,7 +16,7 @@ import com.linkedin.venice.etl.ETLValueSchemaTransformation; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.hadoop.PushJobSetting; -import com.linkedin.venice.hadoop.jobs.DataWriterComputeJob; +import com.linkedin.venice.jobs.DataWriterComputeJob; import com.linkedin.venice.meta.Version; import com.linkedin.venice.partitioner.DefaultVenicePartitioner; import com.linkedin.venice.serializer.FastSerializerDeserializerFactory; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/datawriter/partition/VeniceSparkPartitionerTest.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/datawriter/partition/VeniceSparkPartitionerTest.java index b7d78153e2..f209661ac4 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/datawriter/partition/VeniceSparkPartitionerTest.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/datawriter/partition/VeniceSparkPartitionerTest.java @@ -1,6 +1,6 @@ package com.linkedin.venice.spark.datawriter.partition; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PARTITION_COUNT; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PARTITION_COUNT; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.partitioner.DefaultVenicePartitioner; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/input/hdfs/TestSparkInputFromHdfs.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/input/hdfs/TestSparkInputFromHdfs.java index c47def0872..927bb29057 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/input/hdfs/TestSparkInputFromHdfs.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/input/hdfs/TestSparkInputFromHdfs.java @@ -1,22 +1,22 @@ package com.linkedin.venice.spark.input.hdfs; -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.FILE_KEY_SCHEMA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.FILE_VALUE_SCHEMA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.GENERATE_PARTIAL_UPDATE_RECORD_FROM_INPUT; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.INPUT_PATH_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KEY_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SCHEMA_STRING_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.UPDATE_SCHEMA_STRING_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VSON_PUSH; import static com.linkedin.venice.spark.SparkConstants.DEFAULT_SCHEMA; import static com.linkedin.venice.spark.input.hdfs.VeniceHdfsInputTable.INPUT_TABLE_NAME; import static com.linkedin.venice.utils.TestWriteUtils.DEFAULT_USER_DATA_VALUE_PREFIX; import static com.linkedin.venice.utils.TestWriteUtils.NAME_RECORD_V2_SCHEMA; import static com.linkedin.venice.utils.TestWriteUtils.STRING_TO_STRING_SCHEMA; import static com.linkedin.venice.utils.TestWriteUtils.writeSimpleAvroFileWithStringToNameRecordV1Schema; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.FILE_KEY_SCHEMA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.FILE_VALUE_SCHEMA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.GENERATE_PARTIAL_UPDATE_RECORD_FROM_INPUT; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INPUT_PATH_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SCHEMA_STRING_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.UPDATE_SCHEMA_STRING_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VSON_PUSH; import com.linkedin.venice.schema.vson.VsonAvroSchemaAdapter; import com.linkedin.venice.schema.vson.VsonAvroSerializer; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/zstd/TestZstdLibrary.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/zstd/TestZstdLibrary.java index 748643c7af..0c1b942006 100644 --- a/clients/venice-push-job/src/test/java/com/linkedin/venice/zstd/TestZstdLibrary.java +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/zstd/TestZstdLibrary.java @@ -1,23 +1,23 @@ package com.linkedin.venice.zstd; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_DICTIONARY_SAMPLE_SIZE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_DICTIONARY_SIZE_LIMIT; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PATH_FILTER; import static com.linkedin.venice.utils.ByteUtils.BYTES_PER_KB; import static com.linkedin.venice.utils.ByteUtils.BYTES_PER_MB; import static com.linkedin.venice.utils.TestWriteUtils.writeSimpleAvroFileWithStringToStringSchema; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_DICTIONARY_SAMPLE_SIZE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_DICTIONARY_SIZE_LIMIT; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PATH_FILTER; import com.github.luben.zstd.ZstdDictTrainer; import com.github.luben.zstd.ZstdException; import com.linkedin.venice.etl.ETLValueSchemaTransformation; import com.linkedin.venice.hadoop.InputDataInfoProvider; import com.linkedin.venice.hadoop.PushJobZstdConfig; -import com.linkedin.venice.hadoop.VenicePushJobConstants; import com.linkedin.venice.hadoop.input.recordreader.avro.HdfsAvroUtils; import com.linkedin.venice.hadoop.input.recordreader.avro.VeniceAvroFileIterator; import com.linkedin.venice.hadoop.input.recordreader.avro.VeniceAvroRecordReader; import com.linkedin.venice.utils.Utils; import com.linkedin.venice.utils.VeniceProperties; +import com.linkedin.venice.vpj.VenicePushJobConstants; import java.io.File; import java.nio.ByteBuffer; import java.util.Properties; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/ConsumerIntegrationTestWithSchemaReader.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/ConsumerIntegrationTestWithSchemaReader.java index 27d51cfe3c..9c37f93e01 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/ConsumerIntegrationTestWithSchemaReader.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/ConsumerIntegrationTestWithSchemaReader.java @@ -1,10 +1,10 @@ package com.linkedin.venice.consumer; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_FABRIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_KAFKA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SYSTEM_SCHEMA_READER_ENABLED; import static com.linkedin.venice.utils.IntegrationTestPushUtils.defaultVPJProps; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_FABRIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_KAFKA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SYSTEM_SCHEMA_READER_ENABLED; import com.linkedin.venice.controllerapi.ControllerClient; import com.linkedin.venice.controllerapi.MultiSchemaResponse; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java index bbd160d99f..8a52c8ae55 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/consumer/TestBootstrappingChangelogConsumer.java @@ -5,8 +5,6 @@ import static com.linkedin.venice.ConfigKeys.CLUSTER_NAME; import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; 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; import static com.linkedin.venice.integration.utils.VeniceClusterWrapperConstants.DEFAULT_PARENT_DATA_CENTER_REGION_NAME; import static com.linkedin.venice.integration.utils.VeniceControllerWrapper.D2_SERVICE_NAME; import static com.linkedin.venice.utils.IntegrationTestPushUtils.createStoreForJob; @@ -15,6 +13,8 @@ import static com.linkedin.venice.utils.IntegrationTestPushUtils.sendStreamingRecord; import static com.linkedin.venice.utils.IntegrationTestPushUtils.sendStreamingRecordWithLogicalTimestamp; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; import com.linkedin.davinci.consumer.BootstrappingVeniceChangelogConsumer; import com.linkedin.davinci.consumer.ChangeEvent; 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 64ed5190b1..8251325809 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 @@ -7,13 +7,6 @@ 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; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_ENABLE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REWIND_TIME_IN_SECONDS_OVERRIDE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_KAFKA; import static com.linkedin.venice.integration.utils.VeniceClusterWrapperConstants.DEFAULT_PARENT_DATA_CENTER_REGION_NAME; import static com.linkedin.venice.integration.utils.VeniceControllerWrapper.D2_SERVICE_NAME; import static com.linkedin.venice.utils.IntegrationTestPushUtils.createStoreForJob; @@ -24,6 +17,13 @@ import static com.linkedin.venice.utils.TestWriteUtils.NAME_RECORD_V1_SCHEMA; import static com.linkedin.venice.utils.TestWriteUtils.NAME_RECORD_V2_SCHEMA; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_ENABLE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REWIND_TIME_IN_SECONDS_OVERRIDE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_KAFKA; import com.linkedin.davinci.consumer.BootstrappingVeniceChangelogConsumer; import com.linkedin.davinci.consumer.ChangeEvent; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestParentControllerWithMultiDataCenter.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestParentControllerWithMultiDataCenter.java index f6be499c5a..489200f156 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestParentControllerWithMultiDataCenter.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestParentControllerWithMultiDataCenter.java @@ -3,9 +3,9 @@ import static com.linkedin.venice.ConfigKeys.DEFAULT_MAX_NUMBER_OF_PARTITIONS; import static com.linkedin.venice.ConfigKeys.DEFAULT_NUMBER_OF_PARTITION_FOR_HYBRID; import static com.linkedin.venice.ConfigKeys.DEFAULT_PARTITION_SIZE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFER_VERSION_SWAP; import static com.linkedin.venice.utils.IntegrationTestPushUtils.createStoreForJob; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFER_VERSION_SWAP; import static org.testng.Assert.assertEquals; import static org.testng.AssertJUnit.fail; 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 dda4f1fb6d..7aef12f2ff 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 @@ -13,13 +13,13 @@ 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; import static com.linkedin.venice.utils.IntegrationTestPushUtils.createStoreForJob; import static com.linkedin.venice.utils.IntegrationTestPushUtils.defaultVPJProps; import static com.linkedin.venice.utils.IntegrationTestPushUtils.runVPJ; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PUSH_JOB_STATUS_UPLOAD_ENABLE; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientTest.java index fadeb88cb7..40e6a543f3 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientTest.java @@ -21,7 +21,6 @@ import static com.linkedin.venice.ConfigKeys.SERVER_INGESTION_ISOLATION_SERVICE_PORT; import static com.linkedin.venice.ConfigKeys.SERVER_PROMOTION_TO_LEADER_REPLICA_DELAY_SECONDS; import static com.linkedin.venice.ConfigKeys.VENICE_PARTITIONERS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static com.linkedin.venice.integration.utils.VeniceClusterWrapper.DEFAULT_KEY_SCHEMA; import static com.linkedin.venice.integration.utils.VeniceClusterWrapper.DEFAULT_VALUE_SCHEMA; import static com.linkedin.venice.meta.PersistenceType.ROCKS_DB; @@ -29,6 +28,7 @@ import static com.linkedin.venice.utils.IntegrationTestPushUtils.defaultVPJProps; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; import static com.linkedin.venice.utils.TestWriteUtils.writeSimpleAvroFileWithIntToStringSchema; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; 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 d242d7a60b..b6930ba933 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 @@ -1,18 +1,6 @@ package com.linkedin.venice.endToEnd; import static com.linkedin.davinci.stats.HostLevelIngestionStats.ASSEMBLED_RMD_SIZE_IN_BYTES; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DATA_WRITER_COMPUTE_JOB_CLASS; -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.ENABLE_WRITE_COMPUTE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.INCREMENTAL_PUSH; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_ENABLE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_START_TIMESTAMP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REWIND_TIME_IN_SECONDS_OVERRIDE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_KAFKA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static com.linkedin.venice.integration.utils.VeniceControllerWrapper.PARENT_D2_SERVICE_NAME; import static com.linkedin.venice.samza.VeniceSystemFactory.DEPLOYMENT_ID; import static com.linkedin.venice.samza.VeniceSystemFactory.VENICE_AGGREGATE; @@ -35,6 +23,18 @@ import static com.linkedin.venice.utils.TestWriteUtils.writeSimpleAvroFileWithStringToNameRecordV1Schema; import static com.linkedin.venice.utils.TestWriteUtils.writeSimpleAvroFileWithStringToPartialUpdateOpRecordSchema; import static com.linkedin.venice.utils.TestWriteUtils.writeSimpleAvroFileWithStringToUserWithStringMapSchema; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DATA_WRITER_COMPUTE_JOB_CLASS; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ENABLE_WRITE_COMPUTE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INCREMENTAL_PUSH; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_ENABLE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_START_TIMESTAMP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REWIND_TIME_IN_SECONDS_OVERRIDE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_KAFKA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; 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 631403a962..b0f068dca5 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 @@ -5,11 +5,11 @@ 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.hadoop.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PUSH_JOB_STATUS_UPLOAD_ENABLE; import static com.linkedin.venice.utils.IntegrationTestPushUtils.defaultVPJProps; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PUSH_JOB_STATUS_UPLOAD_ENABLE; 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; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/PushStatusStoreTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/PushStatusStoreTest.java index 760e8f2119..1d1d6dbb20 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/PushStatusStoreTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/PushStatusStoreTest.java @@ -8,12 +8,12 @@ import static com.linkedin.venice.ConfigKeys.SERVER_PROMOTION_TO_LEADER_REPLICA_DELAY_SECONDS; import static com.linkedin.venice.ConfigKeys.USE_PUSH_STATUS_STORE_FOR_INCREMENTAL_PUSH; import static com.linkedin.venice.common.PushStatusStoreUtils.SERVER_INCREMENTAL_PUSH_PREFIX; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.INCREMENTAL_PUSH; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static com.linkedin.venice.integration.utils.VeniceClusterWrapper.DEFAULT_KEY_SCHEMA; import static com.linkedin.venice.utils.IntegrationTestPushUtils.defaultVPJProps; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; import static com.linkedin.venice.utils.TestWriteUtils.writeSimpleAvroFileWithIntToStringSchema; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INCREMENTAL_PUSH; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; 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 ebfdfcb419..28f8f290b7 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 @@ -7,14 +7,6 @@ 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; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REPUSH_TTL_ENABLE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.REWIND_TIME_IN_SECONDS_OVERRIDE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_KAFKA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.TARGETED_REGION_PUSH_ENABLED; import static com.linkedin.venice.integration.utils.VeniceClusterWrapperConstants.DEFAULT_PARENT_DATA_CENTER_REGION_NAME; import static com.linkedin.venice.utils.IntegrationTestPushUtils.createStoreForJob; import static com.linkedin.venice.utils.IntegrationTestPushUtils.getSamzaProducerConfig; @@ -23,6 +15,14 @@ import static com.linkedin.venice.utils.TestUtils.generateInput; import static com.linkedin.venice.utils.TestWriteUtils.STRING_SCHEMA; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_ENABLE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.REWIND_TIME_IN_SECONDS_OVERRIDE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_KAFKA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.TARGETED_REGION_PUSH_ENABLED; import com.linkedin.davinci.kafka.consumer.KafkaConsumerServiceDelegator; import com.linkedin.venice.ConfigKeys; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveReplicationForIncPush.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveReplicationForIncPush.java index fe99cab0ec..430a28dff2 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveReplicationForIncPush.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveReplicationForIncPush.java @@ -8,14 +8,14 @@ import static com.linkedin.venice.ConfigKeys.SERVER_DATABASE_SYNC_BYTES_INTERNAL_FOR_DEFERRED_WRITE_MODE; import static com.linkedin.venice.ConfigKeys.SERVER_DEDICATED_CONSUMER_POOL_FOR_AA_WC_LEADER_ENABLED; import static com.linkedin.venice.ConfigKeys.SERVER_PROMOTION_TO_LEADER_REPLICA_DELAY_SECONDS; -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.SEND_CONTROL_MESSAGES_DIRECTLY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_GRID_FABRIC; import static com.linkedin.venice.utils.TestUtils.assertCommand; import static com.linkedin.venice.utils.TestUtils.waitForNonDeterministicAssertion; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INCREMENTAL_PUSH; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SEND_CONTROL_MESSAGES_DIRECTLY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_GRID_FABRIC; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestBackupVersionDatabaseOptimization.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestBackupVersionDatabaseOptimization.java index c8fe0be16f..9b3ab2ff9c 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestBackupVersionDatabaseOptimization.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestBackupVersionDatabaseOptimization.java @@ -4,11 +4,11 @@ import static com.linkedin.venice.ConfigKeys.SERVER_OPTIMIZE_DATABASE_FOR_BACKUP_VERSION_ENABLED; import static com.linkedin.venice.ConfigKeys.SERVER_OPTIMIZE_DATABASE_FOR_BACKUP_VERSION_NO_READ_THRESHOLD_SECONDS; import static com.linkedin.venice.ConfigKeys.SERVER_OPTIMIZE_DATABASE_SERVICE_SCHEDULE_INTERNAL_SECONDS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static com.linkedin.venice.utils.IntegrationTestPushUtils.createStoreForJob; import static com.linkedin.venice.utils.IntegrationTestPushUtils.defaultVPJProps; import static com.linkedin.venice.utils.TestWriteUtils.DEFAULT_USER_DATA_RECORD_COUNT; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestBatch.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestBatch.java index 833d1d11a4..81a6945c05 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestBatch.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestBatch.java @@ -3,22 +3,6 @@ import static com.linkedin.davinci.stats.HostLevelIngestionStats.ASSEMBLED_RECORD_SIZE_IN_BYTES; import static com.linkedin.davinci.stats.HostLevelIngestionStats.ASSEMBLED_RECORD_SIZE_RATIO; import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ALLOW_DUPLICATE_KEY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_METRIC_COLLECTION_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DATA_WRITER_COMPUTE_JOB_CLASS; -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.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_COMBINER_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_TOPIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SEND_CONTROL_MESSAGES_DIRECTLY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_ETL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_KAFKA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.USE_MAPPER_TO_BUILD_DICTIONARY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_STORE_NAME_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ZSTD_COMPRESSION_LEVEL; import static com.linkedin.venice.system.store.MetaStoreWriter.KEY_STRING_STORE_NAME; import static com.linkedin.venice.system.store.MetaStoreWriter.KEY_STRING_VERSION_NUMBER; import static com.linkedin.venice.utils.ByteUtils.BYTES_PER_MB; @@ -50,6 +34,22 @@ import static com.linkedin.venice.utils.TestWriteUtils.writeSimpleAvroFileWithDuplicateKey; import static com.linkedin.venice.utils.TestWriteUtils.writeSimpleAvroFileWithStringToStringSchema; import static com.linkedin.venice.utils.TestWriteUtils.writeSimpleAvroFileWithStringToStringSchema2; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ALLOW_DUPLICATE_KEY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_METRIC_COLLECTION_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DATA_WRITER_COMPUTE_JOB_CLASS; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INCREMENTAL_PUSH; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_COMBINER_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_TOPIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SEND_CONTROL_MESSAGES_DIRECTLY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_ETL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_KAFKA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.USE_MAPPER_TO_BUILD_DICTIONARY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ZSTD_COMPRESSION_LEVEL; import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; import com.linkedin.venice.client.exceptions.VeniceClientException; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestBatchReportIncrementalPush.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestBatchReportIncrementalPush.java index 7a21e688c4..fc1e92ed25 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestBatchReportIncrementalPush.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestBatchReportIncrementalPush.java @@ -8,13 +8,13 @@ import static com.linkedin.venice.ConfigKeys.PUSH_STATUS_STORE_ENABLED; import static com.linkedin.venice.ConfigKeys.SERVER_BATCH_REPORT_END_OF_INCREMENTAL_PUSH_STATUS_ENABLED; import static com.linkedin.venice.ConfigKeys.SERVER_PROMOTION_TO_LEADER_REPLICA_DELAY_SECONDS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ENABLE_WRITE_COMPUTE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.INCREMENTAL_PUSH; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static com.linkedin.venice.utils.TestUtils.assertCommand; import static com.linkedin.venice.utils.TestWriteUtils.STRING_SCHEMA; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; import static com.linkedin.venice.utils.TestWriteUtils.writeSimpleAvroFileWithStringToPartialUpdateOpRecordSchema; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ENABLE_WRITE_COMPUTE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INCREMENTAL_PUSH; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestEmptyPush.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestEmptyPush.java index 1567b54b65..3a5db98724 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestEmptyPush.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestEmptyPush.java @@ -1,8 +1,6 @@ package com.linkedin.venice.endToEnd; import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.COMPRESSION_METRIC_COLLECTION_ENABLED; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SEND_CONTROL_MESSAGES_DIRECTLY; import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE; import static com.linkedin.venice.utils.IntegrationTestPushUtils.defaultVPJProps; import static com.linkedin.venice.utils.IntegrationTestPushUtils.runVPJ; @@ -11,6 +9,8 @@ import static com.linkedin.venice.utils.TestWriteUtils.USER_SCHEMA; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; import static com.linkedin.venice.utils.TestWriteUtils.writeEmptyAvroFile; +import static com.linkedin.venice.vpj.VenicePushJobConstants.COMPRESSION_METRIC_COLLECTION_ENABLED; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SEND_CONTROL_MESSAGES_DIRECTLY; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertNotNull; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestHybrid.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestHybrid.java index 25a17052c5..a1e11f0df2 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestHybrid.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestHybrid.java @@ -12,7 +12,6 @@ import static com.linkedin.venice.ConfigKeys.SERVER_PROMOTION_TO_LEADER_REPLICA_DELAY_SECONDS; import static com.linkedin.venice.ConfigKeys.SERVER_SHARED_CONSUMER_ASSIGNMENT_STRATEGY; import static com.linkedin.venice.ConfigKeys.SSL_TO_KAFKA_LEGACY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFER_VERSION_SWAP; import static com.linkedin.venice.integration.utils.VeniceClusterWrapper.DEFAULT_KEY_SCHEMA; import static com.linkedin.venice.integration.utils.VeniceClusterWrapper.DEFAULT_VALUE_SCHEMA; import static com.linkedin.venice.meta.BufferReplayPolicy.REWIND_FROM_EOP; @@ -28,6 +27,7 @@ import static com.linkedin.venice.utils.IntegrationTestPushUtils.sendStreamingRecord; import static com.linkedin.venice.utils.TestWriteUtils.STRING_SCHEMA; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFER_VERSION_SWAP; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestLeaderReplicaFailover.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestLeaderReplicaFailover.java index 237663a4ab..078a22d157 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestLeaderReplicaFailover.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestLeaderReplicaFailover.java @@ -3,10 +3,10 @@ import static com.linkedin.davinci.store.rocksdb.RocksDBServerConfig.ROCKSDB_PLAIN_TABLE_FORMAT_ENABLED; import static com.linkedin.venice.ConfigKeys.DEFAULT_OFFLINE_PUSH_STRATEGY; import static com.linkedin.venice.ConfigKeys.DEFAULT_PARTITION_SIZE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; import static com.linkedin.venice.utils.IntegrationTestPushUtils.createStoreForJob; import static com.linkedin.venice.utils.IntegrationTestPushUtils.defaultVPJProps; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestPushJobVersionCleanup.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestPushJobVersionCleanup.java index 06d23bccad..b945dd5f48 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestPushJobVersionCleanup.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestPushJobVersionCleanup.java @@ -3,11 +3,11 @@ import static com.linkedin.venice.ConfigKeys.ADMIN_HELIX_MESSAGING_CHANNEL_ENABLED; import static com.linkedin.venice.ConfigKeys.PARTICIPANT_MESSAGE_CONSUMPTION_DELAY_MS; import static com.linkedin.venice.ConfigKeys.PARTICIPANT_MESSAGE_STORE_ENABLED; -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.SEND_CONTROL_MESSAGES_DIRECTLY; import static com.linkedin.venice.utils.IntegrationTestPushUtils.createStoreForJob; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SEND_CONTROL_MESSAGES_DIRECTLY; import com.linkedin.venice.controllerapi.ControllerClient; import com.linkedin.venice.controllerapi.UpdateStoreQueryParams; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestPushJobWithEmergencySourceRegionSelection.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestPushJobWithEmergencySourceRegionSelection.java index c9b3096788..a4113f88b7 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestPushJobWithEmergencySourceRegionSelection.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestPushJobWithEmergencySourceRegionSelection.java @@ -6,11 +6,11 @@ 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.ConfigKeys.SERVER_PROMOTION_TO_LEADER_REPLICA_DELAY_SECONDS; -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.SEND_CONTROL_MESSAGES_DIRECTLY; import static com.linkedin.venice.utils.IntegrationTestPushUtils.createStoreForJob; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SEND_CONTROL_MESSAGES_DIRECTLY; import com.linkedin.venice.client.store.AvroGenericStoreClient; import com.linkedin.venice.client.store.ClientConfig; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestPushJobWithNativeReplication.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestPushJobWithNativeReplication.java index 2d3accea01..c3540d0916 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestPushJobWithNativeReplication.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestPushJobWithNativeReplication.java @@ -12,18 +12,6 @@ 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.ConfigKeys.SERVER_PROMOTION_TO_LEADER_REPLICA_DELAY_SECONDS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_RE_PUSH_REWIND_IN_SECONDS_OVERRIDE; -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.INPUT_PATH_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_TOPIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PUSH_JOB_STATUS_UPLOAD_ENABLE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SEND_CONTROL_MESSAGES_DIRECTLY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_KAFKA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.TARGETED_REGION_PUSH_ENABLED; import static com.linkedin.venice.integration.utils.VeniceControllerWrapper.D2_SERVICE_NAME; import static com.linkedin.venice.integration.utils.VeniceControllerWrapper.PARENT_D2_SERVICE_NAME; import static com.linkedin.venice.meta.PersistenceType.ROCKS_DB; @@ -43,6 +31,18 @@ import static com.linkedin.venice.utils.TestUtils.waitForNonDeterministicAssertion; import static com.linkedin.venice.utils.TestWriteUtils.STRING_SCHEMA; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_RE_PUSH_REWIND_IN_SECONDS_OVERRIDE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INCREMENTAL_PUSH; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INPUT_PATH_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_TOPIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PUSH_JOB_STATUS_UPLOAD_ENABLE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SEND_CONTROL_MESSAGES_DIRECTLY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_KAFKA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.TARGETED_REGION_PUSH_ENABLED; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestPushJobWithSourceGridFabricSelection.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestPushJobWithSourceGridFabricSelection.java index 80bef8122f..08f1862b5e 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestPushJobWithSourceGridFabricSelection.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestPushJobWithSourceGridFabricSelection.java @@ -4,12 +4,12 @@ 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.ConfigKeys.SERVER_PROMOTION_TO_LEADER_REPLICA_DELAY_SECONDS; -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.SEND_CONTROL_MESSAGES_DIRECTLY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_GRID_FABRIC; import static com.linkedin.venice.utils.IntegrationTestPushUtils.createStoreForJob; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SEND_CONTROL_MESSAGES_DIRECTLY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_GRID_FABRIC; import com.linkedin.venice.client.store.AvroGenericStoreClient; import com.linkedin.venice.client.store.ClientConfig; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestStoreBackupVersionDeletion.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestStoreBackupVersionDeletion.java index 84a424b0cd..a988b7488f 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestStoreBackupVersionDeletion.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestStoreBackupVersionDeletion.java @@ -6,10 +6,10 @@ import static com.linkedin.venice.ConfigKeys.DEFAULT_PARTITION_SIZE; import static com.linkedin.venice.ConfigKeys.SERVER_PROMOTION_TO_LEADER_REPLICA_DELAY_SECONDS; import static com.linkedin.venice.ConfigKeys.TOPIC_CLEANUP_SLEEP_INTERVAL_BETWEEN_TOPIC_LIST_FETCH_MS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_KAFKA; import static com.linkedin.venice.utils.IntegrationTestPushUtils.createStoreForJob; import static com.linkedin.venice.utils.TestWriteUtils.NAME_RECORD_V3_SCHEMA; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_KAFKA; import com.linkedin.venice.controller.StoreBackupVersionCleanupService; import com.linkedin.venice.controller.VeniceHelixAdmin; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestStoreMigration.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestStoreMigration.java index d4d3701c49..1748ecdc11 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestStoreMigration.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestStoreMigration.java @@ -7,14 +7,14 @@ import static com.linkedin.venice.ConfigKeys.SERVER_PROMOTION_TO_LEADER_REPLICA_DELAY_SECONDS; import static com.linkedin.venice.ConfigKeys.SERVER_QUOTA_ENFORCEMENT_ENABLED; import static com.linkedin.venice.ConfigKeys.TOPIC_CLEANUP_SLEEP_INTERVAL_BETWEEN_TOPIC_LIST_FETCH_MS; -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.SEND_CONTROL_MESSAGES_DIRECTLY; import static com.linkedin.venice.system.store.MetaStoreWriter.KEY_STRING_CLUSTER_NAME; import static com.linkedin.venice.system.store.MetaStoreWriter.KEY_STRING_STORE_NAME; import static com.linkedin.venice.utils.IntegrationTestPushUtils.getSamzaProducer; import static com.linkedin.venice.utils.IntegrationTestPushUtils.sendStreamingRecord; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SEND_CONTROL_MESSAGES_DIRECTLY; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestSuperSetSchemaRegistration.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestSuperSetSchemaRegistration.java index 8708d1a85c..7259546355 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestSuperSetSchemaRegistration.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestSuperSetSchemaRegistration.java @@ -1,9 +1,9 @@ package com.linkedin.venice.endToEnd; -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.utils.IntegrationTestPushUtils.createStoreForJob; import static com.linkedin.venice.utils.IntegrationTestPushUtils.defaultVPJProps; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; import com.linkedin.venice.controllerapi.ControllerClient; import com.linkedin.venice.controllerapi.UpdateStoreQueryParams; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestTopicWiseSharedConsumerPoolResilience.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestTopicWiseSharedConsumerPoolResilience.java index 1f3e5694c1..9ad1201ca8 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestTopicWiseSharedConsumerPoolResilience.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestTopicWiseSharedConsumerPoolResilience.java @@ -6,10 +6,10 @@ import static com.linkedin.venice.ConfigKeys.PARTICIPANT_MESSAGE_STORE_ENABLED; import static com.linkedin.venice.ConfigKeys.SERVER_CONSUMER_POOL_SIZE_PER_KAFKA_CLUSTER; import static com.linkedin.venice.ConfigKeys.SERVER_SHARED_CONSUMER_ASSIGNMENT_STRATEGY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static com.linkedin.venice.utils.IntegrationTestPushUtils.createStoreForJob; import static com.linkedin.venice.utils.IntegrationTestPushUtils.defaultVPJProps; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import com.linkedin.davinci.kafka.consumer.KafkaConsumerService; import com.linkedin.davinci.kafka.consumer.ParticipantStoreConsumptionTask; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestVsonStoreBatch.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestVsonStoreBatch.java index 952c8aedc9..2b2642c730 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestVsonStoreBatch.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestVsonStoreBatch.java @@ -1,12 +1,5 @@ package com.linkedin.venice.endToEnd; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DATA_WRITER_COMPUTE_JOB_CLASS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_TOPIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KEY_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_KAFKA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_FIELD_PROP; import static com.linkedin.venice.utils.IntegrationTestPushUtils.createStoreForJob; import static com.linkedin.venice.utils.IntegrationTestPushUtils.defaultVPJPropsWithoutD2Routing; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; @@ -16,6 +9,13 @@ import static com.linkedin.venice.utils.TestWriteUtils.writeSimpleVsonFile; import static com.linkedin.venice.utils.TestWriteUtils.writeSimpleVsonFileWithUserSchema; import static com.linkedin.venice.utils.TestWriteUtils.writeVsonByteAndShort; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DATA_WRITER_COMPUTE_JOB_CLASS; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_TOPIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_KAFKA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_FIELD_PROP; import com.linkedin.venice.client.store.AvroGenericStoreClient; import com.linkedin.venice.client.store.ClientConfig; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/hadoop/TestVenicePushJob.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/hadoop/TestVenicePushJob.java index 2d1366b5e1..e8a6a49770 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/hadoop/TestVenicePushJob.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/hadoop/TestVenicePushJob.java @@ -2,28 +2,28 @@ import static com.linkedin.venice.ConfigKeys.VENICE_PARTITIONERS; import static com.linkedin.venice.hadoop.VenicePushJob.getLatestPathOfInputDirectory; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.CONTROLLER_REQUEST_RETRY_ATTEMPTS; -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.DEFER_VERSION_SWAP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ENABLE_WRITE_COMPUTE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.INCREMENTAL_PUSH; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_FABRIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_TOPIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KEY_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.MAP_REDUCE_PARTITIONER_CLASS_CONFIG; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SEND_CONTROL_MESSAGES_DIRECTLY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_KAFKA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SUPPRESS_END_OF_PUSH_MESSAGE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_FIELD_PROP; import static com.linkedin.venice.utils.IntegrationTestPushUtils.createStoreForJob; import static com.linkedin.venice.utils.IntegrationTestPushUtils.defaultVPJProps; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; import static com.linkedin.venice.utils.TestWriteUtils.writeSimpleAvroFileWithStringToStringSchema; import static com.linkedin.venice.utils.TestWriteUtils.writeSimpleAvroFileWithStringToStringSchema2; import static com.linkedin.venice.utils.TestWriteUtils.writeSimpleVsonFileWithUserSchema; +import static com.linkedin.venice.vpj.VenicePushJobConstants.CONTROLLER_REQUEST_RETRY_ATTEMPTS; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFER_VERSION_SWAP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ENABLE_WRITE_COMPUTE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INCREMENTAL_PUSH; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_FABRIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_TOPIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.MAP_REDUCE_PARTITIONER_CLASS_CONFIG; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SEND_CONTROL_MESSAGES_DIRECTLY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_KAFKA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SUPPRESS_END_OF_PUSH_MESSAGE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_FIELD_PROP; import com.linkedin.venice.client.store.AvroGenericStoreClient; import com.linkedin.venice.client.store.ClientConfig; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/hadoop/input/kafka/TestKafkaInputFormat.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/hadoop/input/kafka/TestKafkaInputFormat.java index 373141ed47..0424e185f0 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/hadoop/input/kafka/TestKafkaInputFormat.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/hadoop/input/kafka/TestKafkaInputFormat.java @@ -1,9 +1,9 @@ package com.linkedin.venice.hadoop.input.kafka; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_TOPIC; import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_MAX_RECORDS_PER_MAPPER; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_TOPIC; import com.linkedin.venice.integration.utils.PubSubBrokerWrapper; import com.linkedin.venice.integration.utils.ServiceFactory; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/hadoop/input/kafka/TestKafkaInputRecordReader.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/hadoop/input/kafka/TestKafkaInputRecordReader.java index 6f1a63ecae..d7ea9d07ff 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/hadoop/input/kafka/TestKafkaInputRecordReader.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/hadoop/input/kafka/TestKafkaInputRecordReader.java @@ -1,9 +1,9 @@ package com.linkedin.venice.hadoop.input.kafka; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_TOPIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_SOURCE_KEY_SCHEMA_STRING_PROP; import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_TOPIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_SOURCE_KEY_SCHEMA_STRING_PROP; import com.linkedin.venice.hadoop.input.kafka.avro.KafkaInputMapperKey; import com.linkedin.venice.hadoop.input.kafka.avro.KafkaInputMapperValue; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/ingestionHeartbeat/IngestionHeartBeatTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/ingestionHeartbeat/IngestionHeartBeatTest.java index bf1edcc0ee..33c105272b 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/ingestionHeartbeat/IngestionHeartBeatTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/ingestionHeartbeat/IngestionHeartBeatTest.java @@ -1,14 +1,14 @@ package com.linkedin.venice.ingestionHeartbeat; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.INCREMENTAL_PUSH; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static com.linkedin.venice.message.KafkaKey.HEART_BEAT; import static com.linkedin.venice.pubsub.api.PubSubMessageHeaders.VENICE_LEADER_COMPLETION_STATE_HEADER; import static com.linkedin.venice.utils.TestUtils.assertCommand; import static com.linkedin.venice.utils.TestWriteUtils.NAME_RECORD_V1_SCHEMA; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; import static com.linkedin.venice.utils.TestWriteUtils.writeSimpleAvroFileWithStringToNameRecordV1Schema; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INCREMENTAL_PUSH; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static com.linkedin.venice.writer.LeaderCompleteState.LEADER_COMPLETED; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/kafka/EndToEndKafkaWithSASLTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/kafka/EndToEndKafkaWithSASLTest.java index 325c892865..d0f4a5be93 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/kafka/EndToEndKafkaWithSASLTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/kafka/EndToEndKafkaWithSASLTest.java @@ -1,12 +1,12 @@ package com.linkedin.venice.kafka; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; import static com.linkedin.venice.integration.utils.VeniceClusterWrapperConstants.STANDALONE_REGION_NAME; import static com.linkedin.venice.utils.IntegrationTestPushUtils.createStoreForJob; import static com.linkedin.venice.utils.IntegrationTestPushUtils.defaultVPJProps; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; import static com.linkedin.venice.utils.TestWriteUtils.loadFileAsString; import static com.linkedin.venice.utils.TestWriteUtils.writeSchemaWithUnknownFieldIntoAvroFile; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; import com.google.common.collect.ImmutableMap; import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/kafka/ssl/TestProduceWithSSL.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/kafka/ssl/TestProduceWithSSL.java index ab18c4feb7..d916d800e5 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/kafka/ssl/TestProduceWithSSL.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/kafka/ssl/TestProduceWithSSL.java @@ -1,15 +1,15 @@ package com.linkedin.venice.kafka.ssl; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.ENABLE_SSL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_KAFKA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_PASSWORD_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_STORE_PASSWORD_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_STORE_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_TRUST_STORE_PROPERTY_NAME; import static com.linkedin.venice.utils.IntegrationTestPushUtils.createStoreForJob; import static com.linkedin.venice.utils.IntegrationTestPushUtils.sslVPJProps; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.vpj.VenicePushJobConstants.ENABLE_SSL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_BROKER_URL; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_KAFKA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_PASSWORD_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_STORE_PASSWORD_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_STORE_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_TRUST_STORE_PROPERTY_NAME; import com.linkedin.venice.client.store.AvroGenericStoreClient; import com.linkedin.venice.client.store.ClientConfig; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/multicluster/TestMetadataOperationInMultiCluster.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/multicluster/TestMetadataOperationInMultiCluster.java index 99e78ae76c..f254d7fbd4 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/multicluster/TestMetadataOperationInMultiCluster.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/multicluster/TestMetadataOperationInMultiCluster.java @@ -1,9 +1,9 @@ package com.linkedin.venice.multicluster; -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.VENICE_STORE_NAME_PROP; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import com.linkedin.venice.controllerapi.ControllerClient; import com.linkedin.venice.controllerapi.ControllerResponse; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/restart/TestRestartServerAfterDeletingSstFilesWithActiveActiveIngestion.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/restart/TestRestartServerAfterDeletingSstFilesWithActiveActiveIngestion.java index 6d16863aa1..f0b0b4a133 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/restart/TestRestartServerAfterDeletingSstFilesWithActiveActiveIngestion.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/restart/TestRestartServerAfterDeletingSstFilesWithActiveActiveIngestion.java @@ -5,11 +5,11 @@ import static com.linkedin.venice.ConfigKeys.PERSISTENCE_TYPE; 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.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; import static com.linkedin.venice.integration.utils.VeniceClusterWrapperConstants.DEFAULT_PARENT_DATA_CENTER_REGION_NAME; import static com.linkedin.venice.utils.IntegrationTestPushUtils.createStoreForJob; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/restart/TestRestartServerDuringIngestion.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/restart/TestRestartServerDuringIngestion.java index b0c1b8754d..73520599a9 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/restart/TestRestartServerDuringIngestion.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/restart/TestRestartServerDuringIngestion.java @@ -1,9 +1,9 @@ package com.linkedin.venice.restart; import static com.linkedin.venice.ConfigKeys.MULTI_REGION; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_DISCOVER_URL_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static com.linkedin.venice.utils.TestUtils.generateInput; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_DISCOVER_URL_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; import com.linkedin.venice.ConfigKeys; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/utils/IntegrationTestPushUtils.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/utils/IntegrationTestPushUtils.java index 236c6424a4..c846b7664f 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/utils/IntegrationTestPushUtils.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/utils/IntegrationTestPushUtils.java @@ -4,15 +4,6 @@ import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; import static com.linkedin.venice.ConfigKeys.MULTI_REGION; import static com.linkedin.venice.VeniceConstants.DEFAULT_PER_ROUTER_READ_QUOTA; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.D2_ZK_HOSTS_PREFIX; -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.KEY_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PARENT_CONTROLLER_REGION_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_GRID_FABRIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_FIELD_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_DISCOVER_URL_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static com.linkedin.venice.integration.utils.VeniceControllerWrapper.D2_SERVICE_NAME; import static com.linkedin.venice.integration.utils.VeniceControllerWrapper.PARENT_D2_SERVICE_NAME; import static com.linkedin.venice.samza.VeniceSystemFactory.DEPLOYMENT_ID; @@ -25,6 +16,15 @@ import static com.linkedin.venice.samza.VeniceSystemFactory.VENICE_PARENT_D2_ZK_HOSTS; import static com.linkedin.venice.samza.VeniceSystemFactory.VENICE_PUSH_TYPE; import static com.linkedin.venice.samza.VeniceSystemFactory.VENICE_STORE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.D2_ZK_HOSTS_PREFIX; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PARENT_CONTROLLER_REGION_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_GRID_FABRIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_DISCOVER_URL_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import com.linkedin.venice.compression.CompressionStrategy; import com.linkedin.venice.controllerapi.ControllerClient; diff --git a/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/KeyAndValueSchemas.java b/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/KeyAndValueSchemas.java index 051acfdbf4..47b364d30a 100644 --- a/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/KeyAndValueSchemas.java +++ b/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/KeyAndValueSchemas.java @@ -1,7 +1,7 @@ package com.linkedin.venice.utils; -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.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; import org.apache.avro.Schema; diff --git a/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/PushInputSchemaBuilder.java b/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/PushInputSchemaBuilder.java index 8f25d8c41a..a1a316bdd4 100644 --- a/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/PushInputSchemaBuilder.java +++ b/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/PushInputSchemaBuilder.java @@ -1,7 +1,7 @@ package com.linkedin.venice.utils; -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.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; import java.util.ArrayList; diff --git a/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/TestWriteUtils.java b/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/TestWriteUtils.java index 7f6dccf732..16ef0b5a48 100644 --- a/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/TestWriteUtils.java +++ b/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/TestWriteUtils.java @@ -1,23 +1,23 @@ package com.linkedin.venice.utils; import static com.linkedin.venice.ConfigKeys.MULTI_REGION; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.CONTROLLER_REQUEST_RETRY_ATTEMPTS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.D2_ZK_HOSTS_PREFIX; -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.INPUT_PATH_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KEY_INPUT_FILE_DATA_SIZE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.KEY_ZSTD_COMPRESSION_DICTIONARY; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PARENT_CONTROLLER_REGION_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.POLL_JOB_STATUS_INTERVAL_MS; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.PUSH_JOB_STATUS_UPLOAD_ENABLE; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SOURCE_GRID_FABRIC; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_PASSWORD_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_STORE_PASSWORD_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_KEY_STORE_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.SSL_TRUST_STORE_PROPERTY_NAME; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_DISCOVER_URL_PROP; -import static com.linkedin.venice.hadoop.VenicePushJobConstants.VENICE_STORE_NAME_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.CONTROLLER_REQUEST_RETRY_ATTEMPTS; +import static com.linkedin.venice.vpj.VenicePushJobConstants.D2_ZK_HOSTS_PREFIX; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INPUT_PATH_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KEY_INPUT_FILE_DATA_SIZE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.KEY_ZSTD_COMPRESSION_DICTIONARY; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PARENT_CONTROLLER_REGION_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.POLL_JOB_STATUS_INTERVAL_MS; +import static com.linkedin.venice.vpj.VenicePushJobConstants.PUSH_JOB_STATUS_UPLOAD_ENABLE; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_GRID_FABRIC; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_PASSWORD_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_STORE_PASSWORD_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_KEY_STORE_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SSL_TRUST_STORE_PROPERTY_NAME; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_DISCOVER_URL_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; import com.linkedin.avroutil1.compatibility.RandomRecordGenerator; From e684eaa87bc5d0770a77253be70685dc149917e0 Mon Sep 17 00:00:00 2001 From: eldernewborn Date: Fri, 20 Sep 2024 08:56:37 -0700 Subject: [PATCH 03/16] [build] Redistribute some of the integration test load to make things faster. (#1188) * Split some of the longer integration tests into smaller chunks * Revamped the test grouping and naming convention. * Added test-suite level time reporting to assist with future rebalances and expose cost of running each test. * Captured and regrouped some of the "other" tests into additional groups. --- .github/workflows/VeniceCI-E2ETests.yml | 620 +++++++++++++++++++++-- internal/venice-test-common/build.gradle | 126 +++-- 2 files changed, 671 insertions(+), 75 deletions(-) diff --git a/.github/workflows/VeniceCI-E2ETests.yml b/.github/workflows/VeniceCI-E2ETests.yml index 6aeec810f1..4207e4ef89 100644 --- a/.github/workflows/VeniceCI-E2ETests.yml +++ b/.github/workflows/VeniceCI-E2ETests.yml @@ -13,7 +13,7 @@ name: E2ETests on: [push, pull_request, workflow_dispatch] jobs: - IntegrationTestsA: + IntegrationTests_1000: strategy: fail-fast: false matrix: @@ -38,7 +38,7 @@ jobs: - name: Build with Gradle uses: gradle/gradle-build-action@v2 with: - arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTestA" + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1000" - name: Package Build Artifacts if: success() || failure() shell: bash @@ -55,7 +55,7 @@ jobs: path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz retention-days: 30 - IntegrationTestsB: + IntegrationTests_1010: strategy: fail-fast: false matrix: @@ -80,7 +80,7 @@ jobs: - name: Build with Gradle uses: gradle/gradle-build-action@v2 with: - arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTestB" + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1010" - name: Package Build Artifacts if: success() || failure() shell: bash @@ -97,7 +97,7 @@ jobs: path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz retention-days: 30 - IntegrationTestsC: + IntegrationTests_1020: strategy: fail-fast: false matrix: @@ -122,7 +122,7 @@ jobs: - name: Build with Gradle uses: gradle/gradle-build-action@v2 with: - arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTestC" + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1020" - name: Package Build Artifacts if: success() || failure() shell: bash @@ -139,7 +139,7 @@ jobs: path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz retention-days: 30 - IntegrationTestsD: + IntegrationTests_1030: strategy: fail-fast: false matrix: @@ -164,7 +164,7 @@ jobs: - name: Build with Gradle uses: gradle/gradle-build-action@v2 with: - arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTestD" + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1030" - name: Package Build Artifacts if: success() || failure() shell: bash @@ -181,7 +181,7 @@ jobs: path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz retention-days: 30 - IntegrationTestsE: + IntegrationTests_1040: strategy: fail-fast: false matrix: @@ -206,7 +206,7 @@ jobs: - name: Build with Gradle uses: gradle/gradle-build-action@v2 with: - arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTestE" + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1040" - name: Package Build Artifacts if: success() || failure() shell: bash @@ -223,7 +223,7 @@ jobs: path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz retention-days: 30 - IntegrationTestsF: + IntegrationTests_1050: strategy: fail-fast: false matrix: @@ -248,7 +248,7 @@ jobs: - name: Build with Gradle uses: gradle/gradle-build-action@v2 with: - arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTestF" + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1050" - name: Package Build Artifacts if: success() || failure() shell: bash @@ -265,7 +265,7 @@ jobs: path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz retention-days: 30 - IntegrationTestsG: + IntegrationTests_1060: strategy: fail-fast: false matrix: @@ -290,7 +290,7 @@ jobs: - name: Build with Gradle uses: gradle/gradle-build-action@v2 with: - arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTestG" + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1060" - name: Package Build Artifacts if: success() || failure() shell: bash @@ -307,7 +307,7 @@ jobs: path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz retention-days: 30 - IntegrationTestsH: + IntegrationTests_1070: strategy: fail-fast: false matrix: @@ -332,7 +332,7 @@ jobs: - name: Build with Gradle uses: gradle/gradle-build-action@v2 with: - arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTestH" + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1070" - name: Package Build Artifacts if: success() || failure() shell: bash @@ -349,7 +349,7 @@ jobs: path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz retention-days: 30 - IntegrationTestsI: + IntegrationTests_1080: strategy: fail-fast: false matrix: @@ -374,7 +374,7 @@ jobs: - name: Build with Gradle uses: gradle/gradle-build-action@v2 with: - arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTestI" + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1080" - name: Package Build Artifacts if: success() || failure() shell: bash @@ -391,7 +391,7 @@ jobs: path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz retention-days: 30 - IntegrationTestsJ: + IntegrationTests_1090: strategy: fail-fast: false matrix: @@ -416,7 +416,7 @@ jobs: - name: Build with Gradle uses: gradle/gradle-build-action@v2 with: - arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTestJ" + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1090" - name: Package Build Artifacts if: success() || failure() shell: bash @@ -433,7 +433,7 @@ jobs: path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz retention-days: 30 - IntegrationTestsK: + IntegrationTests_1100: strategy: fail-fast: false matrix: @@ -458,7 +458,7 @@ jobs: - name: Build with Gradle uses: gradle/gradle-build-action@v2 with: - arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTestK" + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1100" - name: Package Build Artifacts if: success() || failure() shell: bash @@ -475,7 +475,7 @@ jobs: path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz retention-days: 30 - IntegrationTestsL: + IntegrationTests_1110: strategy: fail-fast: false matrix: @@ -500,7 +500,7 @@ jobs: - name: Build with Gradle uses: gradle/gradle-build-action@v2 with: - arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTestL" + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1110" - name: Package Build Artifacts if: success() || failure() shell: bash @@ -517,7 +517,7 @@ jobs: path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz retention-days: 30 - IntegrationTestsM: + IntegrationTests_1120: strategy: fail-fast: false matrix: @@ -542,7 +542,7 @@ jobs: - name: Build with Gradle uses: gradle/gradle-build-action@v2 with: - arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTestM" + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1120" - name: Package Build Artifacts if: success() || failure() shell: bash @@ -559,7 +559,7 @@ jobs: path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz retention-days: 30 - IntegrationTestsN: + IntegrationTests_1130: strategy: fail-fast: false matrix: @@ -584,7 +584,7 @@ jobs: - name: Build with Gradle uses: gradle/gradle-build-action@v2 with: - arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTestN" + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1130" - name: Package Build Artifacts if: success() || failure() shell: bash @@ -601,7 +601,7 @@ jobs: path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz retention-days: 30 - IntegrationTestsO: + IntegrationTests_1200: strategy: fail-fast: false matrix: @@ -626,7 +626,7 @@ jobs: - name: Build with Gradle uses: gradle/gradle-build-action@v2 with: - arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTestO" + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1200" - name: Package Build Artifacts if: success() || failure() shell: bash @@ -643,7 +643,7 @@ jobs: path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz retention-days: 30 - IntegrationTestsP: + IntegrationTests_1210: strategy: fail-fast: false matrix: @@ -668,7 +668,7 @@ jobs: - name: Build with Gradle uses: gradle/gradle-build-action@v2 with: - arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTestP" + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1210" - name: Package Build Artifacts if: success() || failure() shell: bash @@ -685,7 +685,7 @@ jobs: path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz retention-days: 30 - IntegrationTestsQ: + IntegrationTests_1220: strategy: fail-fast: false matrix: @@ -710,7 +710,7 @@ jobs: - name: Build with Gradle uses: gradle/gradle-build-action@v2 with: - arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTestQ" + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1220" - name: Package Build Artifacts if: success() || failure() shell: bash @@ -727,7 +727,7 @@ jobs: path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz retention-days: 30 - IntegrationTestsZ: + IntegrationTests_1230: strategy: fail-fast: false matrix: @@ -752,7 +752,553 @@ jobs: - name: Build with Gradle uses: gradle/gradle-build-action@v2 with: - arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTestZ" + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1230" + - name: Package Build Artifacts + if: success() || failure() + shell: bash + run: | + mkdir ${{ github.job }}-artifacts + find . -path "**/build/reports/*" -or -path "**/build/test-results/*" > artifacts.list + rsync -R --files-from=artifacts.list . ${{ github.job }}-artifacts + tar -zcvf ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz ${{ github.job }}-artifacts + - name: Upload Build Artifacts + if: success() || failure() + uses: actions/upload-artifact@v3 + with: + name: ${{ github.job }} + path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz + retention-days: 30 + + IntegrationTests_1240: + strategy: + fail-fast: false + matrix: + jdk: [17] + runs-on: ubuntu-latest + timeout-minutes: 120 + steps: + - uses: actions/checkout@v4 + with: + fetch-depth: 0 + - name: Set up JDK + uses: actions/setup-java@v3 + with: + java-version: ${{ matrix.jdk }} + distribution: 'temurin' + cache: 'gradle' + - shell: bash + run: | + git remote set-head origin --auto + git remote add upstream https://github.com/linkedin/venice + git fetch upstream + - name: Build with Gradle + uses: gradle/gradle-build-action@v2 + with: + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1240" + - name: Package Build Artifacts + if: success() || failure() + shell: bash + run: | + mkdir ${{ github.job }}-artifacts + find . -path "**/build/reports/*" -or -path "**/build/test-results/*" > artifacts.list + rsync -R --files-from=artifacts.list . ${{ github.job }}-artifacts + tar -zcvf ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz ${{ github.job }}-artifacts + - name: Upload Build Artifacts + if: success() || failure() + uses: actions/upload-artifact@v3 + with: + name: ${{ github.job }} + path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz + retention-days: 30 + + IntegrationTests_1250: + strategy: + fail-fast: false + matrix: + jdk: [17] + runs-on: ubuntu-latest + timeout-minutes: 120 + steps: + - uses: actions/checkout@v4 + with: + fetch-depth: 0 + - name: Set up JDK + uses: actions/setup-java@v3 + with: + java-version: ${{ matrix.jdk }} + distribution: 'temurin' + cache: 'gradle' + - shell: bash + run: | + git remote set-head origin --auto + git remote add upstream https://github.com/linkedin/venice + git fetch upstream + - name: Build with Gradle + uses: gradle/gradle-build-action@v2 + with: + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1250" + - name: Package Build Artifacts + if: success() || failure() + shell: bash + run: | + mkdir ${{ github.job }}-artifacts + find . -path "**/build/reports/*" -or -path "**/build/test-results/*" > artifacts.list + rsync -R --files-from=artifacts.list . ${{ github.job }}-artifacts + tar -zcvf ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz ${{ github.job }}-artifacts + - name: Upload Build Artifacts + if: success() || failure() + uses: actions/upload-artifact@v3 + with: + name: ${{ github.job }} + path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz + retention-days: 30 + + IntegrationTests_1260: + strategy: + fail-fast: false + matrix: + jdk: [17] + runs-on: ubuntu-latest + timeout-minutes: 120 + steps: + - uses: actions/checkout@v4 + with: + fetch-depth: 0 + - name: Set up JDK + uses: actions/setup-java@v3 + with: + java-version: ${{ matrix.jdk }} + distribution: 'temurin' + cache: 'gradle' + - shell: bash + run: | + git remote set-head origin --auto + git remote add upstream https://github.com/linkedin/venice + git fetch upstream + - name: Build with Gradle + uses: gradle/gradle-build-action@v2 + with: + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1260" + - name: Package Build Artifacts + if: success() || failure() + shell: bash + run: | + mkdir ${{ github.job }}-artifacts + find . -path "**/build/reports/*" -or -path "**/build/test-results/*" > artifacts.list + rsync -R --files-from=artifacts.list . ${{ github.job }}-artifacts + tar -zcvf ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz ${{ github.job }}-artifacts + - name: Upload Build Artifacts + if: success() || failure() + uses: actions/upload-artifact@v3 + with: + name: ${{ github.job }} + path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz + retention-days: 30 + + IntegrationTests_1270: + strategy: + fail-fast: false + matrix: + jdk: [17] + runs-on: ubuntu-latest + timeout-minutes: 120 + steps: + - uses: actions/checkout@v4 + with: + fetch-depth: 0 + - name: Set up JDK + uses: actions/setup-java@v3 + with: + java-version: ${{ matrix.jdk }} + distribution: 'temurin' + cache: 'gradle' + - shell: bash + run: | + git remote set-head origin --auto + git remote add upstream https://github.com/linkedin/venice + git fetch upstream + - name: Build with Gradle + uses: gradle/gradle-build-action@v2 + with: + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1270" + - name: Package Build Artifacts + if: success() || failure() + shell: bash + run: | + mkdir ${{ github.job }}-artifacts + find . -path "**/build/reports/*" -or -path "**/build/test-results/*" > artifacts.list + rsync -R --files-from=artifacts.list . ${{ github.job }}-artifacts + tar -zcvf ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz ${{ github.job }}-artifacts + - name: Upload Build Artifacts + if: success() || failure() + uses: actions/upload-artifact@v3 + with: + name: ${{ github.job }} + path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz + retention-days: 30 + + IntegrationTests_1280: + strategy: + fail-fast: false + matrix: + jdk: [17] + runs-on: ubuntu-latest + timeout-minutes: 120 + steps: + - uses: actions/checkout@v4 + with: + fetch-depth: 0 + - name: Set up JDK + uses: actions/setup-java@v3 + with: + java-version: ${{ matrix.jdk }} + distribution: 'temurin' + cache: 'gradle' + - shell: bash + run: | + git remote set-head origin --auto + git remote add upstream https://github.com/linkedin/venice + git fetch upstream + - name: Build with Gradle + uses: gradle/gradle-build-action@v2 + with: + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1280" + - name: Package Build Artifacts + if: success() || failure() + shell: bash + run: | + mkdir ${{ github.job }}-artifacts + find . -path "**/build/reports/*" -or -path "**/build/test-results/*" > artifacts.list + rsync -R --files-from=artifacts.list . ${{ github.job }}-artifacts + tar -zcvf ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz ${{ github.job }}-artifacts + - name: Upload Build Artifacts + if: success() || failure() + uses: actions/upload-artifact@v3 + with: + name: ${{ github.job }} + path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz + retention-days: 30 + + IntegrationTests_1400: + strategy: + fail-fast: false + matrix: + jdk: [17] + runs-on: ubuntu-latest + timeout-minutes: 120 + steps: + - uses: actions/checkout@v4 + with: + fetch-depth: 0 + - name: Set up JDK + uses: actions/setup-java@v3 + with: + java-version: ${{ matrix.jdk }} + distribution: 'temurin' + cache: 'gradle' + - shell: bash + run: | + git remote set-head origin --auto + git remote add upstream https://github.com/linkedin/venice + git fetch upstream + - name: Build with Gradle + uses: gradle/gradle-build-action@v2 + with: + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1400" + - name: Package Build Artifacts + if: success() || failure() + shell: bash + run: | + mkdir ${{ github.job }}-artifacts + find . -path "**/build/reports/*" -or -path "**/build/test-results/*" > artifacts.list + rsync -R --files-from=artifacts.list . ${{ github.job }}-artifacts + tar -zcvf ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz ${{ github.job }}-artifacts + - name: Upload Build Artifacts + if: success() || failure() + uses: actions/upload-artifact@v3 + with: + name: ${{ github.job }} + path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz + retention-days: 30 + + IntegrationTests_1410: + strategy: + fail-fast: false + matrix: + jdk: [17] + runs-on: ubuntu-latest + timeout-minutes: 120 + steps: + - uses: actions/checkout@v4 + with: + fetch-depth: 0 + - name: Set up JDK + uses: actions/setup-java@v3 + with: + java-version: ${{ matrix.jdk }} + distribution: 'temurin' + cache: 'gradle' + - shell: bash + run: | + git remote set-head origin --auto + git remote add upstream https://github.com/linkedin/venice + git fetch upstream + - name: Build with Gradle + uses: gradle/gradle-build-action@v2 + with: + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1410" + - name: Package Build Artifacts + if: success() || failure() + shell: bash + run: | + mkdir ${{ github.job }}-artifacts + find . -path "**/build/reports/*" -or -path "**/build/test-results/*" > artifacts.list + rsync -R --files-from=artifacts.list . ${{ github.job }}-artifacts + tar -zcvf ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz ${{ github.job }}-artifacts + - name: Upload Build Artifacts + if: success() || failure() + uses: actions/upload-artifact@v3 + with: + name: ${{ github.job }} + path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz + retention-days: 30 + + IntegrationTests_1420: + strategy: + fail-fast: false + matrix: + jdk: [17] + runs-on: ubuntu-latest + timeout-minutes: 120 + steps: + - uses: actions/checkout@v4 + with: + fetch-depth: 0 + - name: Set up JDK + uses: actions/setup-java@v3 + with: + java-version: ${{ matrix.jdk }} + distribution: 'temurin' + cache: 'gradle' + - shell: bash + run: | + git remote set-head origin --auto + git remote add upstream https://github.com/linkedin/venice + git fetch upstream + - name: Build with Gradle + uses: gradle/gradle-build-action@v2 + with: + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1420" + - name: Package Build Artifacts + if: success() || failure() + shell: bash + run: | + mkdir ${{ github.job }}-artifacts + find . -path "**/build/reports/*" -or -path "**/build/test-results/*" > artifacts.list + rsync -R --files-from=artifacts.list . ${{ github.job }}-artifacts + tar -zcvf ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz ${{ github.job }}-artifacts + - name: Upload Build Artifacts + if: success() || failure() + uses: actions/upload-artifact@v3 + with: + name: ${{ github.job }} + path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz + retention-days: 30 + + IntegrationTests_1430: + strategy: + fail-fast: false + matrix: + jdk: [17] + runs-on: ubuntu-latest + timeout-minutes: 120 + steps: + - uses: actions/checkout@v4 + with: + fetch-depth: 0 + - name: Set up JDK + uses: actions/setup-java@v3 + with: + java-version: ${{ matrix.jdk }} + distribution: 'temurin' + cache: 'gradle' + - shell: bash + run: | + git remote set-head origin --auto + git remote add upstream https://github.com/linkedin/venice + git fetch upstream + - name: Build with Gradle + uses: gradle/gradle-build-action@v2 + with: + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1430" + - name: Package Build Artifacts + if: success() || failure() + shell: bash + run: | + mkdir ${{ github.job }}-artifacts + find . -path "**/build/reports/*" -or -path "**/build/test-results/*" > artifacts.list + rsync -R --files-from=artifacts.list . ${{ github.job }}-artifacts + tar -zcvf ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz ${{ github.job }}-artifacts + - name: Upload Build Artifacts + if: success() || failure() + uses: actions/upload-artifact@v3 + with: + name: ${{ github.job }} + path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz + retention-days: 30 + + IntegrationTests_1440: + strategy: + fail-fast: false + matrix: + jdk: [17] + runs-on: ubuntu-latest + timeout-minutes: 120 + steps: + - uses: actions/checkout@v4 + with: + fetch-depth: 0 + - name: Set up JDK + uses: actions/setup-java@v3 + with: + java-version: ${{ matrix.jdk }} + distribution: 'temurin' + cache: 'gradle' + - shell: bash + run: | + git remote set-head origin --auto + git remote add upstream https://github.com/linkedin/venice + git fetch upstream + - name: Build with Gradle + uses: gradle/gradle-build-action@v2 + with: + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1440" + - name: Package Build Artifacts + if: success() || failure() + shell: bash + run: | + mkdir ${{ github.job }}-artifacts + find . -path "**/build/reports/*" -or -path "**/build/test-results/*" > artifacts.list + rsync -R --files-from=artifacts.list . ${{ github.job }}-artifacts + tar -zcvf ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz ${{ github.job }}-artifacts + - name: Upload Build Artifacts + if: success() || failure() + uses: actions/upload-artifact@v3 + with: + name: ${{ github.job }} + path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz + retention-days: 30 + + IntegrationTests_1500: + strategy: + fail-fast: false + matrix: + jdk: [17] + runs-on: ubuntu-latest + timeout-minutes: 120 + steps: + - uses: actions/checkout@v4 + with: + fetch-depth: 0 + - name: Set up JDK + uses: actions/setup-java@v3 + with: + java-version: ${{ matrix.jdk }} + distribution: 'temurin' + cache: 'gradle' + - shell: bash + run: | + git remote set-head origin --auto + git remote add upstream https://github.com/linkedin/venice + git fetch upstream + - name: Build with Gradle + uses: gradle/gradle-build-action@v2 + with: + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1500" + - name: Package Build Artifacts + if: success() || failure() + shell: bash + run: | + mkdir ${{ github.job }}-artifacts + find . -path "**/build/reports/*" -or -path "**/build/test-results/*" > artifacts.list + rsync -R --files-from=artifacts.list . ${{ github.job }}-artifacts + tar -zcvf ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz ${{ github.job }}-artifacts + - name: Upload Build Artifacts + if: success() || failure() + uses: actions/upload-artifact@v3 + with: + name: ${{ github.job }} + path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz + retention-days: 30 + + IntegrationTests_1550: + strategy: + fail-fast: false + matrix: + jdk: [17] + runs-on: ubuntu-latest + timeout-minutes: 120 + steps: + - uses: actions/checkout@v4 + with: + fetch-depth: 0 + - name: Set up JDK + uses: actions/setup-java@v3 + with: + java-version: ${{ matrix.jdk }} + distribution: 'temurin' + cache: 'gradle' + - shell: bash + run: | + git remote set-head origin --auto + git remote add upstream https://github.com/linkedin/venice + git fetch upstream + - name: Build with Gradle + uses: gradle/gradle-build-action@v2 + with: + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_1550" + - name: Package Build Artifacts + if: success() || failure() + shell: bash + run: | + mkdir ${{ github.job }}-artifacts + find . -path "**/build/reports/*" -or -path "**/build/test-results/*" > artifacts.list + rsync -R --files-from=artifacts.list . ${{ github.job }}-artifacts + tar -zcvf ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz ${{ github.job }}-artifacts + - name: Upload Build Artifacts + if: success() || failure() + uses: actions/upload-artifact@v3 + with: + name: ${{ github.job }} + path: ${{ github.job }}-jdk${{ matrix.jdk }}-logs.tar.gz + retention-days: 30 + + IntegrationTests_9999: + strategy: + fail-fast: false + matrix: + jdk: [17] + runs-on: ubuntu-latest + timeout-minutes: 120 + steps: + - uses: actions/checkout@v4 + with: + fetch-depth: 0 + - name: Set up JDK + uses: actions/setup-java@v3 + with: + java-version: ${{ matrix.jdk }} + distribution: 'temurin' + cache: 'gradle' + - shell: bash + run: | + git remote set-head origin --auto + git remote add upstream https://github.com/linkedin/venice + git fetch upstream + - name: Build with Gradle + uses: gradle/gradle-build-action@v2 + with: + arguments: "--continue --no-daemon -DforkEvery=1 -DmaxParallelForks=1 integrationTests_9999" - name: Package Build Artifacts if: success() || failure() shell: bash @@ -775,7 +1321,7 @@ jobs: matrix: jdk: [17] runs-on: ubuntu-latest - needs: [IntegrationTestsA, IntegrationTestsB, IntegrationTestsC, IntegrationTestsD, IntegrationTestsE, IntegrationTestsF, IntegrationTestsG, IntegrationTestsH, IntegrationTestsI, IntegrationTestsJ, IntegrationTestsK, IntegrationTestsL, IntegrationTestsM, IntegrationTestsN, IntegrationTestsO, IntegrationTestsP, IntegrationTestsQ, IntegrationTestsZ] + needs: [IntegrationTests_1000, IntegrationTests_1010, IntegrationTests_1020, IntegrationTests_1030, IntegrationTests_1040, IntegrationTests_1050, IntegrationTests_1060, IntegrationTests_1070, IntegrationTests_1080, IntegrationTests_1090, IntegrationTests_1100, IntegrationTests_1110, IntegrationTests_1120, IntegrationTests_1130, IntegrationTests_1200, IntegrationTests_1210, IntegrationTests_1220, IntegrationTests_1230, IntegrationTests_1240, IntegrationTests_1250, IntegrationTests_1260, IntegrationTests_1270, IntegrationTests_1280, IntegrationTests_1400, IntegrationTests_1410, IntegrationTests_1420, IntegrationTests_1430, IntegrationTests_1440, IntegrationTests_1500, IntegrationTests_1550, IntegrationTests_9999] timeout-minutes: 120 steps: - uses: actions/checkout@v4 diff --git a/internal/venice-test-common/build.gradle b/internal/venice-test-common/build.gradle index 404273431d..b037f6eb07 100644 --- a/internal/venice-test-common/build.gradle +++ b/internal/venice-test-common/build.gradle @@ -133,72 +133,104 @@ def integrationTestConfigs = { } def integrationTestBuckets = [ - "A": [ + "1000": [ "com.linkedin.davinci.*", - "com.linkedin.venice.endToEnd.DaVinciClient*"], - "B": [ + "com.linkedin.venice.endToEnd.DaVinciClientDiskFullTest", + "com.linkedin.venice.endToEnd.DaVinciClientMemoryLimitTest"], + "1010": [ + "com.linkedin.venice.endToEnd.DaVinciClientTest"], + "1020": [ "com.linkedin.venice.endToEnd.DaVinciCluster*", "com.linkedin.venice.endToEnd.DaVinciCompute*", "com.linkedin.venice.endToEnd.DaVinciLive*"], - "C": [ + "1030": [ "com.linkedin.venice.consumer.*"], - "D": [ - "com.linkedin.venice.endToEnd.ActiveActive*", + "1040": [ + "com.linkedin.venice.endToEnd.ActiveActive*"], + "1050": [ "com.linkedin.venice.endToEnd.TestActiveActive*"], - "E": [ + "1060": [ "com.linkedin.venice.helix.*", "com.linkedin.venice.helixrebalance.*"], - "F": [ + "1070": [ "com.linkedin.venice.fastclient.*"], - "G": [ + "1080": [ "com.linkedin.venice.endToEnd.TestEmptyPush", - "com.linkedin.venice.router.*", "com.linkedin.venice.ingestionHeartbeat.*"], - "H": [ - "com.linkedin.venice.server.*", - "com.linkedin.venice.storagenode.*", + "1090": [ + "com.linkedin.venice.router.*" + ], + "1100": [ + "com.linkedin.venice.server.*"], + "1110": [ "com.linkedin.venice.restart.*"], - "I": [ + "1120": [ + "com.linkedin.venice.storagenode.*"], + "1130": [ "com.linkedin.venice.endToEnd.TestStoreMigration", "com.linkedin.venice.endToEnd.TestStuckConsumerRepair", "com.linkedin.venice.endToEnd.TestSuperSetSchemaRegistration", "com.linkedin.venice.endToEnd.TestTopicWiseSharedConsumerPoolResilience", - "com.linkedin.venice.endToEnd.TestUnusedValueSchemaCleanup" - ], - "J": [ - "com.linkedin.venice.hadoop.*", + "com.linkedin.venice.endToEnd.TestUnusedValueSchemaCleanup"], + "1200":[ "com.linkedin.venice.endToEnd.TestVson*", "com.linkedin.venice.endToEnd.Push*"], - "K": [ + "1210": [ + "com.linkedin.venice.hadoop.*"], + "1220": [ "com.linkedin.venice.endToEnd.TestPushJob*"], - "L": [ + "1230": [ "com.linkedin.venice.endToEnd.TestBatch*"], - "M": [ + "1240": [ "com.linkedin.venice.kafka.*", "com.linkedin.venice.samza.*", - "com.linkedin.venice.writer.*", - "com.linkedin.venice.endToEnd.PartialUpdateTest", + "com.linkedin.venice.writer.*"], + "1250": [ + "com.linkedin.venice.endToEnd.PartialUpdateTest"], + "1260": [ + "com.linkedin.venice.endToEnd.PartialUpdateWithParallelProcessingTest"], + "1270": [ "com.linkedin.venice.endToEnd.TestWritePathComputation", - "com.linkedin.venice.endToEnd.WriteComputeWithActiveActiveReplicationTest"], - "N": [ + "com.linkedin.venice.endToEnd.WriteComputeWithActiveActiveReplicationTest", "com.linkedin.venice.endToEnd.StoragePersona*", "com.linkedin.venice.endToEnd.TestStoreUpdateStoragePersona", - "com.linkedin.venice.persona.*", + "com.linkedin.venice.persona.*"], + "1280": [ "com.linkedin.venice.pubsub.*"], - "O": [ + "1400": [ "com.linkedin.venice.endToEnd.TestHybrid*"], - "P": [ + "1410": [ "com.linkedin.venice.controller.server.*", "com.linkedin.venice.controller.kafka.consumer.*", - "com.linkedin.venice.controller.migration.*", + "com.linkedin.venice.controller.migration.*"], + "1420": [ "com.linkedin.venice.controller.AdminTool*", "com.linkedin.venice.controller.VeniceParentHelixAdminTest"], - "Q": [ - "com.linkedin.venice.controller.Test*"] + "1430": [ + "com.linkedin.venice.controller.Test*"], + "1440": [ + "com.linkedin.venice.endToEnd.DataRecoveryTest", + "com.linkedin.venice.controllerapi.TestControllerClient"], + "1500":[ + "com.linkedin.venice.multicluster.TestMetadataOperationInMultiCluster", + "com.linkedin.venice.endToEnd.MetaSystemStoreTest", + "com.linkedin.venice.zk.TestMigrateVeniceZKPaths", + "com.linkedin.venice.throttle.TestRouterReadQuotaThrottler"], + "1550": [ + "com.linkedin.venice.stats.TestZkClientStatusStats", + "com.linkedin.venice.multicluster.TestMetadataOperationInMultiCluster", + "com.linkedin.venice.integration.utils.SystemExitPrevention", + "com.linkedin.venice.integration.StorageNodeServiceTest", + "com.linkedin.venice.endToEnd.TestStoreGraveyardCleanupService", + "com.linkedin.venice.endToEnd.TestStoreBackupVersionDeletion", + "com.linkedin.venice.endToEnd.TestStaleDataVisibility"] ] integrationTestBuckets.each { name, patterns -> - task "integrationTest${name}" (type: Test) { + task "integrationTests_${name}" (type: Test) { + ext { + suiteStartTime = 0 + } filter { patterns.each { pattern -> includeTestsMatching pattern @@ -209,10 +241,21 @@ integrationTestBuckets.each { name, patterns -> excludeGroups 'flaky' listeners = ['com.linkedin.venice.testng.VeniceSuiteListener', 'com.linkedin.venice.testng.VeniceTestListener'] } + doFirst { + suiteStartTime = System.currentTimeMillis() + } + afterSuite { descriptor, result -> + if (descriptor.name.startsWith("com.linkedin")) { + println "Test Suite ${descriptor.name} completed in ${(System.currentTimeMillis() - suiteStartTime )/1000} s" + } + } } } -task integrationTestZ(type: Test) { +task integrationTests_9999(type: Test) { + ext { + suiteStartTime = 0 + } filter { integrationTestBuckets.each { name, patterns -> patterns.each { pattern -> @@ -220,12 +263,19 @@ task integrationTestZ(type: Test) { } } } - + doFirst { + suiteStartTime = System.currentTimeMillis() + } configure integrationTestConfigs useTestNG { excludeGroups 'flaky' listeners = ['com.linkedin.venice.testng.VeniceSuiteListener', 'com.linkedin.venice.testng.VeniceTestListener'] } + afterSuite { descriptor, result -> + if (descriptor.name.startsWith("com.linkedin")) { + println "Test Suite ${descriptor.name} completed in ${(System.currentTimeMillis() - suiteStartTime )/1000} s" + } + } } task generateGHCI() { @@ -252,14 +302,14 @@ task generateGHCI() { def common = "--continue --no-daemon " - def integTestGradleArgs = common + "-DforkEvery=1 -DmaxParallelForks=1 integrationTest" + def integTestGradleArgs = common + "-DforkEvery=1 -DmaxParallelForks=1 integrationTests_" integrationTestBuckets.each { name, patterns -> - def flowName = "IntegrationTests" + name + def flowName = "IntegrationTests_" + name jobs << flowName appendToGHCI(paramFileContent, targetFilePath, flowName, 120, integTestGradleArgs + name) } - def otherTest = "IntegrationTestsZ" - appendToGHCI(paramFileContent, targetFilePath, otherTest, 120, integTestGradleArgs + "Z") + def otherTest = "IntegrationTests_9999" + appendToGHCI(paramFileContent, targetFilePath, otherTest, 120, integTestGradleArgs + "9999") jobs << otherTest // define a job that depends others to manage the status check From bd2de8e25db69166dc89d7aa794c4d7bfe34c508 Mon Sep 17 00:00:00 2001 From: Gaojie Liu Date: Fri, 20 Sep 2024 08:57:34 -0700 Subject: [PATCH 04/16] [fast-client] Replace per-store per-route tracking by per-route tracking (#1189) * [fast-client] Replace per-store per-route tracking by per-route tracking This PR is trying to reduce the total number of metrics emitted by fast-client. Previously, Fast-Client will have the instance-level tracking for each store, which will create a lot of metrics when the application is using multiple stores, and this code change will only emit tacking metrics (qps, latency) in instance level rather than per store per instance. This PR also removes the pending request metric, which is per store per route, and we will add back this metric in a separate PR when we improve InstanceHealthMonitor to be cluster-level instead of store-level. --- .../fastclient/BatchGetRequestContext.java | 8 ++ .../venice/fastclient/ClientConfig.java | 15 ++- .../fastclient/ComputeRequestContext.java | 8 ++ .../DispatchingAvroGenericStoreClient.java | 2 + .../venice/fastclient/GetRequestContext.java | 8 ++ .../venice/fastclient/RequestContext.java | 7 +- .../StatsAvroGenericStoreClient.java | 37 ++++-- .../fastclient/meta/RequestBasedMetadata.java | 7 + .../venice/fastclient/meta/StoreMetadata.java | 2 + .../fastclient/stats/ClusterRouteStats.java | 115 +++++++++++++++++ .../venice/fastclient/stats/ClusterStats.java | 37 ------ .../fastclient/stats/FastClientStats.java | 120 ------------------ ...DispatchingAvroGenericStoreClientTest.java | 85 ++++++++++--- .../meta/RequestBasedMetadataTestUtils.java | 3 +- .../fastclient/utils/TestClientSimulator.java | 5 + 15 files changed, 268 insertions(+), 191 deletions(-) create mode 100644 clients/venice-client/src/main/java/com/linkedin/venice/fastclient/stats/ClusterRouteStats.java diff --git a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/BatchGetRequestContext.java b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/BatchGetRequestContext.java index 83506e9787..5f02d102a1 100644 --- a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/BatchGetRequestContext.java +++ b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/BatchGetRequestContext.java @@ -1,5 +1,8 @@ package com.linkedin.venice.fastclient; +import com.linkedin.venice.read.RequestType; + + /** * Keep track of the progress of a batch get request . This includes tracking * all the scatter requests and utilities to gather responses. @@ -10,4 +13,9 @@ public class BatchGetRequestContext extends MultiKeyRequestContext { public BatchGetRequestContext(int numKeysInRequest, boolean isPartialSuccessAllowed) { super(numKeysInRequest, isPartialSuccessAllowed); } + + @Override + public RequestType getRequestType() { + return RequestType.MULTI_GET_STREAMING; + } } diff --git a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/ClientConfig.java b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/ClientConfig.java index fd3283204e..023b966444 100644 --- a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/ClientConfig.java +++ b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/ClientConfig.java @@ -95,6 +95,8 @@ public class ClientConfig { private boolean projectionFieldValidation; private Set harClusters; + private final MetricsRepository metricsRepository; + private ClientConfig( String storeName, Client r2Client, @@ -145,17 +147,16 @@ private ClientConfig( this.r2Client = r2Client; this.storeName = storeName; this.statsPrefix = (statsPrefix == null ? "" : statsPrefix); - if (metricsRepository == null) { - metricsRepository = MetricsRepositoryUtils.createMultiThreadedMetricsRepository(); - } + this.metricsRepository = + metricsRepository != null ? metricsRepository : MetricsRepositoryUtils.createMultiThreadedMetricsRepository(); // TODO consider changing the implementation or make it explicit that the config builder can only build once with // the same metricsRepository for (RequestType requestType: RequestType.values()) { clientStatsMap.put( requestType, - FastClientStats.getClientStats(metricsRepository, this.statsPrefix, storeName, requestType)); + FastClientStats.getClientStats(this.metricsRepository, this.statsPrefix, storeName, requestType)); } - this.clusterStats = new ClusterStats(metricsRepository, storeName); + this.clusterStats = new ClusterStats(this.metricsRepository, storeName); this.speculativeQueryEnabled = speculativeQueryEnabled; this.specificValueClass = specificValueClass; this.deserializationExecutor = deserializationExecutor; @@ -272,6 +273,10 @@ public Client getR2Client() { return r2Client; } + public MetricsRepository getMetricsRepository() { + return metricsRepository; + } + public FastClientStats getStats(RequestType requestType) { return clientStatsMap.get(requestType); } diff --git a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/ComputeRequestContext.java b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/ComputeRequestContext.java index 2fa06bfa1b..0d962fb1e8 100644 --- a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/ComputeRequestContext.java +++ b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/ComputeRequestContext.java @@ -1,5 +1,8 @@ package com.linkedin.venice.fastclient; +import com.linkedin.venice.read.RequestType; + + /** * Keep track of the progress of a compute request . This includes tracking * all the scatter requests and utilities to gather responses. @@ -10,4 +13,9 @@ public class ComputeRequestContext extends MultiKeyRequestContext { public ComputeRequestContext(int numKeysInRequest, boolean isPartialSuccessAllowed) { super(numKeysInRequest, isPartialSuccessAllowed); } + + @Override + public RequestType getRequestType() { + return RequestType.COMPUTE_STREAMING; + } } diff --git a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/DispatchingAvroGenericStoreClient.java b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/DispatchingAvroGenericStoreClient.java index 5f439b2172..140e8895f7 100644 --- a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/DispatchingAvroGenericStoreClient.java +++ b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/DispatchingAvroGenericStoreClient.java @@ -195,6 +195,7 @@ public ClientConfig getClientConfig() { @Override protected CompletableFuture get(GetRequestContext requestContext, K key) throws VeniceClientException { verifyMetadataInitialized(); + requestContext.serverClusterName = metadata.getClusterName(); requestContext.instanceHealthMonitor = metadata.getInstanceHealthMonitor(); if (requestContext.requestUri == null) { /** @@ -425,6 +426,7 @@ private void multiKeyStreamingRequest( long requestId = REQUEST_ID_GENERATOR.getAndIncrement(); + requestContext.serverClusterName = metadata.getClusterName(); /* Prepare each of the routes needed to query the keys */ requestContext.instanceHealthMonitor = metadata.getInstanceHealthMonitor(); int currentVersion = requestContext.currentVersion; diff --git a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/GetRequestContext.java b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/GetRequestContext.java index daa192f2ed..da08505dea 100644 --- a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/GetRequestContext.java +++ b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/GetRequestContext.java @@ -1,5 +1,8 @@ package com.linkedin.venice.fastclient; +import com.linkedin.venice.read.RequestType; + + public class GetRequestContext extends RequestContext { int partitionId; /** @@ -14,6 +17,11 @@ public class GetRequestContext extends RequestContext { retryContext = null; } + @Override + public RequestType getRequestType() { + return RequestType.SINGLE_GET; + } + static class RetryContext { boolean longTailRetryRequestTriggered; boolean errorRetryRequestTriggered; diff --git a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/RequestContext.java b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/RequestContext.java index 9f6ab4f647..bb9c12976a 100644 --- a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/RequestContext.java +++ b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/RequestContext.java @@ -1,6 +1,7 @@ package com.linkedin.venice.fastclient; import com.linkedin.venice.fastclient.meta.InstanceHealthMonitor; +import com.linkedin.venice.read.RequestType; import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap; import java.util.Map; import java.util.concurrent.CompletableFuture; @@ -10,7 +11,7 @@ /** * This class is used to include all the intermediate fields required for the communication between the different tiers. */ -public class RequestContext { +public abstract class RequestContext { int currentVersion = -1; boolean noAvailableReplica = false; @@ -28,6 +29,10 @@ public class RequestContext { Map> routeRequestMap = new VeniceConcurrentHashMap<>(); + String serverClusterName; + public RequestContext() { } + + public abstract RequestType getRequestType(); } diff --git a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/StatsAvroGenericStoreClient.java b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/StatsAvroGenericStoreClient.java index e052720abe..2772a1108e 100644 --- a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/StatsAvroGenericStoreClient.java +++ b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/StatsAvroGenericStoreClient.java @@ -16,11 +16,13 @@ import com.linkedin.venice.client.store.streaming.StreamingResponseTracker; import com.linkedin.venice.compute.ComputeRequestWrapper; import com.linkedin.venice.fastclient.meta.InstanceHealthMonitor; +import com.linkedin.venice.fastclient.stats.ClusterRouteStats; import com.linkedin.venice.fastclient.stats.ClusterStats; import com.linkedin.venice.fastclient.stats.FastClientStats; import com.linkedin.venice.read.RequestType; import com.linkedin.venice.utils.LatencyUtils; import com.linkedin.venice.utils.Time; +import io.tehuti.metrics.MetricsRepository; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -38,6 +40,8 @@ public class StatsAvroGenericStoreClient extends DelegatingAvroStoreClient private final FastClientStats clientStatsForStreamingBatchGet; private final FastClientStats clientStatsForStreamingCompute; private final ClusterStats clusterStats; + private final MetricsRepository metricsRepository; + private final ClusterRouteStats clusterRouteStats; public StatsAvroGenericStoreClient(InternalAvroStoreClient delegate, ClientConfig clientConfig) { super(delegate, clientConfig); @@ -45,6 +49,8 @@ public StatsAvroGenericStoreClient(InternalAvroStoreClient delegate, Clien this.clientStatsForStreamingBatchGet = clientConfig.getStats(RequestType.MULTI_GET_STREAMING); this.clientStatsForStreamingCompute = clientConfig.getStats(RequestType.COMPUTE_STREAMING); this.clusterStats = clientConfig.getClusterStats(); + this.metricsRepository = clientConfig.getMetricsRepository(); + this.clusterRouteStats = ClusterRouteStats.get(); } @Override @@ -242,9 +248,17 @@ private void recordPerRouteMetrics(RequestContext requestContext, FastClientStat } replicaRequestFuture.forEach((instance, future) -> { future.whenComplete((status, throwable) -> { - if (monitor != null) { - clusterStats.recordPendingRequestCount(instance, monitor.getPendingRequestCounter(instance)); - } + ClusterRouteStats.RouteStats routeStats = clusterRouteStats.getRouteStats( + metricsRepository, + /** + * There is a race condition during store migration and the cluster name might not match + * with the requested instance. + * It is fine for tracking purpose as it would only happen for a very short period and + * the wrong cluster/instance combination will be deprecated soon because of a short lifetime. + */ + requestContext.serverClusterName, + instance, + requestContext.getRequestType()); if (throwable != null) { status = (throwable instanceof VeniceClientHttpException) @@ -252,29 +266,28 @@ private void recordPerRouteMetrics(RequestContext requestContext, FastClientStat : SC_SERVICE_UNAVAILABLE; } - clientStats.recordRequest(instance); - clientStats - .recordResponseWaitingTime(instance, LatencyUtils.getElapsedTimeFromNSToMS(requestSentTimestampNS)); + routeStats.recordRequest(); + routeStats.recordResponseWaitingTime(LatencyUtils.getElapsedTimeFromNSToMS(requestSentTimestampNS)); switch (status) { case SC_OK: case SC_NOT_FOUND: - clientStats.recordHealthyRequest(instance); + routeStats.recordHealthyRequest(); break; case SC_TOO_MANY_REQUESTS: - clientStats.recordQuotaExceededRequest(instance); + routeStats.recordQuotaExceededRequest(); break; case SC_INTERNAL_SERVER_ERROR: - clientStats.recordInternalServerErrorRequest(instance); + routeStats.recordInternalServerErrorRequest(); break; case SC_GONE: /* Check {@link InstanceHealthMonitor#trackHealthBasedOnRequestToInstance} to understand this special http status. */ - clientStats.recordLeakedRequest(instance); + routeStats.recordLeakedRequest(); break; case SC_SERVICE_UNAVAILABLE: - clientStats.recordServiceUnavailableRequest(instance); + routeStats.recordServiceUnavailableRequest(); break; default: - clientStats.recordOtherErrorRequest(instance); + routeStats.recordOtherErrorRequest(); } }); }); diff --git a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/meta/RequestBasedMetadata.java b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/meta/RequestBasedMetadata.java index 1361c90d8a..21a39bd985 100644 --- a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/meta/RequestBasedMetadata.java +++ b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/meta/RequestBasedMetadata.java @@ -112,6 +112,7 @@ public class RequestBasedMetadata extends AbstractStoreMetadata { private volatile boolean isServiceDiscovered; private volatile boolean isReady; private CountDownLatch isReadyLatch = new CountDownLatch(1); + private AtomicReference serverClusterName = new AtomicReference<>(); private Set harClusters; @@ -146,6 +147,11 @@ synchronized void setMetadataResponseSchemaReader(RouterBackedSchemaReader metad this.metadataResponseSchemaReader = metadataResponseSchemaReader; } + @Override + public String getClusterName() { + return serverClusterName.get(); + } + @Override public int getCurrentStoreVersion() { if (!isReady()) { @@ -220,6 +226,7 @@ void discoverD2Service() { d2TransportClient.setServiceName(clusterDiscoveryD2ServiceName); String serverD2ServiceName = d2ServiceDiscovery.find(d2TransportClient, storeName, true).getServerD2Service(); d2TransportClient.setServiceName(serverD2ServiceName); + serverClusterName.set(serverD2ServiceName); isServiceDiscovered = true; if (harClusters.contains(serverD2ServiceName)) { LOGGER.info( diff --git a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/meta/StoreMetadata.java b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/meta/StoreMetadata.java index 694d9c6ea6..d258dda461 100644 --- a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/meta/StoreMetadata.java +++ b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/meta/StoreMetadata.java @@ -18,6 +18,8 @@ * to decide the healthiness of each replica. */ public interface StoreMetadata extends SchemaReader { + String getClusterName(); + String getStoreName(); int getCurrentStoreVersion(); diff --git a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/stats/ClusterRouteStats.java b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/stats/ClusterRouteStats.java new file mode 100644 index 0000000000..b1ce105e58 --- /dev/null +++ b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/stats/ClusterRouteStats.java @@ -0,0 +1,115 @@ +package com.linkedin.venice.fastclient.stats; + +import com.linkedin.venice.read.RequestType; +import com.linkedin.venice.stats.AbstractVeniceHttpStats; +import com.linkedin.venice.stats.StatsUtils; +import com.linkedin.venice.stats.TehutiUtils; +import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap; +import io.tehuti.metrics.MetricsRepository; +import io.tehuti.metrics.Sensor; +import io.tehuti.metrics.stats.OccurrenceRate; +import java.net.MalformedURLException; +import java.net.URL; +import java.util.Map; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + + +public class ClusterRouteStats { + private static final Logger LOGGER = LogManager.getLogger(ClusterRouteStats.class); + + private static final ClusterRouteStats DEFAULT = new ClusterRouteStats(); + + private final Map perRouteStatMap = new VeniceConcurrentHashMap<>(); + + public static ClusterRouteStats get() { + return DEFAULT; + } + + private ClusterRouteStats() { + } + + public RouteStats getRouteStats( + MetricsRepository metricsRepository, + String clusterName, + String instanceUrl, + RequestType requestType) { + String combinedKey = clusterName + "-" + instanceUrl + "-" + requestType.toString(); + return perRouteStatMap.computeIfAbsent(combinedKey, ignored -> { + String instanceName = instanceUrl; + try { + URL url = new URL(instanceUrl); + instanceName = url.getHost(); + } catch (MalformedURLException e) { + LOGGER.error("Invalid instance url: {}", instanceUrl); + } + return new RouteStats(metricsRepository, clusterName, instanceName, requestType); + }); + } + + /** + * Per-route request metrics. + */ + public static class RouteStats extends AbstractVeniceHttpStats { + private final Sensor requestCountSensor; + private final Sensor responseWaitingTimeSensor; + private final Sensor healthyRequestCountSensor; + private final Sensor quotaExceededRequestCountSensor; + private final Sensor internalServerErrorRequestCountSensor; + private final Sensor serviceUnavailableRequestCountSensor; + private final Sensor leakedRequestCountSensor; + private final Sensor otherErrorRequestCountSensor; + + public RouteStats( + MetricsRepository metricsRepository, + String clusterName, + String instanceName, + RequestType requestType) { + super(metricsRepository, clusterName + "." + StatsUtils.convertHostnameToMetricName(instanceName), requestType); + this.requestCountSensor = registerSensor("request_count", new OccurrenceRate()); + this.responseWaitingTimeSensor = registerSensor( + "response_waiting_time", + TehutiUtils.getPercentileStat(getName(), getFullMetricName("response_waiting_time"))); + this.healthyRequestCountSensor = registerSensor("healthy_request_count", new OccurrenceRate()); + this.quotaExceededRequestCountSensor = registerSensor("quota_exceeded_request_count", new OccurrenceRate()); + this.internalServerErrorRequestCountSensor = + registerSensor("internal_server_error_request_count", new OccurrenceRate()); + this.serviceUnavailableRequestCountSensor = + registerSensor("service_unavailable_request_count", new OccurrenceRate()); + this.leakedRequestCountSensor = registerSensor("leaked_request_count", new OccurrenceRate()); + this.otherErrorRequestCountSensor = registerSensor("other_error_request_count", new OccurrenceRate()); + } + + public void recordRequest() { + requestCountSensor.record(); + } + + public void recordResponseWaitingTime(double latency) { + responseWaitingTimeSensor.record(latency); + } + + public void recordHealthyRequest() { + healthyRequestCountSensor.record(); + } + + public void recordQuotaExceededRequest() { + quotaExceededRequestCountSensor.record(); + } + + public void recordInternalServerErrorRequest() { + internalServerErrorRequestCountSensor.record(); + } + + public void recordServiceUnavailableRequest() { + serviceUnavailableRequestCountSensor.record(); + } + + public void recordLeakedRequest() { + leakedRequestCountSensor.record(); + } + + public void recordOtherErrorRequest() { + otherErrorRequestCountSensor.record(); + } + } +} diff --git a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/stats/ClusterStats.java b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/stats/ClusterStats.java index 213cb3d2b5..c198554bc6 100644 --- a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/stats/ClusterStats.java +++ b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/stats/ClusterStats.java @@ -1,8 +1,6 @@ package com.linkedin.venice.fastclient.stats; import com.linkedin.venice.stats.AbstractVeniceStats; -import com.linkedin.venice.stats.StatsUtils; -import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap; import io.tehuti.Metric; import io.tehuti.metrics.MetricsRepository; import io.tehuti.metrics.Sensor; @@ -10,11 +8,8 @@ import io.tehuti.metrics.stats.Avg; import io.tehuti.metrics.stats.Max; import io.tehuti.metrics.stats.OccurrenceRate; -import java.net.MalformedURLException; -import java.net.URL; import java.util.Arrays; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -28,7 +23,6 @@ public class ClusterStats extends AbstractVeniceStats { private static final Logger LOGGER = LogManager.getLogger(ClusterStats.class); private final String storeName; - private final Map perRouteStats = new VeniceConcurrentHashMap<>(); private final Sensor blockedInstanceCount; private final Sensor unhealthyInstanceCount; private final Sensor versionUpdateFailureSensor; @@ -55,10 +49,6 @@ public void recordUnhealthyInstanceCount(int count) { this.unhealthyInstanceCount.record(count); } - public void recordPendingRequestCount(String instance, int count) { - getRouteStats(instance).recordPendingRequestCount(count); - } - public void updateCurrentVersion(int currentVersion) { this.currentVersion = currentVersion; } @@ -75,31 +65,4 @@ public List getMetricValues(String sensorName, String... stats) { }).collect(Collectors.toList()); return collect; } - - private RouteStats getRouteStats(String instanceUrl) { - return perRouteStats.computeIfAbsent(instanceUrl, k -> { - String instanceName = instanceUrl; - try { - URL url = new URL(instanceUrl); - instanceName = url.getHost() + "_" + url.getPort(); - } catch (MalformedURLException e) { - LOGGER.error("Invalid instance url: {}", instanceUrl); - } - return new RouteStats(getMetricsRepository(), storeName, instanceName); - }); - } - - private static class RouteStats extends AbstractVeniceStats { - private final Sensor pendingRequestCounterSensor; - - public RouteStats(MetricsRepository metricsRepository, String storeName, String instanceName) { - super(metricsRepository, storeName + "." + StatsUtils.convertHostnameToMetricName(instanceName)); - - this.pendingRequestCounterSensor = registerSensor("pending_request_count", new Avg(), new Max()); - } - - public void recordPendingRequestCount(int count) { - pendingRequestCounterSensor.record(count); - } - } } diff --git a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/stats/FastClientStats.java b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/stats/FastClientStats.java index 2febabd1f8..812d17f9ba 100644 --- a/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/stats/FastClientStats.java +++ b/clients/venice-client/src/main/java/com/linkedin/venice/fastclient/stats/FastClientStats.java @@ -1,10 +1,7 @@ package com.linkedin.venice.fastclient.stats; import com.linkedin.venice.read.RequestType; -import com.linkedin.venice.stats.AbstractVeniceStats; -import com.linkedin.venice.stats.StatsUtils; import com.linkedin.venice.stats.TehutiUtils; -import com.linkedin.venice.utils.concurrent.VeniceConcurrentHashMap; import io.tehuti.Metric; import io.tehuti.metrics.MetricsRepository; import io.tehuti.metrics.Sensor; @@ -13,20 +10,13 @@ import io.tehuti.metrics.stats.Max; import io.tehuti.metrics.stats.OccurrenceRate; import io.tehuti.metrics.stats.Rate; -import java.net.MalformedURLException; -import java.net.URL; import java.util.Arrays; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; import java.util.stream.IntStream; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; public class FastClientStats extends com.linkedin.venice.client.stats.ClientStats { - private static final Logger LOGGER = LogManager.getLogger(FastClientStats.class); - private final String storeName; private final Sensor noAvailableReplicaRequestCountSensor; @@ -46,9 +36,6 @@ public class FastClientStats extends com.linkedin.venice.client.stats.ClientStat private final Sensor retryFanoutSizeSensor; private long cacheTimeStampInMs = 0; - // Routing stats - private final Map perRouteStats = new VeniceConcurrentHashMap<>(); - public static FastClientStats getClientStats( MetricsRepository metricsRepository, String statsPrefix, @@ -117,52 +104,6 @@ public void recordThinClientFastClientLatencyDelta(double latencyDelta) { dualReadThinClientFastClientLatencyDeltaSensor.record(latencyDelta); } - private RouteStats getRouteStats(String instanceUrl) { - return perRouteStats.computeIfAbsent(instanceUrl, k -> { - String instanceName = instanceUrl; - try { - URL url = new URL(instanceUrl); - instanceName = url.getHost() + "_" + url.getPort(); - } catch (MalformedURLException e) { - LOGGER.error("Invalid instance url: {}", instanceUrl); - } - return new RouteStats(getMetricsRepository(), storeName, instanceName); - }); - } - - public void recordRequest(String instance) { - getRouteStats(instance).recordRequest(); - } - - public void recordResponseWaitingTime(String instance, double latency) { - getRouteStats(instance).recordResponseWaitingTime(latency); - } - - public void recordHealthyRequest(String instance) { - getRouteStats(instance).recordHealthyRequest(); - } - - public void recordQuotaExceededRequest(String instance) { - getRouteStats(instance).recordQuotaExceededRequest(); - } - - public void recordInternalServerErrorRequest(String instance) { - getRouteStats(instance).recordInternalServerErrorRequest(); - } - - public void recordServiceUnavailableRequest(String instance) { - getRouteStats(instance).recordServiceUnavailableRequest(); - } - - public void recordLeakedRequest(String instance) { - leakedRequestCountSensor.record(); - getRouteStats(instance).recordLeakedRequest(); - } - - public void recordOtherErrorRequest(String instance) { - getRouteStats(instance).recordOtherErrorRequest(); - } - public void recordLongTailRetryRequest() { longTailRetryRequestSensor.record(); } @@ -219,65 +160,4 @@ public List getMetricValues(String sensorName, String... stats) { }).collect(Collectors.toList()); return collect; } - - /** - * Per-route request metrics. - */ - private static class RouteStats extends AbstractVeniceStats { - private final Sensor requestCountSensor; - private final Sensor responseWaitingTimeSensor; - private final Sensor healthyRequestCountSensor; - private final Sensor quotaExceededRequestCountSensor; - private final Sensor internalServerErrorRequestCountSensor; - private final Sensor serviceUnavailableRequestCountSensor; - private final Sensor leakedRequestCountSensor; - private final Sensor otherErrorRequestCountSensor; - - public RouteStats(MetricsRepository metricsRepository, String storeName, String instanceName) { - super(metricsRepository, storeName + "." + StatsUtils.convertHostnameToMetricName(instanceName)); - this.requestCountSensor = registerSensor("request_count", new OccurrenceRate()); - this.responseWaitingTimeSensor = - registerSensor("response_waiting_time", TehutiUtils.getPercentileStat(getName(), "response_waiting_time")); - this.healthyRequestCountSensor = registerSensor("healthy_request_count", new OccurrenceRate()); - this.quotaExceededRequestCountSensor = registerSensor("quota_exceeded_request_count", new OccurrenceRate()); - this.internalServerErrorRequestCountSensor = - registerSensor("internal_server_error_request_count", new OccurrenceRate()); - this.serviceUnavailableRequestCountSensor = - registerSensor("service_unavailable_request_count", new OccurrenceRate()); - this.leakedRequestCountSensor = registerSensor("leaked_request_count", new OccurrenceRate()); - this.otherErrorRequestCountSensor = registerSensor("other_error_request_count", new OccurrenceRate()); - } - - public void recordRequest() { - requestCountSensor.record(); - } - - public void recordResponseWaitingTime(double latency) { - responseWaitingTimeSensor.record(latency); - } - - public void recordHealthyRequest() { - healthyRequestCountSensor.record(); - } - - public void recordQuotaExceededRequest() { - quotaExceededRequestCountSensor.record(); - } - - public void recordInternalServerErrorRequest() { - internalServerErrorRequestCountSensor.record(); - } - - public void recordServiceUnavailableRequest() { - serviceUnavailableRequestCountSensor.record(); - } - - public void recordLeakedRequest() { - leakedRequestCountSensor.record(); - } - - public void recordOtherErrorRequest() { - otherErrorRequestCountSensor.record(); - } - } } diff --git a/clients/venice-client/src/test/java/com/linkedin/venice/fastclient/DispatchingAvroGenericStoreClientTest.java b/clients/venice-client/src/test/java/com/linkedin/venice/fastclient/DispatchingAvroGenericStoreClientTest.java index 039f5fa1eb..397bf7b9f7 100644 --- a/clients/venice-client/src/test/java/com/linkedin/venice/fastclient/DispatchingAvroGenericStoreClientTest.java +++ b/clients/venice-client/src/test/java/com/linkedin/venice/fastclient/DispatchingAvroGenericStoreClientTest.java @@ -15,7 +15,6 @@ import static org.mockito.Mockito.verify; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; @@ -51,6 +50,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -62,6 +62,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; @@ -402,6 +403,38 @@ private void validateComputeRequestMetrics( numBlockedReplicas); } + private void checkRouteMetricForSingleGet( + String metricName1, + String metricName2, + Consumer conditionCheckFunc) { + assertTrue( + metrics.containsKey(metricName1) || metrics.containsKey(metricName2), + "At least one of the metric must exist for single get metric validation: " + metricName1 + ", " + metricName2); + + for (String metricName: Arrays.asList(metricName1, metricName2)) { + Metric metric = metrics.get(metricName); + if (metric != null) { + conditionCheckFunc.accept(metric.value()); + } + } + } + + private void checkRouteMetricForBatchGet( + String metricName1, + String metricName2, + Consumer conditionCheckFunc) { + assertTrue( + metrics.containsKey(metricName1) && metrics.containsKey(metricName2), + "Metrics must exist for batch get metric validation: " + metricName1 + ", " + metricName2); + + for (String metricName: Arrays.asList(metricName1, metricName2)) { + Metric metric = metrics.get(metricName); + if (metric != null) { + conditionCheckFunc.accept(metric.value()); + } + } + } + private void validateMetrics( GetRequestContext getRequestContext, BatchGetRequestContext batchGetRequestContext, @@ -439,10 +472,46 @@ private void validateMetrics( assertTrue(metrics.get(metricPrefix + "response_tt95pr.Avg").value() > 0); assertTrue(metrics.get(metricPrefix + "response_tt99pr.Avg").value() > 0); assertEquals(batchGetRequestContext.successRequestKeyCount.get(), (int) successKeyCount); + + // Check route metrics for batch-get + String replica1RouterMetricPrefix = + "." + RequestBasedMetadataTestUtils.SERVER_D2_SERVICE + "_" + REPLICA1_NAME + "--multiget_streaming_"; + String replica2RouterMetricPrefix = + "." + RequestBasedMetadataTestUtils.SERVER_D2_SERVICE + "_" + REPLICA2_NAME + "--multiget_streaming_"; + + checkRouteMetricForBatchGet( + replica1RouterMetricPrefix + "healthy_request_count.OccurrenceRate", + replica2RouterMetricPrefix + "healthy_request_count.OccurrenceRate", + v -> assertTrue(v > 0)); + checkRouteMetricForBatchGet( + replica1RouterMetricPrefix + "response_waiting_time.99thPercentile", + replica2RouterMetricPrefix + "response_waiting_time.99thPercentile", + v -> assertTrue(v > 0)); + checkRouteMetricForBatchGet( + replica1RouterMetricPrefix + "quota_exceeded_request_count.OccurrenceRate", + replica2RouterMetricPrefix + "quota_exceeded_request_count.OccurrenceRate", + v -> assertTrue(v == 0.0d)); + } else if (computeRequest) { // Do nothing since we don't have the ComputeRequestContext to test } else { assertEquals(getRequestContext.successRequestKeyCount.get(), (int) successKeyCount); + + // Check route metrics for single-get + String replica1RouterMetricPrefix = "." + RequestBasedMetadataTestUtils.SERVER_D2_SERVICE + "_" + REPLICA1_NAME; + String replica2RouterMetricPrefix = "." + RequestBasedMetadataTestUtils.SERVER_D2_SERVICE + "_" + REPLICA2_NAME; + checkRouteMetricForSingleGet( + replica1RouterMetricPrefix + "--healthy_request_count.OccurrenceRate", + replica2RouterMetricPrefix + "--healthy_request_count.OccurrenceRate", + v -> assertTrue(v > 0)); + checkRouteMetricForSingleGet( + replica1RouterMetricPrefix + "--response_waiting_time.99thPercentile", + replica2RouterMetricPrefix + "--response_waiting_time.99thPercentile", + v -> assertTrue(v > 0)); + checkRouteMetricForSingleGet( + replica1RouterMetricPrefix + "--quota_exceeded_request_count.OccurrenceRate", + replica2RouterMetricPrefix + "--quota_exceeded_request_count.OccurrenceRate", + v -> assertTrue(v == 0.0d)); } } else if (partialHealthyRequest) { assertFalse(metrics.get(metricPrefix + "healthy_request.OccurrenceRate").value() > 0); @@ -476,19 +545,6 @@ private void validateMetrics( if (noAvailableReplicas) { assertTrue(metrics.get(metricPrefix + "no_available_replica_request_count.OccurrenceRate").value() > 0); - TestUtils.waitForNonDeterministicAssertion(5, TimeUnit.SECONDS, () -> { - if (numBlockedReplicas == 2) { - // some test cases only have 1 replica having pending and some have 2. - assertNotNull(metrics.get(routeMetricsPrefix + "_" + REPLICA1_NAME + "--pending_request_count.Max")); - assertEquals( - metrics.get(routeMetricsPrefix + "_" + REPLICA1_NAME + "--pending_request_count.Max").value(), - 1.0); - } - assertNotNull(metrics.get(routeMetricsPrefix + "_" + REPLICA2_NAME + "--pending_request_count.Max")); - assertEquals( - metrics.get(routeMetricsPrefix + "_" + REPLICA2_NAME + "--pending_request_count.Max").value(), - 1.0); - }); assertEquals(metrics.get(routeMetricsPrefix + "--blocked_instance_count.Max").value(), numBlockedReplicas); if (batchGet) { assertTrue(batchGetRequestContext.noAvailableReplica); @@ -507,7 +563,6 @@ private void validateMetrics( assertFalse(getRequestContext.noAvailableReplica); } } - validateRetryMetrics(getRequestContext, batchGetRequestContext, batchGet, computeRequest, metricPrefix); } diff --git a/clients/venice-client/src/test/java/com/linkedin/venice/fastclient/meta/RequestBasedMetadataTestUtils.java b/clients/venice-client/src/test/java/com/linkedin/venice/fastclient/meta/RequestBasedMetadataTestUtils.java index 2afc240c9c..f15571a116 100644 --- a/clients/venice-client/src/test/java/com/linkedin/venice/fastclient/meta/RequestBasedMetadataTestUtils.java +++ b/clients/venice-client/src/test/java/com/linkedin/venice/fastclient/meta/RequestBasedMetadataTestUtils.java @@ -54,6 +54,7 @@ public class RequestBasedMetadataTestUtils { public static final String NEW_REPLICA_NAME = "host3"; public static final String KEY_SCHEMA = "\"string\""; public static final String VALUE_SCHEMA = "\"string\""; + public static final String SERVER_D2_SERVICE = "test-d2-service"; private static final byte[] DICTIONARY = ZstdWithDictCompressor.buildDictionaryOnSyntheticAvroData(); public static ClientConfig getMockClientConfig(String storeName) { @@ -192,8 +193,8 @@ public static D2TransportClient getMockD2TransportClient( public static D2ServiceDiscovery getMockD2ServiceDiscovery(D2TransportClient d2TransportClient, String storeName) { D2ServiceDiscovery d2ServiceDiscovery = mock(D2ServiceDiscovery.class); - D2ServiceDiscoveryResponse d2ServiceDiscoveryResponse = new D2ServiceDiscoveryResponse(); + d2ServiceDiscoveryResponse.setServerD2Service(SERVER_D2_SERVICE); doReturn(d2ServiceDiscoveryResponse).when(d2ServiceDiscovery) .find(eq(d2TransportClient), eq(storeName), anyBoolean()); diff --git a/clients/venice-client/src/test/java/com/linkedin/venice/fastclient/utils/TestClientSimulator.java b/clients/venice-client/src/test/java/com/linkedin/venice/fastclient/utils/TestClientSimulator.java index eff671aa43..15f19abf6d 100644 --- a/clients/venice-client/src/test/java/com/linkedin/venice/fastclient/utils/TestClientSimulator.java +++ b/clients/venice-client/src/test/java/com/linkedin/venice/fastclient/utils/TestClientSimulator.java @@ -554,6 +554,11 @@ public AvroGenericStoreClient getFastClient() { clientConfig = clientConfigBuilder.build(); AbstractStoreMetadata metadata = new AbstractStoreMetadata(clientConfig) { + @Override + public String getClusterName() { + return "test-cluster"; + } + @Override public int getCurrentStoreVersion() { return 1; From 262101c6968abbd47b5d2ea28e4eb9e0c4784778 Mon Sep 17 00:00:00 2001 From: Sourav Maji Date: Fri, 20 Sep 2024 09:38:36 -0700 Subject: [PATCH 05/16] [common][server][dvc-client] Make a true non-blocking end-offset fetcher (#1170) In recent past we have seen cases where end offset fetch call stalls for a very long time. It could happen due to multiple reasons, like general Kafka outage, or mismatched VT vs RT partition count leading to non-existed partition or other network issues. Even though we cache the offset value, there is a blocking call before cache is updated. This hold the lock for a very long time and subsequent calls (either from metics collection) or the ready to serve call waits forever. Since drainer thread is shared this blocks the processing of other resources in a drainer thread leading to cluster-wide impact. This PR check for cache miss and returns immediately with some sentinel value while a nonblocking call updates the cache asynchronously. --------- Co-authored-by: Sourav Maji --- .../kafka/consumer/StoreIngestionTask.java | 14 +++++++++++++- .../consumer/StoreIngestionTaskTest.java | 12 ++++++++++++ .../venice/pubsub/manager/TopicManager.java | 5 +++++ .../pubsub/manager/TopicMetadataFetcher.java | 19 +++++++++++++++++++ .../manager/TopicMetadataFetcherTest.java | 3 +++ 5 files changed, 52 insertions(+), 1 deletion(-) 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 bf3e35fb22..75a8d6c454 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 @@ -102,6 +102,7 @@ import com.linkedin.venice.utils.LatencyUtils; import com.linkedin.venice.utils.PartitionUtils; import com.linkedin.venice.utils.RedundantExceptionFilter; +import com.linkedin.venice.utils.RetryUtils; import com.linkedin.venice.utils.SparseConcurrentList; import com.linkedin.venice.utils.Time; import com.linkedin.venice.utils.Timer; @@ -2219,7 +2220,18 @@ protected long getTopicPartitionEndOffSet(String kafkaUrl, PubSubTopic pubSubTop if (offsetFromConsumer >= 0) { return offsetFromConsumer; } - return getTopicManager(kafkaUrl).getLatestOffsetCached(pubSubTopic, partition); + try { + return RetryUtils.executeWithMaxAttempt(() -> { + long offset = getTopicManager(kafkaUrl).getLatestOffsetCachedNonBlocking(pubSubTopic, partition); + if (offset == -1) { + throw new VeniceException("Found latest offset -1"); + } + return offset; + }, 5, Duration.ofSeconds(1), Collections.singletonList(VeniceException.class)); + } catch (Exception e) { + LOGGER.error("Could not find latest offset for {} even after 5 retries", pubSubTopic.getName()); + return -1; + } } protected long getPartitionOffsetLagBasedOnMetrics(String kafkaSourceAddress, PubSubTopic topic, int partition) { 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 a5b5969a80..d02e59bd9c 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 @@ -3017,6 +3017,9 @@ public void testIsReadyToServe(NodeType nodeType, AAConfig aaConfig, DataReplica doReturn(5L).when(mockTopicManager).getLatestOffsetCached(any(), anyInt()); doReturn(150L).when(mockTopicManagerRemoteKafka).getLatestOffsetCached(any(), anyInt()); doReturn(150L).when(aggKafkaConsumerService).getLatestOffsetBasedOnMetrics(anyString(), any(), any()); + long endOffset = + storeIngestionTaskUnderTest.getTopicPartitionEndOffSet(localKafkaConsumerService.kafkaUrl, pubSubTopic, 1); + assertEquals(endOffset, 150L); if (nodeType == NodeType.LEADER) { // case 6a: leader replica => partition is not ready to serve doReturn(LeaderFollowerStateType.LEADER).when(mockPcsBufferReplayStartedRemoteLagging).getLeaderFollowerState(); @@ -3176,6 +3179,15 @@ public void testActiveActiveStoreIsReadyToServe(HybridConfig hybridConfig, NodeT } else { assertTrue(storeIngestionTaskUnderTest.isReadyToServe(mockPcsMultipleSourceKafkaServers)); } + doReturn(10L).when(aggKafkaConsumerService).getLatestOffsetBasedOnMetrics(anyString(), any(), any()); + long endOffset = + storeIngestionTaskUnderTest.getTopicPartitionEndOffSet(localKafkaConsumerService.kafkaUrl, pubSubTopic, 0); + assertEquals(endOffset, 10L); + doReturn(-1L).when(aggKafkaConsumerService).getLatestOffsetBasedOnMetrics(anyString(), any(), any()); + endOffset = + storeIngestionTaskUnderTest.getTopicPartitionEndOffSet(localKafkaConsumerService.kafkaUrl, pubSubTopic, 0); + assertEquals(endOffset, 0L); + } @DataProvider diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java index 1c3c0ed915..c91d3c452e 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java @@ -731,6 +731,11 @@ public long getLatestOffsetCached(PubSubTopic pubSubTopic, int partitionId) { return topicMetadataFetcher.getLatestOffsetCached(new PubSubTopicPartitionImpl(pubSubTopic, partitionId)); } + public long getLatestOffsetCachedNonBlocking(PubSubTopic pubSubTopic, int partitionId) { + return topicMetadataFetcher + .getLatestOffsetCachedNonBlocking(new PubSubTopicPartitionImpl(pubSubTopic, partitionId)); + } + public long getProducerTimestampOfLastDataMessageWithRetries(PubSubTopicPartition pubSubTopicPartition, int retries) { return topicMetadataFetcher.getProducerTimestampOfLastDataMessageWithRetries(pubSubTopicPartition, retries); } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java index 9c8e80f4ac..4a4e7ac85e 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java @@ -379,6 +379,25 @@ CompletableFuture getLatestOffsetWithRetriesAsync(PubSubTopicPartition pub .supplyAsync(() -> getLatestOffsetWithRetries(pubSubTopicPartition, retries), threadPoolExecutor); } + long getLatestOffsetCachedNonBlocking(PubSubTopicPartition pubSubTopicPartition) { + ValueAndExpiryTime cachedValue; + cachedValue = latestOffsetCache.get(pubSubTopicPartition); + updateCacheAsync( + pubSubTopicPartition, + cachedValue, + latestOffsetCache, + () -> getLatestOffsetWithRetriesAsync( + pubSubTopicPartition, + DEFAULT_MAX_RETRIES_FOR_POPULATING_TMD_CACHE_ENTRY)); + if (cachedValue == null) { + cachedValue = latestOffsetCache.get(pubSubTopicPartition); + if (cachedValue == null) { + return -1; + } + } + return cachedValue.getValue(); + } + long getLatestOffsetCached(PubSubTopicPartition pubSubTopicPartition) { ValueAndExpiryTime cachedValue; try { diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcherTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcherTest.java index 214be223c5..bca7d15629 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcherTest.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcherTest.java @@ -260,6 +260,9 @@ public void testGetTopicLatestOffsets() { assertEquals(res.size(), offsetsMap.size()); assertEquals(res.get(0), 111L); assertEquals(res.get(1), 222L); + assertEquals( + topicMetadataFetcher.getLatestOffsetCachedNonBlocking(new PubSubTopicPartitionImpl(pubSubTopic, 0)), + -1); verify(consumerMock, times(3)).partitionsFor(pubSubTopic); verify(consumerMock, times(1)).endOffsets(eq(offsetsMap.keySet()), any(Duration.class)); From 56199932c93dab6157a5c92acedda9d1c0748cb7 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Fri, 20 Sep 2024 14:07:59 -0700 Subject: [PATCH 06/16] [controller] Add support for controller instance tagging (#1190) * Add ability to set tag on controller instances * Fix tagging and test * Fix imports * Make sure to revert new helix admin * Don't close twice --- .../main/java/com/linkedin/venice/ConfigKeys.java | 2 ++ .../controller/AbstractTestVeniceHelixAdmin.java | 2 ++ .../TestVeniceHelixAdminWithSharedEnvironment.java | 12 ++++++++++++ .../linkedin/venice/controller/HelixAdminClient.java | 5 +++++ .../controller/VeniceControllerClusterConfig.java | 7 +++++++ .../VeniceControllerMultiClusterConfig.java | 4 ++++ .../linkedin/venice/controller/VeniceHelixAdmin.java | 8 ++++++++ .../venice/controller/ZkHelixAdminClient.java | 8 ++++++++ 8 files changed, 48 insertions(+) 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 3aed8933cf..33d6b312ee 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 @@ -214,6 +214,8 @@ private ConfigKeys() { public static final String CONTROLLER_CLUSTER_ZK_ADDRESSS = "controller.cluster.zk.address"; // Name of the Helix cluster for controllers public static final String CONTROLLER_CLUSTER = "controller.cluster.name"; + // What tags to assign to a controller instance + public static final String CONTROLLER_INSTANCE_TAG_LIST = "controller.instance.tag.list"; /** List of forbidden admin paths */ public static final String CONTROLLER_DISABLED_ROUTES = "controller.cluster.disabled.routes"; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/AbstractTestVeniceHelixAdmin.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/AbstractTestVeniceHelixAdmin.java index 8c4a649aa3..5fe353a373 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/AbstractTestVeniceHelixAdmin.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/AbstractTestVeniceHelixAdmin.java @@ -5,6 +5,7 @@ import static com.linkedin.venice.ConfigKeys.CLUSTER_TO_D2; import static com.linkedin.venice.ConfigKeys.CLUSTER_TO_SERVER_D2; import static com.linkedin.venice.ConfigKeys.CONTROLLER_ADD_VERSION_VIA_ADMIN_PROTOCOL; +import static com.linkedin.venice.ConfigKeys.CONTROLLER_INSTANCE_TAG_LIST; import static com.linkedin.venice.ConfigKeys.CONTROLLER_SSL_ENABLED; import static com.linkedin.venice.ConfigKeys.CONTROLLER_SYSTEM_SCHEMA_CLUSTER_NAME; import static com.linkedin.venice.ConfigKeys.DEFAULT_MAX_NUMBER_OF_PARTITIONS; @@ -92,6 +93,7 @@ public void setupCluster(boolean createParticipantStore, MetricsRepository metri properties.put(ADMIN_HELIX_MESSAGING_CHANNEL_ENABLED, true); } properties.put(UNREGISTER_METRIC_FOR_DELETED_STORE_ENABLED, true); + properties.put(CONTROLLER_INSTANCE_TAG_LIST, "GENERAL,TEST"); controllerProps = new VeniceProperties(properties); helixMessageChannelStats = new HelixMessageChannelStats(new MetricsRepository(), clusterName); controllerConfig = new VeniceControllerClusterConfig(controllerProps); diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestVeniceHelixAdminWithSharedEnvironment.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestVeniceHelixAdminWithSharedEnvironment.java index b635940e8e..40dc8d98e6 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestVeniceHelixAdminWithSharedEnvironment.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestVeniceHelixAdminWithSharedEnvironment.java @@ -2054,4 +2054,16 @@ public void testRaceConditionFixForKillOfflinePushAndVersionSwap(boolean isKillO stopParticipant(newNodeId); } + @Test + public void testInstanceTagging() { + List instanceTagList = Arrays.asList("GENERAL", "TEST"); + String controllerClusterName = "venice-controllers"; + + for (String instanceTag: instanceTagList) { + List instances = + veniceAdmin.getHelixAdmin().getInstancesInClusterWithTag(controllerClusterName, instanceTag); + Assert.assertEquals(instances.size(), 1); + } + } + } diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/HelixAdminClient.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/HelixAdminClient.java index 11ecc667c0..4baaed38d3 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/HelixAdminClient.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/HelixAdminClient.java @@ -128,4 +128,9 @@ void createVeniceStorageClusterResources( * Release resources. */ void close(); + + /** + * Adds a tag to an instance + */ + void addInstanceTag(String clusterName, String instanceName, String tag); } diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java index 618aefa8b7..1e0ed716ff 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java @@ -47,6 +47,7 @@ import static com.linkedin.venice.ConfigKeys.CONTROLLER_ENABLE_DISABLED_REPLICA_ENABLED; import static com.linkedin.venice.ConfigKeys.CONTROLLER_ENFORCE_SSL; import static com.linkedin.venice.ConfigKeys.CONTROLLER_HAAS_SUPER_CLUSTER_NAME; +import static com.linkedin.venice.ConfigKeys.CONTROLLER_INSTANCE_TAG_LIST; import static com.linkedin.venice.ConfigKeys.CONTROLLER_IN_AZURE_FABRIC; import static com.linkedin.venice.ConfigKeys.CONTROLLER_JETTY_CONFIG_OVERRIDE_PREFIX; import static com.linkedin.venice.ConfigKeys.CONTROLLER_MIN_SCHEMA_COUNT_TO_KEEP; @@ -223,6 +224,7 @@ public class VeniceControllerClusterConfig { // Name of the Helix cluster for controllers private final String controllerClusterName; private final String controllerClusterZkAddress; + private final List controllerInstanceTagList; private final boolean multiRegion; private final boolean parent; private final ParentControllerRegionState parentControllerRegionState; @@ -635,6 +637,7 @@ public VeniceControllerClusterConfig(VeniceProperties props) { */ this.adminCheckReadMethodForKafka = props.getBoolean(ADMIN_CHECK_READ_METHOD_FOR_KAFKA, true); this.controllerClusterName = props.getString(CONTROLLER_CLUSTER, "venice-controllers"); + this.controllerInstanceTagList = props.getList(CONTROLLER_INSTANCE_TAG_LIST, Collections.emptyList()); this.controllerClusterReplica = props.getInt(CONTROLLER_CLUSTER_REPLICA, 3); this.controllerClusterZkAddress = props.getString(CONTROLLER_CLUSTER_ZK_ADDRESSS, getZkAddress()); this.parent = props.getBoolean(CONTROLLER_PARENT_MODE, false); @@ -1161,6 +1164,10 @@ public String getControllerClusterName() { return controllerClusterName; } + public List getControllerInstanceTagList() { + return controllerInstanceTagList; + } + public String getControllerClusterZkAddress() { return controllerClusterZkAddress; } diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerMultiClusterConfig.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerMultiClusterConfig.java index 63ce26155e..2f9ea6d8a5 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerMultiClusterConfig.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerMultiClusterConfig.java @@ -282,4 +282,8 @@ public int getDefaultMaxRecordSizeBytes() { public long getServiceDiscoveryRegistrationRetryMS() { return getCommonConfig().getServiceDiscoveryRegistrationRetryMS(); } + + public List getControllerInstanceTagList() { + return getCommonConfig().getControllerInstanceTagList(); + } } 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 517f0f0a4c..51e55e8c45 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 @@ -671,6 +671,7 @@ public VeniceHelixAdmin( if (!multiClusterConfigs.getControllerConfig(clusterName).isErrorLeaderReplicaFailOverEnabled()) { continue; } + HelixLiveInstanceMonitor liveInstanceMonitor = new HelixLiveInstanceMonitor(this.zkClient, clusterName); DisabledPartitionStats disabledPartitionStats = new DisabledPartitionStats(metricsRepository, clusterName); disabledPartitionStatMap.put(clusterName, disabledPartitionStats); @@ -793,6 +794,13 @@ private synchronized void connectToControllerCluster() { } controllerClusterKeyBuilder = new PropertyKey.Builder(tempManager.getClusterName()); helixManager = tempManager; + + List instanceTagList = multiClusterConfigs.getControllerInstanceTagList(); + for (String instanceTag: instanceTagList) { + helixAdminClient.addInstanceTag(controllerClusterName, helixManager.getInstanceName(), instanceTag); + } + LOGGER.info("Connected to controller cluster {} with controller {}", controllerClusterName, controllerName); + } public ZkClient getZkClient() { diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/ZkHelixAdminClient.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/ZkHelixAdminClient.java index dd740ff7d1..cad3dfd036 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/ZkHelixAdminClient.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/ZkHelixAdminClient.java @@ -314,4 +314,12 @@ public void resetPartition( public void close() { helixAdmin.close(); } + + /** + * @see HelixAdminClient#addInstanceTag(String, String, String)() + */ + @Override + public void addInstanceTag(String clusterName, String instanceName, String tag) { + helixAdmin.addInstanceTag(clusterName, instanceName, tag); + } } From 328d72ab1e50df97b308970325bc540cee7d349c Mon Sep 17 00:00:00 2001 From: Manoj Nagarajan Date: Mon, 23 Sep 2024 23:12:52 -0700 Subject: [PATCH 07/16] [vpj][controller] Emit push job status metrics from controller (#1185) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit VPJ communicates with the controller to write PushJobDetails to the PUSH_JOB_DETAILS_STORE_NAME system store. This PR introduces new metrics emitted by the controller for push job success/failure. New Metrics Added (Count and CountSinceLastMeasurement added in Tehuti, hence using tehuti:0.12.2): batch_push_job_success, batch_push_job_failed_user_error, batch_push_job_failed_non_user_error incremental_push_job_success, incremental_push_job_failed_user_error, incremental_push_job_failed_non_user_error Current flow is VPJ checks 'push.job.status.upload.enable' config and sends PushJobDetails to '/send_push_job_details' path in Venice-controller, which writes it to the push job details system store. Derive success/failure in the controller and emit metrics, tying metric emission and push job details upload via the same config. This config is enabled everywhere, and this approach doesn’t require deployment ordering (controllers -> VPJ) unlike other options and no schema evolution needed. Config introduced: Added parent controller config 'push.job.failure.checkpoints.to.define.user.error' to provide a custom list of these checkpoints based on the use case to emit the metrics accordingly. DEFAULT_PUSH_JOB_USER_ERROR_CHECKPOINTS will be used by default. --- build.gradle | 2 +- .../linkedin/venice/hadoop/VenicePushJob.java | 44 +- .../hadoop/TestVenicePushJobCheckpoints.java | 195 ++++---- .../venice/hadoop/VenicePushJobTest.java | 5 +- .../venice/pushmonitor/ExecutionStatus.java | 27 +- .../com/linkedin/venice/utils/EnumUtils.java | 32 ++ .../pushmonitor/ExecutionStatusTest.java | 83 ++-- .../linkedin/venice/utils/EnumUtilsTest.java | 152 +++++++ .../java/com/linkedin/venice/ConfigKeys.java | 13 + .../linkedin/venice/PushJobCheckpoints.java | 64 +++ .../controllerapi/ControllerClient.java | 8 - .../venice/status/PushJobDetailsStatus.java | 22 +- .../venice/PushJobCheckPointsTest.java | 82 ++++ .../status/TestPushJobDetailsStatus.java | 65 +++ .../endToEnd/DaVinciClientDiskFullTest.java | 7 +- .../venice/endToEnd/PushJobDetailsTest.java | 427 ++++++++++++------ ...tFatalDataValidationExceptionHandling.java | 4 +- .../venice/utils/VeniceEnumValueTest.java | 48 +- .../VeniceControllerClusterConfig.java | 27 ++ .../venice/controller/VeniceHelixAdmin.java | 73 +++ .../venice/controller/server/JobRoutes.java | 17 +- .../controller/stats/PushJobStatusStats.java | 60 +++ .../venice/pushmonitor/PushMonitorUtils.java | 4 +- .../controller/TestPushJobStatusStats.java | 113 +++++ .../TestVeniceControllerClusterConfig.java | 57 ++- 25 files changed, 1247 insertions(+), 384 deletions(-) create mode 100644 internal/venice-client-common/src/test/java/com/linkedin/venice/utils/EnumUtilsTest.java create mode 100644 internal/venice-common/src/main/java/com/linkedin/venice/PushJobCheckpoints.java create mode 100644 internal/venice-common/src/test/java/com/linkedin/venice/PushJobCheckPointsTest.java create mode 100644 internal/venice-common/src/test/java/com/linkedin/venice/status/TestPushJobDetailsStatus.java create mode 100644 services/venice-controller/src/main/java/com/linkedin/venice/controller/stats/PushJobStatusStats.java create mode 100644 services/venice-controller/src/test/java/com/linkedin/venice/controller/TestPushJobStatusStats.java diff --git a/build.gradle b/build.gradle index 5535754d07..e6ad01275b 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', 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 36eaae8def..e593a97caf 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.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; 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; @@ -259,35 +260,9 @@ public class VenicePushJob implements AutoCloseable { private InputStorageQuotaTracker inputStorageQuotaTracker; private final PushJobHeartbeatSenderFactory pushJobHeartbeatSenderFactory; + private PushJobHeartbeatSender pushJobHeartbeatSender = null; 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 @@ -675,7 +650,6 @@ DataWriterComputeJob getDataWriterComputeJob() { * @throws VeniceException */ public void run() { - PushJobHeartbeatSender pushJobHeartbeatSender = null; try { Optional sslFactory = VPJSSLUtils.createSSLFactory( pushJobSetting.enableSSL, @@ -874,8 +848,6 @@ public void run() { pushJobDetails.overallStatus.add(getPushJobDetailsStatusTuple(PushJobDetailsStatus.COMPLETED.getValue())); } pushJobDetails.jobDurationInMs = LatencyUtils.getElapsedTimeFromMsToMs(pushJobSetting.jobStartTimeMs); - updatePushJobDetailsWithConfigs(); - updatePushJobDetailsWithLivenessHeartbeatException(pushJobHeartbeatSender); sendPushJobDetailsToController(); // only kick off the validation and post-validation flow when everything has to be done in a single VPJ @@ -903,8 +875,6 @@ public void run() { pushJobDetails.overallStatus.add(getPushJobDetailsStatusTuple(PushJobDetailsStatus.ERROR.getValue())); pushJobDetails.failureDetails = e.toString(); pushJobDetails.jobDurationInMs = LatencyUtils.getElapsedTimeFromMsToMs(pushJobSetting.jobStartTimeMs); - updatePushJobDetailsWithConfigs(); - updatePushJobDetailsWithLivenessHeartbeatException(pushJobHeartbeatSender); sendPushJobDetailsToController(); closeVeniceWriter(); } catch (Exception ex) { @@ -924,6 +894,7 @@ public void run() { Utils.closeQuietlyWithErrorLogged(inputDataInfoProvider); if (pushJobHeartbeatSender != null) { pushJobHeartbeatSender.stop(); + pushJobHeartbeatSender = null; } inputDataInfoProvider = null; if (pushJobSetting.rmdSchemaDir != null) { @@ -1018,7 +989,7 @@ private PushJobHeartbeatSender createPushJobHeartbeatSender(final boolean sslEna } } - private void updatePushJobDetailsWithLivenessHeartbeatException(PushJobHeartbeatSender pushJobHeartbeatSender) { + private void updatePushJobDetailsWithLivenessHeartbeatException() { if (pushJobHeartbeatSender == null || this.pushJobDetails == null) { return; } @@ -1840,6 +1811,12 @@ private void sendPushJobDetailsToController() { LOGGER.warn("Unable to send push job details for monitoring purpose. The payload was not populated properly"); return; } + + // update push job details with more info if needed + updatePushJobDetailsWithConfigs(); + updatePushJobDetailsWithLivenessHeartbeatException(); + + // send push job details to controller try { pushJobDetails.reportTimestamp = System.currentTimeMillis(); int version = pushJobSetting.version <= 0 ? UNCREATED_VERSION_NUMBER : pushJobSetting.version; @@ -2779,7 +2756,6 @@ public void cancel() { pushJobDetails.overallStatus.add(getPushJobDetailsStatusTuple(PushJobDetailsStatus.KILLED.getValue())); } pushJobDetails.jobDurationInMs = LatencyUtils.getElapsedTimeFromMsToMs(pushJobSetting.jobStartTimeMs); - updatePushJobDetailsWithConfigs(); sendPushJobDetailsToController(); } 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 3561508cd6..c82c9789d6 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 9de5bd827f..c5161eb82a 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/internal/venice-client-common/src/main/java/com/linkedin/venice/pushmonitor/ExecutionStatus.java b/internal/venice-client-common/src/main/java/com/linkedin/venice/pushmonitor/ExecutionStatus.java index 0eecd1bcb8..8902b68f3d 100644 --- a/internal/venice-client-common/src/main/java/com/linkedin/venice/pushmonitor/ExecutionStatus.java +++ b/internal/venice-client-common/src/main/java/com/linkedin/venice/pushmonitor/ExecutionStatus.java @@ -1,8 +1,8 @@ package com.linkedin.venice.pushmonitor; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; +import com.linkedin.venice.utils.EnumUtils; +import com.linkedin.venice.utils.VeniceEnumValue; +import java.util.List; /** @@ -21,7 +21,7 @@ * * TODO: Break this up in JobExecutionStatus and TaskExecutionStatus. It's pretty confusing to mix them ): ... */ -public enum ExecutionStatus { +public enum ExecutionStatus implements VeniceEnumValue { /** Job doesn't yet exist */ NOT_CREATED(true, false, false, false, 0), @@ -139,6 +139,8 @@ public enum ExecutionStatus { this.value = value; } + private static final List TYPES = EnumUtils.getEnumValuesList(ExecutionStatus.class); + /** * Some of the statuses are like watermark. These statuses are used in {@link PushMonitor} and * {@link com.linkedin.venice.router.api.VeniceVersionFinder} to determine whether a job is finished @@ -178,24 +180,13 @@ public static boolean isIncrementalPushStatus(int statusVal) { || statusVal == END_OF_INCREMENTAL_PUSH_RECEIVED.getValue(); } + @Override public int getValue() { return value; } - /** - * Get ExecutionStatus from integer ordinal value in avro. - */ - private static final Map idMapping = new HashMap<>(); - static { - Arrays.stream(values()).forEach(s -> idMapping.put(s.value, s)); - } - - public static ExecutionStatus fromInt(int v) { - ExecutionStatus status = idMapping.get(v); - if (status == null) { - return ExecutionStatus.UNKNOWN; - } - return status; + public static ExecutionStatus valueOf(int value) { + return EnumUtils.valueOf(TYPES, value, ExecutionStatus.class); } public ExecutionStatus getRootStatus() { 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 417d73a952..f24f975fc9 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 @@ -4,7 +4,9 @@ import java.lang.reflect.Array; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.function.Function; @@ -56,6 +58,21 @@ public static List getEnumValuesList(Class enu return Collections.unmodifiableList(Arrays.asList(array)); } + /** + * This is a relaxed version of {@link #getEnumValuesList(Class)} which returns a map instead of a list. + * This is useful when the values are not contiguous, or when the values are not starting from 0. + */ + public static Map getEnumValuesSparseList(Class enumToProvideArrayOf) { + String name = enumToProvideArrayOf.getSimpleName(); + Map map = new HashMap<>(); + for (V type: enumToProvideArrayOf.getEnumConstants()) { + if (map.put(type.getValue(), type) != null) { + throw new IllegalStateException(name + " values must be unique!"); + } + } + return Collections.unmodifiableMap(map); + } + public static V valueOf(List valuesList, int value, Class enumClass) { return valueOf(valuesList, value, enumClass, VeniceException::new); } @@ -71,4 +88,19 @@ public static V valueOf( throw exceptionConstructor.apply("Invalid enum value for " + enumClass.getSimpleName() + ": " + value); } } + + public static V valueOf(Map valuesMap, int value, Class enumClass) { + return valueOf(valuesMap, value, enumClass, VeniceException::new); + } + + public static V valueOf( + Map valuesMap, + int value, + Class enumClass, + Function exceptionConstructor) { + if (!valuesMap.containsKey(value)) { + throw exceptionConstructor.apply("Invalid enum value for " + enumClass.getSimpleName() + ": " + value); + } + return valuesMap.get(value); + } } diff --git a/internal/venice-client-common/src/test/java/com/linkedin/venice/pushmonitor/ExecutionStatusTest.java b/internal/venice-client-common/src/test/java/com/linkedin/venice/pushmonitor/ExecutionStatusTest.java index 19db8b8b56..6f2d1f5ce1 100644 --- a/internal/venice-client-common/src/test/java/com/linkedin/venice/pushmonitor/ExecutionStatusTest.java +++ b/internal/venice-client-common/src/test/java/com/linkedin/venice/pushmonitor/ExecutionStatusTest.java @@ -1,9 +1,7 @@ package com.linkedin.venice.pushmonitor; import static com.linkedin.venice.pushmonitor.ExecutionStatus.ARCHIVED; -import static com.linkedin.venice.pushmonitor.ExecutionStatus.CATCH_UP_BASE_TOPIC_OFFSET_LAG; import static com.linkedin.venice.pushmonitor.ExecutionStatus.COMPLETED; -import static com.linkedin.venice.pushmonitor.ExecutionStatus.DATA_RECOVERY_COMPLETED; import static com.linkedin.venice.pushmonitor.ExecutionStatus.DROPPED; import static com.linkedin.venice.pushmonitor.ExecutionStatus.DVC_INGESTION_ERROR_DISK_FULL; import static com.linkedin.venice.pushmonitor.ExecutionStatus.DVC_INGESTION_ERROR_MEMORY_LIMIT_REACHED; @@ -12,21 +10,15 @@ import static com.linkedin.venice.pushmonitor.ExecutionStatus.END_OF_INCREMENTAL_PUSH_RECEIVED; import static com.linkedin.venice.pushmonitor.ExecutionStatus.END_OF_PUSH_RECEIVED; import static com.linkedin.venice.pushmonitor.ExecutionStatus.ERROR; -import static com.linkedin.venice.pushmonitor.ExecutionStatus.NEW; -import static com.linkedin.venice.pushmonitor.ExecutionStatus.NOT_CREATED; -import static com.linkedin.venice.pushmonitor.ExecutionStatus.NOT_STARTED; -import static com.linkedin.venice.pushmonitor.ExecutionStatus.PROGRESS; import static com.linkedin.venice.pushmonitor.ExecutionStatus.STARTED; -import static com.linkedin.venice.pushmonitor.ExecutionStatus.START_OF_BUFFER_REPLAY_RECEIVED; -import static com.linkedin.venice.pushmonitor.ExecutionStatus.START_OF_INCREMENTAL_PUSH_RECEIVED; -import static com.linkedin.venice.pushmonitor.ExecutionStatus.TOPIC_SWITCH_RECEIVED; -import static com.linkedin.venice.pushmonitor.ExecutionStatus.UNKNOWN; import static com.linkedin.venice.pushmonitor.ExecutionStatus.WARNING; import static com.linkedin.venice.pushmonitor.ExecutionStatus.isDeterminedStatus; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; +import com.linkedin.alpini.base.misc.CollectionUtil; +import com.linkedin.venice.utils.VeniceEnumValueTest; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -35,7 +27,39 @@ import org.testng.annotations.Test; -public class ExecutionStatusTest { +public class ExecutionStatusTest extends VeniceEnumValueTest { + public ExecutionStatusTest() { + super(ExecutionStatus.class); + } + + @Override + protected Map expectedMapping() { + return CollectionUtil.mapBuilder() + .put(0, ExecutionStatus.NOT_CREATED) + .put(1, ExecutionStatus.NEW) + .put(2, ExecutionStatus.STARTED) + .put(3, ExecutionStatus.PROGRESS) + .put(4, ExecutionStatus.END_OF_PUSH_RECEIVED) + .put(5, ExecutionStatus.START_OF_BUFFER_REPLAY_RECEIVED) + .put(6, ExecutionStatus.TOPIC_SWITCH_RECEIVED) + .put(7, ExecutionStatus.START_OF_INCREMENTAL_PUSH_RECEIVED) + .put(8, ExecutionStatus.END_OF_INCREMENTAL_PUSH_RECEIVED) + .put(9, ExecutionStatus.DROPPED) + .put(10, ExecutionStatus.COMPLETED) + .put(11, ExecutionStatus.WARNING) + .put(12, ExecutionStatus.ERROR) + .put(13, ExecutionStatus.CATCH_UP_BASE_TOPIC_OFFSET_LAG) + .put(14, ExecutionStatus.ARCHIVED) + .put(15, ExecutionStatus.UNKNOWN) + .put(16, ExecutionStatus.NOT_STARTED) + .put(17, ExecutionStatus.DATA_RECOVERY_COMPLETED) + .put(18, ExecutionStatus.DVC_INGESTION_ERROR_DISK_FULL) + .put(19, ExecutionStatus.DVC_INGESTION_ERROR_MEMORY_LIMIT_REACHED) + .put(20, ExecutionStatus.DVC_INGESTION_ERROR_TOO_MANY_DEAD_INSTANCES) + .put(21, ExecutionStatus.DVC_INGESTION_ERROR_OTHER) + .build(); + } + @Test public void testisDVCIngestionError() { for (ExecutionStatus status: ExecutionStatus.values()) { @@ -73,43 +97,6 @@ public void testIsErrorWithInputString() { assertFalse(ExecutionStatus.isError("123")); } - /** - * Test to prevent unintentional changing of the values of ExecutionStatus - * as values are persisted and used across services - */ - @Test - public void testExecutionStatusValue() { - assertEquals(ExecutionStatus.values().length, 22); - - for (ExecutionStatus status: ExecutionStatus.values()) { - assertEquals(status.getValue(), status.ordinal()); - } - - // check all the values in the enum one by one to make sure it's not changed - assertEquals(NOT_CREATED.getValue(), 0); - assertEquals(NEW.getValue(), 1); - assertEquals(STARTED.getValue(), 2); - assertEquals(PROGRESS.getValue(), 3); - assertEquals(END_OF_PUSH_RECEIVED.getValue(), 4); - assertEquals(START_OF_BUFFER_REPLAY_RECEIVED.getValue(), 5); - assertEquals(TOPIC_SWITCH_RECEIVED.getValue(), 6); - assertEquals(START_OF_INCREMENTAL_PUSH_RECEIVED.getValue(), 7); - assertEquals(END_OF_INCREMENTAL_PUSH_RECEIVED.getValue(), 8); - assertEquals(DROPPED.getValue(), 9); - assertEquals(COMPLETED.getValue(), 10); - assertEquals(WARNING.getValue(), 11); - assertEquals(ERROR.getValue(), 12); - assertEquals(CATCH_UP_BASE_TOPIC_OFFSET_LAG.getValue(), 13); - assertEquals(ARCHIVED.getValue(), 14); - assertEquals(UNKNOWN.getValue(), 15); - assertEquals(NOT_STARTED.getValue(), 16); - assertEquals(DATA_RECOVERY_COMPLETED.getValue(), 17); - assertEquals(DVC_INGESTION_ERROR_DISK_FULL.getValue(), 18); - assertEquals(DVC_INGESTION_ERROR_MEMORY_LIMIT_REACHED.getValue(), 19); - assertEquals(DVC_INGESTION_ERROR_TOO_MANY_DEAD_INSTANCES.getValue(), 20); - assertEquals(DVC_INGESTION_ERROR_OTHER.getValue(), 21); - } - @Test public void testIsUsedByDaVinciClientOnly() { Set dvcOnly = new HashSet<>(); diff --git a/internal/venice-client-common/src/test/java/com/linkedin/venice/utils/EnumUtilsTest.java b/internal/venice-client-common/src/test/java/com/linkedin/venice/utils/EnumUtilsTest.java new file mode 100644 index 0000000000..d099b50799 --- /dev/null +++ b/internal/venice-client-common/src/test/java/com/linkedin/venice/utils/EnumUtilsTest.java @@ -0,0 +1,152 @@ +package com.linkedin.venice.utils; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertThrows; + +import com.linkedin.venice.exceptions.VeniceException; +import java.util.List; +import java.util.Map; +import org.testng.annotations.Test; + + +public class EnumUtilsTest { + private enum ValidEnum implements VeniceEnumValue { + A(0), B(1), C(2); + + private final int value; + + ValidEnum(int value) { + this.value = value; + } + + @Override + public int getValue() { + return value; + } + } + + private enum InvalidEnum implements VeniceEnumValue { + INVALID(-1); + + private final int value; + + InvalidEnum(int value) { + this.value = value; + } + + @Override + public int getValue() { + return value; + } + } + + private enum DuplicateEnum implements VeniceEnumValue { + A(0), B(1), C(1); + + private final int value; + + DuplicateEnum(int value) { + this.value = value; + } + + @Override + public int getValue() { + return value; + } + } + + private enum GapEnum implements VeniceEnumValue { + A(0), B(2); + + private final int value; + + GapEnum(int value) { + this.value = value; + } + + @Override + public int getValue() { + return value; + } + } + + private enum ValidEnumSparse implements VeniceEnumValue { + A(10), B(20), C(30); + + private final int value; + + ValidEnumSparse(int value) { + this.value = value; + } + + @Override + public int getValue() { + return value; + } + } + + @Test + public void testGetEnumValuesList() { + + List values = EnumUtils.getEnumValuesList(ValidEnum.class); + assertEquals(3, values.size()); + assertEquals(ValidEnum.A, values.get(0)); + assertEquals(ValidEnum.B, values.get(1)); + assertEquals(ValidEnum.C, values.get(2)); + + // Test with negative value + assertThrows(IllegalStateException.class, () -> { + EnumUtils.getEnumValuesList(InvalidEnum.class); + }); + + // Test with duplicate values + assertThrows(IllegalStateException.class, () -> { + EnumUtils.getEnumValuesList(DuplicateEnum.class); + }); + + // Test with gaps + assertThrows(IllegalStateException.class, () -> { + EnumUtils.getEnumValuesList(GapEnum.class); + }); + } + + @Test + public void testGetEnumValuesSparseList() { + Map values = EnumUtils.getEnumValuesSparseList(ValidEnumSparse.class); + assertEquals(3, values.size()); + assertEquals(ValidEnumSparse.A, values.get(10)); + assertEquals(ValidEnumSparse.B, values.get(20)); + assertEquals(ValidEnumSparse.C, values.get(30)); + + // Test with negative value + Map invalidEnumValues = EnumUtils.getEnumValuesSparseList(InvalidEnum.class); + assertEquals(1, invalidEnumValues.size()); + assertEquals(InvalidEnum.INVALID, invalidEnumValues.get(-1)); + + // Test with duplicate values + assertThrows(IllegalStateException.class, () -> { + EnumUtils.getEnumValuesSparseList(DuplicateEnum.class); + }); + + // Test with gaps + EnumUtils.getEnumValuesSparseList(GapEnum.class); + } + + @Test + public void testValueOf() { + List valuesList = EnumUtils.getEnumValuesList(ValidEnum.class); + Map valuesMap = EnumUtils.getEnumValuesSparseList(ValidEnum.class); + + assertEquals(ValidEnum.A, EnumUtils.valueOf(valuesList, 0, ValidEnum.class)); + assertEquals(ValidEnum.B, EnumUtils.valueOf(valuesList, 1, ValidEnum.class)); + assertEquals(ValidEnum.C, EnumUtils.valueOf(valuesList, 2, ValidEnum.class)); + + assertEquals(ValidEnum.A, EnumUtils.valueOf(valuesMap, 0, ValidEnum.class)); + assertEquals(ValidEnum.B, EnumUtils.valueOf(valuesMap, 1, ValidEnum.class)); + assertEquals(ValidEnum.C, EnumUtils.valueOf(valuesMap, 2, ValidEnum.class)); + + // Test invalid value + assertThrows(VeniceException.class, () -> EnumUtils.valueOf(valuesList, 3, ValidEnum.class)); + assertThrows(VeniceException.class, () -> EnumUtils.valueOf(valuesMap, 3, ValidEnum.class)); + } +} 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 33d6b312ee..5560a6304e 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 @@ -2172,6 +2172,19 @@ private ConfigKeys() { public static final String CONTROLLER_DANGLING_TOPIC_OCCURRENCE_THRESHOLD_FOR_CLEANUP = "controller.dangling.topic.occurrence.threshold.for.cleanup"; + /** + * Configure in controllers to provide a custom list of checkpoints to define user errors and to + * override the default checkpoint list {@link PushJobCheckpoints#DEFAULT_PUSH_JOB_USER_ERROR_CHECKPOINTS}. + * This is useful to emit the push job failure metrics due to user errors or not due + * to user errors based on the custom checkpoint list. + * + * Check {@link PushJobCheckpoints} for the list of supported checkpoints: Config should contain one or more + * of the checkpoints strings separated by comma. In case of invalid config, the default list of checkpoints + * will be used. + */ + public static final String PUSH_JOB_FAILURE_CHECKPOINTS_TO_DEFINE_USER_ERROR = + "push.job.failure.checkpoints.to.define.user.error"; + /** * Config for the default value which is filled in when the store-level config * {@link com.linkedin.venice.writer.VeniceWriter#maxRecordSizeBytes} is left unset. Used as a controller config for 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 0000000000..d5156e5ff6 --- /dev/null +++ b/internal/venice-common/src/main/java/com/linkedin/venice/PushJobCheckpoints.java @@ -0,0 +1,64 @@ +package com.linkedin.venice; + +import com.linkedin.venice.utils.EnumUtils; +import com.linkedin.venice.utils.VeniceEnumValue; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + + +/** + * 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 implements VeniceEnumValue { + 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; + + /** + * Default set checkpoints to define push job failures are user errors which can be overridden via controller config + * {@link ConfigKeys#PUSH_JOB_FAILURE_CHECKPOINTS_TO_DEFINE_USER_ERROR} + */ + public static final Set DEFAULT_PUSH_JOB_USER_ERROR_CHECKPOINTS = Collections.unmodifiableSet( + new HashSet<>( + Arrays.asList( + QUOTA_EXCEEDED, + WRITE_ACL_FAILED, + DUP_KEY_WITH_DIFF_VALUE, + INPUT_DATA_SCHEMA_VALIDATION_FAILED, + EXTENDED_INPUT_DATA_SCHEMA_VALIDATION_FAILED, + RECORD_TOO_LARGE_FAILED, + CONCURRENT_BATCH_PUSH, + DATASET_CHANGED, + INVALID_INPUT_FILE, + DVC_INGESTION_ERROR_DISK_FULL, + DVC_INGESTION_ERROR_MEMORY_LIMIT_REACHED))); + + private static final Map TYPES = + EnumUtils.getEnumValuesSparseList(PushJobCheckpoints.class); + + PushJobCheckpoints(int value) { + this.value = value; + } + + @Override + public int getValue() { + return value; + } + + public static PushJobCheckpoints valueOf(int value) { + return EnumUtils.valueOf(TYPES, value, PushJobCheckpoints.class); + } +} 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 e8853edc39..5d812b4463 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/status/PushJobDetailsStatus.java b/internal/venice-common/src/main/java/com/linkedin/venice/status/PushJobDetailsStatus.java index c5ba1a618d..9e96fd5cf4 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,22 +1,30 @@ package com.linkedin.venice.status; +import com.linkedin.venice.utils.EnumUtils; +import com.linkedin.venice.utils.VeniceEnumValue; import java.util.Arrays; import java.util.HashSet; +import java.util.List; import java.util.Set; -public enum PushJobDetailsStatus { +public enum PushJobDetailsStatus implements VeniceEnumValue { STARTED(0), COMPLETED(1), ERROR(2), NOT_CREATED(3), UNKNOWN(4), TOPIC_CREATED(5), DATA_WRITER_COMPLETED(6), KILLED(7), END_OF_PUSH_RECEIVED(8), START_OF_INCREMENTAL_PUSH_RECEIVED(9), END_OF_INCREMENTAL_PUSH_RECEIVED(10); 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) { this.value = value; } + private static final List TYPES = EnumUtils.getEnumValuesList(PushJobDetailsStatus.class); + + @Override public int getValue() { return value; } @@ -24,4 +32,16 @@ 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); + } + + public static PushJobDetailsStatus valueOf(int value) { + return EnumUtils.valueOf(TYPES, value, PushJobDetailsStatus.class); + } } diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/PushJobCheckPointsTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/PushJobCheckPointsTest.java new file mode 100644 index 0000000000..eb282b6352 --- /dev/null +++ b/internal/venice-common/src/test/java/com/linkedin/venice/PushJobCheckPointsTest.java @@ -0,0 +1,82 @@ +package com.linkedin.venice; + +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; + +import com.linkedin.alpini.base.misc.CollectionUtil; +import com.linkedin.venice.utils.VeniceEnumValueTest; +import java.util.Map; +import org.testng.annotations.Test; + + +public class PushJobCheckPointsTest extends VeniceEnumValueTest { + public PushJobCheckPointsTest() { + super(PushJobCheckpoints.class); + } + + @Override + protected Map expectedMapping() { + return CollectionUtil.mapBuilder() + .put(0, PushJobCheckpoints.INITIALIZE_PUSH_JOB) + .put(1, PushJobCheckpoints.NEW_VERSION_CREATED) + .put(2, PushJobCheckpoints.START_DATA_WRITER_JOB) + .put(3, PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED) + .put(4, PushJobCheckpoints.START_JOB_STATUS_POLLING) + .put(5, PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED) + .put(6, PushJobCheckpoints.START_VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB) + .put(7, PushJobCheckpoints.VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB_COMPLETED) + .put(-1, PushJobCheckpoints.QUOTA_EXCEEDED) + .put(-2, PushJobCheckpoints.WRITE_ACL_FAILED) + .put(-3, PushJobCheckpoints.DUP_KEY_WITH_DIFF_VALUE) + .put(-4, PushJobCheckpoints.INPUT_DATA_SCHEMA_VALIDATION_FAILED) + .put(-5, PushJobCheckpoints.EXTENDED_INPUT_DATA_SCHEMA_VALIDATION_FAILED) + .put(-6, PushJobCheckpoints.RECORD_TOO_LARGE_FAILED) + .put(-7, PushJobCheckpoints.CONCURRENT_BATCH_PUSH) + .put(-8, PushJobCheckpoints.DATASET_CHANGED) + .put(-9, PushJobCheckpoints.INVALID_INPUT_FILE) + .put(-10, PushJobCheckpoints.ZSTD_DICTIONARY_CREATION_FAILED) + .put(-11, PushJobCheckpoints.DVC_INGESTION_ERROR_DISK_FULL) + .put(-12, PushJobCheckpoints.DVC_INGESTION_ERROR_MEMORY_LIMIT_REACHED) + .put(-13, PushJobCheckpoints.DVC_INGESTION_ERROR_TOO_MANY_DEAD_INSTANCES) + .put(-14, PushJobCheckpoints.DVC_INGESTION_ERROR_OTHER) + .build(); + } + + @Test + public void testDefaultPushJobUserErrorCheckpoints() { + for (PushJobCheckpoints checkpoint: PushJobCheckpoints.values()) { + 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: + assertTrue(PushJobCheckpoints.DEFAULT_PUSH_JOB_USER_ERROR_CHECKPOINTS.contains(checkpoint)); + 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: + assertFalse(PushJobCheckpoints.DEFAULT_PUSH_JOB_USER_ERROR_CHECKPOINTS.contains(checkpoint)); + break; + + default: + throw new IllegalArgumentException("Unknown checkpoint: " + checkpoint); + } + } + } +} 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 0000000000..1941c315f6 --- /dev/null +++ b/internal/venice-common/src/test/java/com/linkedin/venice/status/TestPushJobDetailsStatus.java @@ -0,0 +1,65 @@ +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 com.linkedin.alpini.base.misc.CollectionUtil; +import com.linkedin.venice.utils.VeniceEnumValueTest; +import java.util.Map; +import org.testng.annotations.Test; + + +public class TestPushJobDetailsStatus extends VeniceEnumValueTest { + public TestPushJobDetailsStatus() { + super(PushJobDetailsStatus.class); + } + + @Override + protected Map expectedMapping() { + return CollectionUtil.mapBuilder() + .put(0, PushJobDetailsStatus.STARTED) + .put(1, PushJobDetailsStatus.COMPLETED) + .put(2, PushJobDetailsStatus.ERROR) + .put(3, PushJobDetailsStatus.NOT_CREATED) + .put(4, PushJobDetailsStatus.UNKNOWN) + .put(5, PushJobDetailsStatus.TOPIC_CREATED) + .put(6, PushJobDetailsStatus.DATA_WRITER_COMPLETED) + .put(7, PushJobDetailsStatus.KILLED) + .put(8, PushJobDetailsStatus.END_OF_PUSH_RECEIVED) + .put(9, PushJobDetailsStatus.START_OF_INCREMENTAL_PUSH_RECEIVED) + .put(10, PushJobDetailsStatus.END_OF_INCREMENTAL_PUSH_RECEIVED) + .build(); + } + + @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-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 7aef12f2ff..c7a7a32176 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.integration.utils.ServiceFactory.getVeniceCluster; import static com.linkedin.venice.meta.PersistenceType.ROCKS_DB; import static com.linkedin.venice.utils.IntegrationTestPushUtils.createStoreForJob; @@ -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/PushJobDetailsTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/PushJobDetailsTest.java index b0f068dca5..33a0d183a1 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 @@ -1,16 +1,22 @@ package com.linkedin.venice.endToEnd; import static com.linkedin.davinci.store.rocksdb.RocksDBServerConfig.ROCKSDB_PLAIN_TABLE_FORMAT_ENABLED; +import static com.linkedin.venice.ConfigKeys.PUSH_JOB_FAILURE_CHECKPOINTS_TO_DEFINE_USER_ERROR; 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.DEFAULT_PUSH_JOB_USER_ERROR_CHECKPOINTS; +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.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.vpj.VenicePushJobConstants.DEFAULT_KEY_FIELD_PROP; import static com.linkedin.venice.vpj.VenicePushJobConstants.DEFAULT_VALUE_FIELD_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.INCREMENTAL_PUSH; import static com.linkedin.venice.vpj.VenicePushJobConstants.PUSH_JOB_STATUS_UPLOAD_ENABLE; -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 +25,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; @@ -37,10 +44,13 @@ 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.utils.DataProviderUtils; import com.linkedin.venice.utils.TestUtils; 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,11 +59,8 @@ 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; -import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -65,16 +72,30 @@ public class PushJobDetailsTest { private ControllerClient controllerClient; private ControllerClient parentControllerClient; private Schema recordSchema; - private String inputDirPath; + private String inputDirPathForFullPush; + private String inputDirPathForIncPush; + private String inputDirPathWithDupKeys; + private MetricsRepository metricsRepository; - @BeforeClass - public void setUp() throws IOException { + private void setUp(boolean useCustomCheckpoints) throws IOException { Properties serverProperties = new Properties(); serverProperties.setProperty(ROCKSDB_PLAIN_TABLE_FORMAT_ENABLED, "false"); serverProperties.setProperty(SERVER_DATABASE_CHECKSUM_VERIFICATION_ENABLED, "true"); serverProperties.setProperty(SERVER_DATABASE_SYNC_BYTES_INTERNAL_FOR_DEFERRED_WRITE_MODE, "300"); Properties parentControllerProperties = new Properties(); + if (useCustomCheckpoints) { + StringBuilder customUserErrorCheckpoints = new StringBuilder(); + for (PushJobCheckpoints checkpoint: DEFAULT_PUSH_JOB_USER_ERROR_CHECKPOINTS) { + if (checkpoint != DUP_KEY_WITH_DIFF_VALUE) { + // Skip DUP_KEY_WITH_DIFF_VALUE as it is tested to see that it is not an user error + customUserErrorCheckpoints.append(checkpoint.toString()).append(","); + } + } + parentControllerProperties + .put(PUSH_JOB_FAILURE_CHECKPOINTS_TO_DEFINE_USER_ERROR, customUserErrorCheckpoints.toString()); + } + // Need to add this in controller props when creating venice system for tests parentControllerProperties.setProperty(ConfigKeys.PUSH_JOB_STATUS_STORE_CLUSTER_NAME, "venice-cluster0"); multiRegionMultiClusterWrapper = ServiceFactory.getVeniceTwoLayerMultiRegionMultiClusterWrapper( @@ -93,7 +114,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,37 +124,117 @@ 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")); } + + File inputDirForIncPush = getTempDataDirectory(); + inputDirPathForIncPush = "file://" + inputDirForIncPush.getAbsolutePath(); + TestWriteUtils.writeSimpleAvroFileWithStringToStringSchema2(inputDirForIncPush); + + // input dir with dup keys + File inputDirWithDupKeys = getTempDataDirectory(); + inputDirPathWithDupKeys = "file://" + inputDirWithDupKeys.getAbsolutePath(); + TestWriteUtils.writeSimpleAvroFileWithDuplicateKey(inputDirWithDupKeys); } - @AfterClass - public void cleanUp() { + private void cleanUp() { Utils.closeQuietlyWithErrorLogged(parentControllerClient); 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 verifyMetric( + String metricName, + HashMap metricsExpectedCount, + HashMap metricsExpectedCountSinceLastMeasurement) { + Map metrics = metricsRepository.metrics(); + double metricValueCount = metrics.containsKey(".venice-cluster0--" + metricName + ".Count") + ? metrics.get(".venice-cluster0--" + metricName + ".Count").value() + : 0.0; + double metricValueCountSinceLastMeasurement = + metrics.containsKey(".venice-cluster0--" + metricName + ".CountSinceLastMeasurement") + ? metrics.get(".venice-cluster0--" + metricName + ".CountSinceLastMeasurement").value() + : 0.0; + assertEquals( + metricValueCount, + metricsExpectedCount.getOrDefault(metricName, 0.0), + "Metric " + metricName + ".Count is incorrect"); + assertEquals( + metricValueCountSinceLastMeasurement, + metricsExpectedCountSinceLastMeasurement.getOrDefault(metricName, 0.0), + "Metric " + metricName + ".CountSinceLastMeasurement is incorrect"); + } + + private void validatePushJobMetrics( + boolean isSucceeded, + boolean isUserError, + boolean isIncrementalPush, + HashMap metricsExpectedCount) { + // create a map for expected metrics for CountSinceLastMeasurement type which will be reset after each measurement + HashMap metricsExpectedCountSinceLastMeasurement = new HashMap<>(); + + if (isSucceeded) { + if (isIncrementalPush) { + metricsExpectedCount.compute("incremental_push_job_success", (key, value) -> (value == null) ? 1.0 : value + 1); + metricsExpectedCountSinceLastMeasurement.computeIfAbsent("incremental_push_job_success", k -> 1.0); + } else { + metricsExpectedCount.compute("batch_push_job_success", (key, value) -> (value == null) ? 1.0 : value + 1); + metricsExpectedCountSinceLastMeasurement.computeIfAbsent("batch_push_job_success", k -> 1.0); + } + } else { + if (isUserError) { + if (isIncrementalPush) { + metricsExpectedCount + .compute("incremental_push_job_failed_user_error", (key, value) -> (value == null) ? 1.0 : value + 1); + metricsExpectedCountSinceLastMeasurement.computeIfAbsent("incremental_push_job_failed_user_error", k -> 1.0); + } else { + metricsExpectedCount + .compute("batch_push_job_failed_user_error", (key, value) -> (value == null) ? 1.0 : value + 1); + metricsExpectedCountSinceLastMeasurement.computeIfAbsent("batch_push_job_failed_user_error", k -> 1.0); + } + } else { + if (isIncrementalPush) { + metricsExpectedCount + .compute("incremental_push_job_failed_non_user_error", (key, value) -> (value == null) ? 1.0 : value + 1); + metricsExpectedCountSinceLastMeasurement + .computeIfAbsent("incremental_push_job_failed_non_user_error", k -> 1.0); + } else { + metricsExpectedCount + .compute("batch_push_job_failed_non_user_error", (key, value) -> (value == null) ? 1.0 : value + 1); + metricsExpectedCountSinceLastMeasurement.computeIfAbsent("batch_push_job_failed_non_user_error", k -> 1.0); + } + } } + TestUtils.waitForNonDeterministicAssertion(30, TimeUnit.SECONDS, true, () -> { + verifyMetric("batch_push_job_success", metricsExpectedCount, metricsExpectedCountSinceLastMeasurement); + verifyMetric("incremental_push_job_success", metricsExpectedCount, metricsExpectedCountSinceLastMeasurement); + verifyMetric("batch_push_job_failed_user_error", metricsExpectedCount, metricsExpectedCountSinceLastMeasurement); + verifyMetric( + "batch_push_job_failed_non_user_error", + metricsExpectedCount, + metricsExpectedCountSinceLastMeasurement); + verifyMetric( + "incremental_push_job_failed_user_error", + metricsExpectedCount, + metricsExpectedCountSinceLastMeasurement); + verifyMetric( + "incremental_push_job_failed_non_user_error", + metricsExpectedCount, + metricsExpectedCountSinceLastMeasurement); + }); + } + + 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 +243,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 +252,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 +262,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 +319,139 @@ 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"; - 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); - pushJobProps.setProperty(PUSH_JOB_STATUS_UPLOAD_ENABLE, String.valueOf(true)); - try (VenicePushJob testPushJob = new VenicePushJob("test-push-job-details-job", 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()); - } - } + @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class, timeOut = 180 + * Time.MS_PER_SECOND) + public void testPushJobDetails(boolean useCustomCheckpoints) throws IOException { + try { + setUp(useCustomCheckpoints); + // create a map for expected metrics for Count type which will be incremented through the test + HashMap metricsExpectedCount = new HashMap<>(); - /** - * 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( - 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); - parentControllerClient.updateStore(testStoreName, queryParams); - - Properties pushJobProps = defaultVPJProps(multiRegionMultiClusterWrapper, inputDirPath, 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 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"); + // case 1: successful batch push job + String testStoreName = "test-push-store" + useCustomCheckpoints; + 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).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, metricsExpectedCount); + + // 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, metricsExpectedCount); + + // 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); + } + + 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, metricsExpectedCount); + + // 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); + } + + validatePushJobDetailsStatus( + true, + 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, true, metricsExpectedCount); + + // case 5: failed batch push job, user error: data with duplicate keys + UpdateStoreQueryParams queryParams = new UpdateStoreQueryParams().setStorageQuotaInByte(-1); + parentControllerClient.updateStore(testStoreName, queryParams); + + pushJobProps = defaultVPJProps(multiRegionMultiClusterWrapper, inputDirPathWithDupKeys, testStoreName); + pushJobProps.setProperty(PUSH_JOB_STATUS_UPLOAD_ENABLE, String.valueOf(true)); + try (final VenicePushJob testPushJob = new VenicePushJob("test-push-job-details-job-v3", pushJobProps)) { + assertThrows(VeniceException.class, testPushJob::run); // Push job should fail + } + + 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, !useCustomCheckpoints, false, metricsExpectedCount); + + // 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, !useCustomCheckpoints, true, metricsExpectedCount); + } finally { + cleanUp(); } } } 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 2dcfd00c20..cdd919d02c 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 index 04602ca5f1..05a0861ac6 100644 --- 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 @@ -8,9 +8,11 @@ import static org.testng.Assert.fail; import com.linkedin.venice.exceptions.VeniceException; +import java.lang.reflect.Field; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.lang.reflect.Modifier; +import java.util.List; import java.util.Map; import java.util.function.Function; import org.testng.annotations.Test; @@ -39,14 +41,23 @@ protected VeniceEnumValueTest(Class enumClass) { @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!"); + // check if there is a TYPES field which is either a List or a Map + Field typesField = getPrivateStaticField(this.enumClass, "TYPES"); + boolean isStrictCheck = true; + if (List.class.isAssignableFrom(typesField.getType())) { + isStrictCheck = true; + } else if (Map.class.isAssignableFrom(typesField.getType())) { + isStrictCheck = false; + } else { + fail("The TYPES field should be a List or a Map!"); + } + Function valueOfFunction = value -> { try { return (T) valueOfMethod.invoke(null, value); @@ -66,19 +77,20 @@ public void test() { assertFalse(expectedMapping.isEmpty()); // Check that all mappings are as expected + int highestValue = INVALID_NEGATIVE_VALUE; 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)); + if (isStrictCheck) { + // 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]); @@ -111,4 +123,22 @@ private static Method getPublicStaticFunction(Class klass, String functionName, return null; } } + + private static Field getPrivateStaticField(Class klass, String fieldName) { + try { + Field field = klass.getDeclaredField(fieldName); + assertTrue( + Modifier.isStatic(field.getModifiers()), + "Class " + klass.getSimpleName() + " should have a static " + fieldName + " field!"); + assertTrue( + Modifier.isPrivate(field.getModifiers()), + "Class " + klass.getSimpleName() + " should have a private " + fieldName + " field!"); + return field; + } catch (NoSuchFieldException e) { + fail("Class " + klass.getSimpleName() + " should have a " + fieldName + " field!", 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/VeniceControllerClusterConfig.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java index 1e0ed716ff..d311bcf3f1 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java @@ -136,6 +136,7 @@ import static com.linkedin.venice.ConfigKeys.PERSISTENCE_TYPE; import static com.linkedin.venice.ConfigKeys.PUBSUB_TOPIC_MANAGER_METADATA_FETCHER_CONSUMER_POOL_SIZE; import static com.linkedin.venice.ConfigKeys.PUBSUB_TOPIC_MANAGER_METADATA_FETCHER_THREAD_POOL_SIZE; +import static com.linkedin.venice.ConfigKeys.PUSH_JOB_FAILURE_CHECKPOINTS_TO_DEFINE_USER_ERROR; import static com.linkedin.venice.ConfigKeys.PUSH_JOB_STATUS_STORE_CLUSTER_NAME; import static com.linkedin.venice.ConfigKeys.PUSH_MONITOR_TYPE; import static com.linkedin.venice.ConfigKeys.PUSH_SSL_ALLOWLIST; @@ -158,6 +159,7 @@ import static com.linkedin.venice.ConfigKeys.USE_PUSH_STATUS_STORE_FOR_INCREMENTAL_PUSH; import static com.linkedin.venice.ConfigKeys.VENICE_STORAGE_CLUSTER_LEADER_HAAS; import static com.linkedin.venice.ConfigKeys.ZOOKEEPER_ADDRESS; +import static com.linkedin.venice.PushJobCheckpoints.DEFAULT_PUSH_JOB_USER_ERROR_CHECKPOINTS; import static com.linkedin.venice.SSLConfig.DEFAULT_CONTROLLER_SSL_ENABLED; import static com.linkedin.venice.VeniceConstants.DEFAULT_PER_ROUTER_READ_QUOTA; import static com.linkedin.venice.VeniceConstants.DEFAULT_SSL_FACTORY_CLASS_NAME; @@ -167,6 +169,7 @@ import static com.linkedin.venice.utils.ByteUtils.BYTES_PER_MB; import static com.linkedin.venice.utils.ByteUtils.generateHumanReadableByteCountString; +import com.linkedin.venice.PushJobCheckpoints; import com.linkedin.venice.SSLConfig; import com.linkedin.venice.authorization.DefaultIdentityParser; import com.linkedin.venice.client.store.ClientConfig; @@ -510,6 +513,8 @@ public class VeniceControllerClusterConfig { private final Set childDatacenters; private final long serviceDiscoveryRegistrationRetryMS; + private Set pushJobUserErrorCheckpoints; + public VeniceControllerClusterConfig(VeniceProperties props) { this.props = props; this.clusterName = props.getString(CLUSTER_NAME); @@ -908,6 +913,7 @@ public VeniceControllerClusterConfig(VeniceProperties props) { props.getInt(CONTROLLER_DANGLING_TOPIC_OCCURRENCE_THRESHOLD_FOR_CLEANUP, 3); this.serviceDiscoveryRegistrationRetryMS = props.getLong(SERVICE_DISCOVERY_REGISTRATION_RETRY_MS, 30L * Time.MS_PER_SECOND); + this.pushJobUserErrorCheckpoints = parsePushJobUserErrorCheckpoints(props); } public VeniceProperties getProps() { @@ -1684,4 +1690,25 @@ public int getDanglingTopicOccurrenceThresholdForCleanup() { interface PutToMap { void apply(Map map, String key, String value, String errorMessage); } + + /** + * Parse the input to get the custom user error checkpoints for push jobs or use the default checkpoints. + */ + static Set parsePushJobUserErrorCheckpoints(VeniceProperties props) { + if (props.containsKey(PUSH_JOB_FAILURE_CHECKPOINTS_TO_DEFINE_USER_ERROR)) { + String pushJobUserErrorCheckpoints = props.getString(PUSH_JOB_FAILURE_CHECKPOINTS_TO_DEFINE_USER_ERROR); + LOGGER.info("Using configured Push job user error checkpoints: {}", pushJobUserErrorCheckpoints); + return Utils.parseCommaSeparatedStringToSet(pushJobUserErrorCheckpoints) + .stream() + .map(checkpointStr -> PushJobCheckpoints.valueOf(checkpointStr)) + .collect(Collectors.toSet()); + } else { + LOGGER.info("Using default Push job user error checkpoints: {}", DEFAULT_PUSH_JOB_USER_ERROR_CHECKPOINTS); + return DEFAULT_PUSH_JOB_USER_ERROR_CHECKPOINTS; + } + } + + public Set getPushJobUserErrorCheckpoints() { + return pushJobUserErrorCheckpoints; + } } 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 51e55e8c45..55f03147a6 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<>(); @@ -428,6 +434,8 @@ public class VeniceHelixAdmin implements Admin, StoreCleaner { private final Lazy emptyPushZSTDDictionary; + private Set pushJobUserErrorCheckpoints; + public VeniceHelixAdmin( VeniceControllerMultiClusterConfig multiClusterConfigs, MetricsRepository metricsRepository, @@ -674,8 +682,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 @@ -704,6 +714,8 @@ public void handleDeletedInstances(Set deletedInstances) { } emptyPushZSTDDictionary = Lazy.of(() -> ByteBuffer.wrap(ZstdWithDictCompressor.buildDictionaryOnSyntheticAvroData())); + + pushJobUserErrorCheckpoints = commonConfig.getPushJobUserErrorCheckpoints(); } private VeniceProperties getPubSubSSLPropertiesFromControllerConfig(String pubSubBootstrapServers) { @@ -1182,6 +1194,64 @@ private Integer fetchSystemStoreSchemaId(String clusterName, String storeName, S return response.getId(); } + static boolean isPushJobFailedDueToUserError( + PushJobDetailsStatus status, + PushJobDetails pushJobDetails, + Set pushJobUserErrorCheckpoints) { + if (PushJobDetailsStatus.isFailed(status)) { + PushJobCheckpoints checkpoint = PushJobCheckpoints.valueOf(pushJobDetails.getPushJobLatestCheckpoint()); + return (checkpoint != null && pushJobUserErrorCheckpoints.contains(checkpoint)); + } + return false; + } + + static void emitPushJobStatusMetrics( + Map pushJobStatusStatsMap, + PushJobDetails pushJobDetails, + Set pushJobUserErrorCheckpoints) { + List overallStatuses = pushJobDetails.getOverallStatus(); + if (overallStatuses.isEmpty()) { + return; + } + try { + PushJobDetailsStatus overallStatus = + PushJobDetailsStatus.valueOf(overallStatuses.get(overallStatuses.size() - 1).getStatus()); + if (PushJobDetailsStatus.isTerminal(overallStatus.getValue())) { + String cluster = pushJobDetails.getClusterName().toString(); + PushJobStatusStats pushJobStatusStats = pushJobStatusStatsMap.get(cluster); + Utf8 incPushKey = new Utf8("incremental.push"); + boolean isIncrementalPush = false; + if (pushJobDetails.getPushJobConfigs().containsKey(incPushKey)) { + isIncrementalPush = Boolean.parseBoolean(pushJobDetails.getPushJobConfigs().get(incPushKey).toString()); + } + if (PushJobDetailsStatus.isFailed(overallStatus)) { + if (isPushJobFailedDueToUserError(overallStatus, pushJobDetails, pushJobUserErrorCheckpoints)) { + 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(); + } + } + } + } catch (Exception e) { + LOGGER.error("Failed to emit push job status metrics with pushJobDetails: {}", pushJobDetails.toString(), e); + } + } + /** * 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). @@ -1190,6 +1260,9 @@ private Integer fetchSystemStoreSchemaId(String clusterName, String storeName, S */ @Override public void sendPushJobDetails(PushJobStatusRecordKey key, PushJobDetails value) { + // Emit push job status metrics + emitPushJobStatusMetrics(pushJobStatusStatsMap, value, pushJobUserErrorCheckpoints); + // 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) 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 31b36030c2..31f55e5987 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 0000000000..3013e4c987 --- /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/main/java/com/linkedin/venice/pushmonitor/PushMonitorUtils.java b/services/venice-controller/src/main/java/com/linkedin/venice/pushmonitor/PushMonitorUtils.java index bd24ba0b5e..d66b4a69cd 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/pushmonitor/PushMonitorUtils.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/pushmonitor/PushMonitorUtils.java @@ -89,7 +89,7 @@ public static ExecutionStatusWithDetails getDaVinciPushStatusAndDetails( Set incompleteInstanceList = new HashSet<>(); ExecutionStatus errorStatus = ExecutionStatus.ERROR; for (Map.Entry entry: instances.entrySet()) { - ExecutionStatus status = ExecutionStatus.fromInt(entry.getValue()); + ExecutionStatus status = ExecutionStatus.valueOf(entry.getValue()); // We will skip completed instances, as they have stopped emitting heartbeats and will not be counted as live // instances. if (status == completeStatus) { @@ -241,7 +241,7 @@ public static ExecutionStatusWithDetails getDaVinciPartitionLevelPushStatusAndDe boolean allInstancesCompleted = true; totalReplicaCount += instances.size(); for (Map.Entry entry: instances.entrySet()) { - ExecutionStatus status = ExecutionStatus.fromInt(entry.getValue()); + ExecutionStatus status = ExecutionStatus.valueOf(entry.getValue()); // We will skip completed replicas, as they have stopped emitting heartbeats and will not be counted as live // replicas. if (status == completeStatus) { 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 0000000000..92b1ae9dbf --- /dev/null +++ b/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestPushJobStatusStats.java @@ -0,0 +1,113 @@ +package com.linkedin.venice.controller; + +import static com.linkedin.venice.PushJobCheckpoints.DEFAULT_PUSH_JOB_USER_ERROR_CHECKPOINTS; +import static com.linkedin.venice.PushJobCheckpoints.DVC_INGESTION_ERROR_OTHER; +import static com.linkedin.venice.controller.VeniceHelixAdmin.emitPushJobStatusMetrics; +import static com.linkedin.venice.controller.VeniceHelixAdmin.isPushJobFailedDueToUserError; +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.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.avro.util.Utf8; +import org.testng.annotations.Test; + + +public class TestPushJobStatusStats { + private static final Set CUSTOM_USER_ERROR_CHECKPOINTS = + new HashSet<>(Collections.singletonList(DVC_INGESTION_ERROR_OTHER)); + + @Test(dataProvider = "Two-True-and-False", dataProviderClass = DataProviderUtils.class) + public void testEmitPushJobStatusMetrics(boolean isIncrementalPush, boolean useUserProvidedUserErrorCheckpoints) { + Set userErrorCheckpoints = + useUserProvidedUserErrorCheckpoints ? CUSTOM_USER_ERROR_CHECKPOINTS : DEFAULT_PUSH_JOB_USER_ERROR_CHECKPOINTS; + PushJobDetails pushJobDetails = mock(PushJobDetails.class); + Map pushJobConfigs = new HashMap<>(); + pushJobConfigs.put(new Utf8("incremental.push"), String.valueOf(isIncrementalPush)); + when(pushJobDetails.getPushJobConfigs()).thenReturn(pushJobConfigs); + + when(pushJobDetails.getClusterName()).thenReturn(new Utf8("cluster1")); + List statusTuples = new ArrayList<>(); + when(pushJobDetails.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(pushJobDetails.getPushJobLatestCheckpoint()).thenReturn(checkpoint.getValue()); + emitPushJobStatusMetrics(pushJobStatusStatsMap, pushJobDetails, userErrorCheckpoints); + boolean isUserError = userErrorCheckpoints.contains(checkpoint); + + if (isUserError) { + if (recordMetrics) { + if (isFailed(status)) { + assertTrue(isPushJobFailedDueToUserError(status, pushJobDetails, userErrorCheckpoints)); + 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(); + } + } + } + } else { + if (recordMetrics) { + assertFalse(isPushJobFailedDueToUserError(status, pushJobDetails, userErrorCheckpoints)); + 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(); + } + } + } + } + } + } + } +} diff --git a/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestVeniceControllerClusterConfig.java b/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestVeniceControllerClusterConfig.java index 1cc92ed9fc..159b88caee 100644 --- a/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestVeniceControllerClusterConfig.java +++ b/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestVeniceControllerClusterConfig.java @@ -20,16 +20,29 @@ import static com.linkedin.venice.ConfigKeys.MULTI_REGION; import static com.linkedin.venice.ConfigKeys.NATIVE_REPLICATION_FABRIC_ALLOWLIST; import static com.linkedin.venice.ConfigKeys.PARTICIPANT_MESSAGE_STORE_ENABLED; +import static com.linkedin.venice.ConfigKeys.PUSH_JOB_FAILURE_CHECKPOINTS_TO_DEFINE_USER_ERROR; import static com.linkedin.venice.ConfigKeys.ZOOKEEPER_ADDRESS; - +import static com.linkedin.venice.PushJobCheckpoints.DVC_INGESTION_ERROR_OTHER; +import static com.linkedin.venice.PushJobCheckpoints.QUOTA_EXCEEDED; +import static com.linkedin.venice.controller.VeniceControllerClusterConfig.parsePushJobUserErrorCheckpoints; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertThrows; + +import com.linkedin.venice.PushJobCheckpoints; import com.linkedin.venice.controllerapi.ControllerRoute; import com.linkedin.venice.exceptions.VeniceException; +import com.linkedin.venice.status.protocol.PushJobDetails; import com.linkedin.venice.utils.DataProviderUtils; import com.linkedin.venice.utils.PropertyBuilder; import com.linkedin.venice.utils.TestUtils; import com.linkedin.venice.utils.Utils; import com.linkedin.venice.utils.VeniceProperties; +import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Properties; @@ -50,13 +63,13 @@ public void canParseClusterMap() { Map map = VeniceControllerClusterConfig.parseClusterMap(builder.build(), REGION_ALLOW_LIST); - Assert.assertEquals(map.size(), 2); + assertEquals(map.size(), 2); Assert.assertTrue(map.containsKey("dc1")); Assert.assertTrue(map.containsKey("dc2")); String[] uris = map.get("dc1").split(DELIMITER); - Assert.assertEquals(uris[0], "http://host:1234"); - Assert.assertEquals(uris[1], "http://host:5678"); + assertEquals(uris[0], "http://host:1234"); + assertEquals(uris[1], "http://host:5678"); } @Test @@ -65,8 +78,8 @@ public void canParseD2ClusterMap() { builder.put("child.cluster.d2.zkHost.dc1", "zkAddress1").put("child.cluster.d2.zkHost.dc2", "zkAddress2"); Map map = VeniceControllerClusterConfig.parseClusterMap(builder.build(), REGION_ALLOW_LIST, true); - Assert.assertEquals(map.get("dc1").split(DELIMITER).length, 1); - Assert.assertEquals(map.get("dc2").split(DELIMITER)[0], "zkAddress2"); + assertEquals(map.get("dc1").split(DELIMITER).length, 1); + assertEquals(map.get("dc2").split(DELIMITER)[0], "zkAddress2"); } @Test @@ -81,7 +94,7 @@ public void canParseBannedPaths() { .parseControllerRoutes(builder.build(), CONTROLLER_DISABLED_ROUTES, Collections.emptyList()); // Make sure it looks right. - Assert.assertEquals(parsedRoutes.size(), 2); + assertEquals(parsedRoutes.size(), 2); Assert.assertTrue(parsedRoutes.contains(ControllerRoute.REQUEST_TOPIC)); Assert.assertTrue(parsedRoutes.contains(ControllerRoute.CLUSTER_DISCOVERY)); } @@ -182,4 +195,34 @@ public void testMultiRegionConfig(boolean explicitMultiRegionConfig) { new VeniceControllerClusterConfig(new VeniceProperties(parentControllerProps)); Assert.assertTrue(parentControllerConfig.isMultiRegion()); } + + @Test + public void testParsePushJobUserErrorCheckpoints() { + PushJobDetails pushJobDetails = mock(PushJobDetails.class); + Map pushJobConfigs = new HashMap<>(); + when(pushJobDetails.getPushJobConfigs()).thenReturn(pushJobConfigs); + when(pushJobDetails.getPushJobLatestCheckpoint()).thenReturn(DVC_INGESTION_ERROR_OTHER.getValue()); + + // valid + Properties properties = new Properties(); + properties.put(PUSH_JOB_FAILURE_CHECKPOINTS_TO_DEFINE_USER_ERROR, "QUOTA_EXCEEDED,DVC_INGESTION_ERROR_OTHER"); + VeniceProperties controllerProps = new VeniceProperties(properties); + Set expectedCustomUserErrorCheckpoints = + new HashSet<>(Arrays.asList(QUOTA_EXCEEDED, DVC_INGESTION_ERROR_OTHER)); + assertEquals(expectedCustomUserErrorCheckpoints, parsePushJobUserErrorCheckpoints(controllerProps)); + + // invalid cases: Should throw IllegalArgumentException + Set invalidCheckpointConfigs = new HashSet<>( + Arrays.asList( + "INVALID_CHECKPOINT", + "[DVC_INGESTION_ERROR_OTHER", + "DVC_INGESTION_ERROR_OTHER, RECORD_TOO_LARGE_FAILED]", + "DVC_INGESTION_ERROR_OTHER, TEST", + "-14")); + for (String invalidCheckpointConfig: invalidCheckpointConfigs) { + properties.put(PUSH_JOB_FAILURE_CHECKPOINTS_TO_DEFINE_USER_ERROR, invalidCheckpointConfig); + VeniceProperties controllerPropsInvalid = new VeniceProperties(properties); + assertThrows(IllegalArgumentException.class, () -> parsePushJobUserErrorCheckpoints(controllerPropsInvalid)); + } + } } From 9b3350204c580d543d6d2d44fe676a6e80097503 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Tue, 24 Sep 2024 14:13:36 -0700 Subject: [PATCH 08/16] [controller] Make setting instance group tags for controller cluster resources configurable (#1194) * [controller] Make setting instance group tags for controller cluster resources configurable --- .../java/com/linkedin/venice/ConfigKeys.java | 2 + .../venice/controller/TestHAASController.java | 22 +++++++++++ .../VeniceControllerClusterConfig.java | 7 ++++ .../venice/controller/ZkHelixAdminClient.java | 7 ++++ .../controller/TestZkHelixAdminClient.java | 39 +++++++++++++++++++ 5 files changed, 77 insertions(+) create mode 100644 services/venice-controller/src/test/java/com/linkedin/venice/controller/TestZkHelixAdminClient.java 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 5560a6304e..e25fc8fa0e 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 @@ -214,6 +214,8 @@ private ConfigKeys() { public static final String CONTROLLER_CLUSTER_ZK_ADDRESSS = "controller.cluster.zk.address"; // Name of the Helix cluster for controllers public static final String CONTROLLER_CLUSTER = "controller.cluster.name"; + // What instance group tag to assign to a cluster resource + public static final String CONTROLLER_RESOURCE_INSTANCE_GROUP_TAG = "controller.resource.instance.group.tag"; // What tags to assign to a controller instance public static final String CONTROLLER_INSTANCE_TAG_LIST = "controller.instance.tag.list"; diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestHAASController.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestHAASController.java index f26b6bb515..796edb2aa2 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestHAASController.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestHAASController.java @@ -35,6 +35,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import org.apache.helix.HelixAdmin; import org.apache.helix.manager.zk.ZKHelixManager; import org.apache.helix.model.LiveInstance; import org.testng.annotations.BeforeClass; @@ -56,6 +57,27 @@ public void setUp() { .put(ConfigKeys.VENICE_STORAGE_CLUSTER_LEADER_HAAS, String.valueOf(true)); } + @Test(timeOut = 60 * Time.MS_PER_SECOND) + public void testClusterResourceInstanceTag() { + try (VeniceClusterWrapper venice = ServiceFactory.getVeniceCluster(0, 0, 0, 1); + HelixAsAServiceWrapper helixAsAServiceWrapper = startAndWaitForHAASToBeAvailable(venice.getZk().getAddress())) { + String instanceTag = "GENERAL"; + String controllerClusterName = "venice-controllers"; + + Properties clusterProperties = (Properties) enableControllerAndStorageClusterHAASProperties.clone(); + clusterProperties.put(ConfigKeys.CONTROLLER_RESOURCE_INSTANCE_GROUP_TAG, instanceTag); + clusterProperties.put(ConfigKeys.CONTROLLER_INSTANCE_TAG_LIST, instanceTag); + + VeniceControllerWrapper controllerWrapper = venice.addVeniceController(clusterProperties); + + HelixAdmin helixAdmin = controllerWrapper.getVeniceHelixAdmin().getHelixAdmin(); + List resources = helixAdmin.getResourcesInClusterWithTag(controllerClusterName, instanceTag); + assertEquals(resources.size(), 1); + List instances = helixAdmin.getInstancesInClusterWithTag(controllerClusterName, instanceTag); + assertEquals(instances.size(), 1); + } + } + @Test(timeOut = 60 * Time.MS_PER_SECOND) public void testStartHAASHelixControllerAsControllerClusterLeader() { try (VeniceClusterWrapper venice = ServiceFactory.getVeniceCluster(0, 0, 0, 1); diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java index d311bcf3f1..50bdfaec33 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java @@ -59,6 +59,7 @@ import static com.linkedin.venice.ConfigKeys.CONTROLLER_PARENT_SYSTEM_STORE_REPAIR_CHECK_INTERVAL_SECONDS; import static com.linkedin.venice.ConfigKeys.CONTROLLER_PARENT_SYSTEM_STORE_REPAIR_RETRY_COUNT; import static com.linkedin.venice.ConfigKeys.CONTROLLER_PARENT_SYSTEM_STORE_REPAIR_SERVICE_ENABLED; +import static com.linkedin.venice.ConfigKeys.CONTROLLER_RESOURCE_INSTANCE_GROUP_TAG; import static com.linkedin.venice.ConfigKeys.CONTROLLER_SCHEMA_VALIDATION_ENABLED; import static com.linkedin.venice.ConfigKeys.CONTROLLER_SSL_ENABLED; import static com.linkedin.venice.ConfigKeys.CONTROLLER_STORE_GRAVEYARD_CLEANUP_DELAY_MINUTES; @@ -227,6 +228,7 @@ public class VeniceControllerClusterConfig { // Name of the Helix cluster for controllers private final String controllerClusterName; private final String controllerClusterZkAddress; + private final String controllerResourceInstanceGroupTag; private final List controllerInstanceTagList; private final boolean multiRegion; private final boolean parent; @@ -642,6 +644,7 @@ public VeniceControllerClusterConfig(VeniceProperties props) { */ this.adminCheckReadMethodForKafka = props.getBoolean(ADMIN_CHECK_READ_METHOD_FOR_KAFKA, true); this.controllerClusterName = props.getString(CONTROLLER_CLUSTER, "venice-controllers"); + this.controllerResourceInstanceGroupTag = props.getString(CONTROLLER_RESOURCE_INSTANCE_GROUP_TAG, ""); this.controllerInstanceTagList = props.getList(CONTROLLER_INSTANCE_TAG_LIST, Collections.emptyList()); this.controllerClusterReplica = props.getInt(CONTROLLER_CLUSTER_REPLICA, 3); this.controllerClusterZkAddress = props.getString(CONTROLLER_CLUSTER_ZK_ADDRESSS, getZkAddress()); @@ -1170,6 +1173,10 @@ public String getControllerClusterName() { return controllerClusterName; } + public String getControllerResourceInstanceGroupTag() { + return controllerResourceInstanceGroupTag; + } + public List getControllerInstanceTagList() { return controllerInstanceTagList; } diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/ZkHelixAdminClient.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/ZkHelixAdminClient.java index cad3dfd036..38afcaecd5 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/ZkHelixAdminClient.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/ZkHelixAdminClient.java @@ -164,10 +164,17 @@ public void addVeniceStorageClusterToControllerCluster(String clusterName) { LeaderStandbySMD.name, IdealState.RebalanceMode.FULL_AUTO.toString(), AutoRebalanceStrategy.class.getName()); + VeniceControllerClusterConfig config = multiClusterConfigs.getControllerConfig(clusterName); IdealState idealState = helixAdmin.getResourceIdealState(controllerClusterName, clusterName); idealState.setMinActiveReplicas(controllerClusterReplicaCount); idealState.setRebalancerClassName(DelayedAutoRebalancer.class.getName()); idealState.setRebalanceStrategy(CrushRebalanceStrategy.class.getName()); + + String instanceGroupTag = config.getControllerResourceInstanceGroupTag(); + if (!instanceGroupTag.isEmpty()) { + idealState.setInstanceGroupTag(instanceGroupTag); + } + helixAdmin.setResourceIdealState(controllerClusterName, clusterName, idealState); helixAdmin.rebalance(controllerClusterName, clusterName, controllerClusterReplicaCount); } catch (Exception e) { diff --git a/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestZkHelixAdminClient.java b/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestZkHelixAdminClient.java new file mode 100644 index 0000000000..361a307c67 --- /dev/null +++ b/services/venice-controller/src/test/java/com/linkedin/venice/controller/TestZkHelixAdminClient.java @@ -0,0 +1,39 @@ +package com.linkedin.venice.controller; + +import static org.mockito.Mockito.*; + +import java.lang.reflect.Field; +import org.apache.helix.HelixAdmin; +import org.apache.helix.model.IdealState; +import org.testng.annotations.Test; + + +public class TestZkHelixAdminClient { + @Test + public void testInstanceGroupTag() throws NoSuchFieldException, IllegalAccessException { + ZkHelixAdminClient zkHelixAdminClient = mock(ZkHelixAdminClient.class); + HelixAdmin mockHelixAdmin = mock(HelixAdmin.class); + VeniceControllerMultiClusterConfig mockMultiClusterConfigs = mock(VeniceControllerMultiClusterConfig.class); + VeniceControllerClusterConfig mockClusterConfig = mock(VeniceControllerClusterConfig.class); + IdealState mockIdealState = mock(IdealState.class); + + when(mockClusterConfig.getControllerResourceInstanceGroupTag()).thenReturn("GENERAL"); + when(mockMultiClusterConfigs.getControllerConfig(anyString())).thenReturn(mockClusterConfig); + when(mockHelixAdmin.getResourceIdealState(any(), any())).thenReturn(mockIdealState); + + doCallRealMethod().when(zkHelixAdminClient).addVeniceStorageClusterToControllerCluster(anyString()); + + Field multiClusterConfigsField = ZkHelixAdminClient.class.getDeclaredField("multiClusterConfigs"); + multiClusterConfigsField.setAccessible(true); + multiClusterConfigsField.set(zkHelixAdminClient, mockMultiClusterConfigs); + + Field helixAdminField = ZkHelixAdminClient.class.getDeclaredField("helixAdmin"); + helixAdminField.setAccessible(true); + helixAdminField.set(zkHelixAdminClient, mockHelixAdmin); + + String clusterName = "test-cluster"; + zkHelixAdminClient.addVeniceStorageClusterToControllerCluster(clusterName); + + verify(mockIdealState).setInstanceGroupTag("GENERAL"); + } +} From 5d92c7d5dc740a3dd14293022404e7ba4ceca4a0 Mon Sep 17 00:00:00 2001 From: Nisarg Thakkar Date: Tue, 24 Sep 2024 14:15:47 -0700 Subject: [PATCH 09/16] [vpj] Use native Spark readers for reading from Avro and VSON files (#1191) Currently, we implement a custom input data source to read Avro and VSON files from HDFS. This input data source isn't optimized for large files (i.e. files larger than HDFS block size) and is not aware of blocks and location of those on HDFS. This PR modifies the input logic to use Spark's native Avro and SequenceFile input formats that should be much more optimal for files larger than the block size on HDFS. Additionally, internally at LinkedIn, we have another input format that reads from a Spark table. This commit adds a lot of the base code that can be used to add arbitrary Spark dataframes as supported input formats. The remaining gap is the schema validation done before the DataWriterComputeJob. --- build.gradle | 1 + clients/venice-push-job/build.gradle | 7 + .../datawriter/AbstractPartitionWriter.java | 45 +- .../datawriter/jobs/DataWriterSparkJob.java | 79 ++ .../spark/utils/RowToAvroConverter.java | 483 +++++++ .../venice/vpj/VenicePushJobConstants.java | 4 + .../spark/utils/RowToAvroConverterTest.java | 1214 +++++++++++++++++ .../venice/endToEnd/PartialUpdateTest.java | 2 + .../linkedin/venice/endToEnd/TestBatch.java | 10 +- .../venice/endToEnd/TestVsonStoreBatch.java | 2 + 10 files changed, 1803 insertions(+), 44 deletions(-) create mode 100644 clients/venice-push-job/src/main/java/com/linkedin/venice/spark/utils/RowToAvroConverter.java create mode 100644 clients/venice-push-job/src/test/java/com/linkedin/venice/spark/utils/RowToAvroConverterTest.java diff --git a/build.gradle b/build.gradle index e6ad01275b..7d1a332888 100644 --- a/build.gradle +++ b/build.gradle @@ -85,6 +85,7 @@ ext.libraries = [ grpcServices: "io.grpc:grpc-services:${grpcVersion}", grpcStub: "io.grpc:grpc-stub:${grpcVersion}", hadoopCommon: "org.apache.hadoop:hadoop-common:${hadoopVersion}", + hadoopHdfs: "org.apache.hadoop:hadoop-hdfs:${hadoopVersion}", httpAsyncClient: 'org.apache.httpcomponents:httpasyncclient:4.1.5', httpClient5: 'org.apache.httpcomponents.client5:httpclient5:5.3', httpCore5: 'org.apache.httpcomponents.core5:httpcore5:5.2.4', diff --git a/clients/venice-push-job/build.gradle b/clients/venice-push-job/build.gradle index efc07f7068..1c8b4f98e6 100644 --- a/clients/venice-push-job/build.gradle +++ b/clients/venice-push-job/build.gradle @@ -27,6 +27,13 @@ dependencies { exclude group: 'javax.servlet' } + implementation (libraries.hadoopHdfs) { + // Exclude transitive dependency + exclude group: 'org.apache.avro' + exclude group: 'javax.servlet' + exclude group: 'com.fasterxml.jackson.core' + } + implementation (libraries.apacheSparkAvro) { // Spark 3.1 depends on Avro 1.8.2 - which uses avro-mapred with the hadoop2 classifier. Starting from Avro 1.9 // onwards, avro-mapred is no longer published with a hadoop2 classifier, but Gradle still looks for one. diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/task/datawriter/AbstractPartitionWriter.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/task/datawriter/AbstractPartitionWriter.java index f8df8c2575..661290ebb0 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/task/datawriter/AbstractPartitionWriter.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/task/datawriter/AbstractPartitionWriter.java @@ -10,9 +10,7 @@ import static com.linkedin.venice.vpj.VenicePushJobConstants.TELEMETRY_MESSAGE_INTERVAL; import static com.linkedin.venice.vpj.VenicePushJobConstants.TOPIC_PROP; import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_SCHEMA_ID_PROP; -import static com.linkedin.venice.vpj.VenicePushJobConstants.VSON_PUSH; -import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; import com.linkedin.venice.ConfigKeys; import com.linkedin.venice.annotation.NotThreadsafe; import com.linkedin.venice.exceptions.RecordTooLargeException; @@ -21,17 +19,12 @@ import com.linkedin.venice.guid.GuidUtils; import com.linkedin.venice.hadoop.InputStorageQuotaTracker; import com.linkedin.venice.hadoop.engine.EngineTaskConfigProvider; -import com.linkedin.venice.hadoop.input.recordreader.AbstractVeniceRecordReader; -import com.linkedin.venice.hadoop.input.recordreader.avro.VeniceAvroRecordReader; -import com.linkedin.venice.hadoop.input.recordreader.vson.VeniceVsonRecordReader; import com.linkedin.venice.hadoop.task.TaskTracker; import com.linkedin.venice.meta.Store; import com.linkedin.venice.partitioner.VenicePartitioner; import com.linkedin.venice.pubsub.api.PubSubProduceResult; import com.linkedin.venice.pubsub.api.PubSubProducerCallback; import com.linkedin.venice.serialization.DefaultSerializer; -import com.linkedin.venice.serializer.FastSerializerDeserializerFactory; -import com.linkedin.venice.serializer.RecordDeserializer; import com.linkedin.venice.utils.ByteUtils; import com.linkedin.venice.utils.PartitionUtils; import com.linkedin.venice.utils.SystemTime; @@ -44,7 +37,6 @@ import com.linkedin.venice.writer.VeniceWriter; import com.linkedin.venice.writer.VeniceWriterFactory; import com.linkedin.venice.writer.VeniceWriterOptions; -import java.io.ByteArrayOutputStream; import java.io.Closeable; import java.io.IOException; import java.nio.ByteBuffer; @@ -56,9 +48,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericDatumWriter; -import org.apache.avro.io.Encoder; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -247,7 +236,7 @@ protected VeniceWriterMessage extract( if (duplicateKeyPrinter == null) { throw new VeniceException("'DuplicateKeyPrinter' is not initialized properly"); } - duplicateKeyPrinter.detectAndHandleDuplicateKeys(keyBytes, valueBytes, values, dataWriterTaskTracker); + duplicateKeyPrinter.detectAndHandleDuplicateKeys(valueBytes, values, dataWriterTaskTracker); return new VeniceWriterMessage( keyBytes, valueBytes, @@ -545,24 +534,13 @@ public static class DuplicateKeyPrinter implements AutoCloseable, Closeable { private final boolean isDupKeyAllowed; - private final Schema keySchema; - private final RecordDeserializer keyDeserializer; - private final GenericDatumWriter avroDatumWriter; private int numOfDupKey = 0; DuplicateKeyPrinter(VeniceProperties props) { this.isDupKeyAllowed = props.getBoolean(ALLOW_DUPLICATE_KEY, false); - - AbstractVeniceRecordReader schemaReader = props.getBoolean(VSON_PUSH, false) - ? new VeniceVsonRecordReader(props) - : VeniceAvroRecordReader.fromProps(props); - this.keySchema = schemaReader.getKeySchema(); - this.keyDeserializer = FastSerializerDeserializerFactory.getFastAvroGenericDeserializer(keySchema, keySchema); - this.avroDatumWriter = new GenericDatumWriter<>(keySchema); } protected void detectAndHandleDuplicateKeys( - byte[] keyBytes, byte[] valueBytes, Iterator values, DataWriterTaskTracker dataWriterTaskTracker) { @@ -579,7 +557,8 @@ protected void detectAndHandleDuplicateKeys( identicalValuesToKeyCount++; if (shouldPrint) { shouldPrint = false; - LOGGER.warn(printDuplicateKey(keyBytes)); + numOfDupKey++; + LOGGER.warn("There are multiple records for the same key"); } } else { // Distinct values map to the same key. E.g. key:[ value_1, value_2 ] @@ -588,7 +567,8 @@ protected void detectAndHandleDuplicateKeys( if (isDupKeyAllowed) { if (shouldPrint) { shouldPrint = false; - LOGGER.warn(printDuplicateKey(keyBytes)); + numOfDupKey++; + LOGGER.warn("There are multiple records for the same key"); } } } @@ -597,21 +577,6 @@ protected void detectAndHandleDuplicateKeys( dataWriterTaskTracker.trackDuplicateKeyWithDistinctValue(distinctValuesToKeyCount); } - private String printDuplicateKey(byte[] keyBytes) { - Object keyRecord = keyDeserializer.deserialize(keyBytes); - try (ByteArrayOutputStream output = new ByteArrayOutputStream()) { - Encoder jsonEncoder = AvroCompatibilityHelper.newJsonEncoder(keySchema, output, false); - avroDatumWriter.write(keyRecord, jsonEncoder); - jsonEncoder.flush(); - output.flush(); - - numOfDupKey++; - return String.format("There are multiple records for key:\n%s", new String(output.toByteArray())); - } catch (IOException exception) { - throw new VeniceException(exception); - } - } - @Override public void close() { // Nothing to do diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/jobs/DataWriterSparkJob.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/jobs/DataWriterSparkJob.java index 2288aa370b..4c46568b49 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/jobs/DataWriterSparkJob.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/datawriter/jobs/DataWriterSparkJob.java @@ -1,23 +1,40 @@ package com.linkedin.venice.spark.datawriter.jobs; +import static com.linkedin.venice.spark.SparkConstants.DEFAULT_SCHEMA; import static com.linkedin.venice.vpj.VenicePushJobConstants.ETL_VALUE_SCHEMA_TRANSFORMATION; import static com.linkedin.venice.vpj.VenicePushJobConstants.FILE_KEY_SCHEMA; import static com.linkedin.venice.vpj.VenicePushJobConstants.FILE_VALUE_SCHEMA; import static com.linkedin.venice.vpj.VenicePushJobConstants.GENERATE_PARTIAL_UPDATE_RECORD_FROM_INPUT; +import static com.linkedin.venice.vpj.VenicePushJobConstants.GLOB_FILTER_PATTERN; import static com.linkedin.venice.vpj.VenicePushJobConstants.INPUT_PATH_PROP; import static com.linkedin.venice.vpj.VenicePushJobConstants.KEY_FIELD_PROP; import static com.linkedin.venice.vpj.VenicePushJobConstants.SCHEMA_STRING_PROP; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SPARK_NATIVE_INPUT_FORMAT_ENABLED; import static com.linkedin.venice.vpj.VenicePushJobConstants.UPDATE_SCHEMA_STRING_PROP; import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_FIELD_PROP; import static com.linkedin.venice.vpj.VenicePushJobConstants.VSON_PUSH; +import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; import com.linkedin.venice.hadoop.PushJobSetting; +import com.linkedin.venice.hadoop.input.recordreader.avro.VeniceAvroRecordReader; +import com.linkedin.venice.hadoop.input.recordreader.vson.VeniceVsonRecordReader; import com.linkedin.venice.spark.input.hdfs.VeniceHdfsSource; +import com.linkedin.venice.spark.utils.RowToAvroConverter; +import com.linkedin.venice.utils.VeniceProperties; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.mapred.AvroWrapper; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.DataFrameReader; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.encoders.RowEncoder; +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema; /** @@ -29,6 +46,19 @@ protected Dataset getUserInputDataFrame() { SparkSession sparkSession = getSparkSession(); PushJobSetting pushJobSetting = getPushJobSetting(); + VeniceProperties jobProps = getJobProperties(); + boolean useNativeInputFormat = jobProps.getBoolean(SPARK_NATIVE_INPUT_FORMAT_ENABLED, false); + + if (!useNativeInputFormat) { + return getDataFrameFromCustomInputFormat(sparkSession, pushJobSetting); + } else if (pushJobSetting.isAvro) { + return getAvroDataFrame(sparkSession, pushJobSetting); + } else { + return getVsonDataFrame(sparkSession, pushJobSetting); + } + } + + private Dataset getDataFrameFromCustomInputFormat(SparkSession sparkSession, PushJobSetting pushJobSetting) { DataFrameReader dataFrameReader = sparkSession.read(); dataFrameReader.format(VeniceHdfsSource.class.getCanonicalName()); setInputConf(sparkSession, dataFrameReader, INPUT_PATH_PROP, new Path(pushJobSetting.inputURI).toString()); @@ -55,4 +85,53 @@ protected Dataset getUserInputDataFrame() { } return dataFrameReader.load(); } + + private Dataset getAvroDataFrame(SparkSession sparkSession, PushJobSetting pushJobSetting) { + Dataset df = + sparkSession.read().format("avro").option("pathGlobFilter", GLOB_FILTER_PATTERN).load(pushJobSetting.inputURI); + + // Transforming the input data format + df = df.map((MapFunction) (record) -> { + Schema updateSchema = null; + if (pushJobSetting.generatePartialUpdateRecordFromInput) { + updateSchema = AvroCompatibilityHelper.parse(pushJobSetting.valueSchemaString); + } + + GenericRecord rowRecord = RowToAvroConverter.convert(record, pushJobSetting.inputDataSchema); + VeniceAvroRecordReader recordReader = new VeniceAvroRecordReader( + pushJobSetting.inputDataSchema, + pushJobSetting.keyField, + pushJobSetting.valueField, + pushJobSetting.etlValueSchemaTransformation, + updateSchema); + + AvroWrapper recordAvroWrapper = new AvroWrapper<>(rowRecord); + final byte[] inputKeyBytes = recordReader.getKeyBytes(recordAvroWrapper, null); + final byte[] inputValueBytes = recordReader.getValueBytes(recordAvroWrapper, null); + + return new GenericRowWithSchema(new Object[] { inputKeyBytes, inputValueBytes }, DEFAULT_SCHEMA); + }, RowEncoder.apply(DEFAULT_SCHEMA)); + + return df; + } + + @Deprecated + private Dataset getVsonDataFrame(SparkSession sparkSession, PushJobSetting pushJobSetting) { + JavaRDD rdd = sparkSession.sparkContext() + .sequenceFile(pushJobSetting.inputURI, BytesWritable.class, BytesWritable.class) + .toJavaRDD() + .map(record -> { + VeniceVsonRecordReader recordReader = new VeniceVsonRecordReader( + pushJobSetting.vsonInputKeySchemaString, + pushJobSetting.vsonInputValueSchemaString, + pushJobSetting.keyField, + pushJobSetting.valueField); + + final byte[] inputKeyBytes = recordReader.getKeyBytes(record._1, record._2); + final byte[] inputValueBytes = recordReader.getValueBytes(record._1, record._2); + + return new GenericRowWithSchema(new Object[] { inputKeyBytes, inputValueBytes }, DEFAULT_SCHEMA); + }); + return sparkSession.createDataFrame(rdd, DEFAULT_SCHEMA); + } } diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/utils/RowToAvroConverter.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/utils/RowToAvroConverter.java new file mode 100644 index 0000000000..8ba86c734a --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/spark/utils/RowToAvroConverter.java @@ -0,0 +1,483 @@ +package com.linkedin.venice.spark.utils; + +import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; +import com.linkedin.venice.utils.ByteUtils; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.sql.Date; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.Period; +import java.time.temporal.ChronoUnit; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import org.apache.avro.Conversions; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericEnumSymbol; +import org.apache.avro.generic.GenericFixed; +import org.apache.avro.generic.GenericRecord; +import org.apache.commons.lang3.Validate; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.types.ArrayType; +import org.apache.spark.sql.types.BinaryType; +import org.apache.spark.sql.types.BooleanType; +import org.apache.spark.sql.types.ByteType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DateType; +import org.apache.spark.sql.types.DayTimeIntervalType; +import org.apache.spark.sql.types.DecimalType; +import org.apache.spark.sql.types.DoubleType; +import org.apache.spark.sql.types.FloatType; +import org.apache.spark.sql.types.IntegerType; +import org.apache.spark.sql.types.LongType; +import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.ShortType; +import org.apache.spark.sql.types.StringType; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.TimestampNTZType; +import org.apache.spark.sql.types.TimestampType; +import org.apache.spark.sql.types.YearMonthIntervalType; +import scala.collection.JavaConverters; + + +/** + * A utility class to convert Spark SQL Row to an Avro GenericRecord with the specified schema. This has been written in + * accordance with the following resources: + * + * + * Spark's implementation is not ideal to be used directly for two reasons: + *
    + *
  • It cannot handle complex unions in the version of Spark that we use (3.3.3). The support was added in 3.4.0.
  • + *
  • It converts directly to Avro binary that we need to deserialize, and that incurs an additional serde cost.
  • + *
+ */ +public final class RowToAvroConverter { + private RowToAvroConverter() { + } + + private static final Conversions.DecimalConversion DECIMAL_CONVERTER = new Conversions.DecimalConversion(); + + public static GenericRecord convert(Row row, Schema schema) { + Validate.notNull(row, "Row must not be null"); + Validate.notNull(schema, "Schema must not be null"); + Validate + .isTrue(schema.getType().equals(Schema.Type.RECORD), "Schema must be of type RECORD. Got: " + schema.getType()); + Validate.isInstanceOf(Row.class, row, "Row must be of type Row. Got: " + row.getClass().getName()); + + return convertToRecord(row, row.schema(), schema); + } + + static GenericRecord convertToRecord(Object o, DataType dataType, Schema schema) { + Validate.isInstanceOf(StructType.class, dataType, "Expected StructType, got: " + dataType.getClass().getName()); + Validate.isInstanceOf(Row.class, o, "Expected Row, got: " + o.getClass().getName()); + GenericRecord aResult = new GenericData.Record(schema); + + Row row = (Row) o; + + StructType sType = row.schema(); + StructField[] sFields = sType.fields(); + List aFields = schema.getFields(); + + Validate.isTrue( + sFields.length == aFields.size(), + "Row and Avro schema must have the same number of fields. Row: " + sFields.length + ", Avro: " + + aFields.size()); + + for (int i = 0; i < sFields.length; i++) { + StructField structField = sFields[i]; + Schema.Field avroField = aFields.get(i); + + // Spark field names are case-insensitive + Validate.isTrue( + structField.name().equalsIgnoreCase(avroField.name()), + "Field names must match. Row: " + structField.name() + ", Avro: " + avroField.name()); + + Object elem = row.get(i); + aResult.put(i, convertInternal(elem, structField.dataType(), avroField.schema())); + } + + return aResult; + } + + static Boolean convertToBoolean(Object o, DataType dataType) { + Validate.isInstanceOf(BooleanType.class, dataType, "Expected BooleanType, got: " + dataType.getClass().getName()); + Validate.isInstanceOf(Boolean.class, o, "Expected Boolean, got: " + o.getClass().getName()); + return ((Boolean) o); + } + + static Integer convertToInt(Object o, DataType dataType, Schema schema) { + // IntegerType + if (dataType instanceof IntegerType) { + Validate.isInstanceOf(Integer.class, o, "Expected Integer, got: " + o.getClass().getName()); + return ((Integer) o); + } + + // Avro logical type "date" is read as DateType in Spark + if (dataType instanceof DateType) { + validateLogicalType(schema, LogicalTypes.date()); + + LocalDate localDate; + + if (o instanceof LocalDate) { + localDate = ((LocalDate) o); + } else if (o instanceof Date) { + localDate = ((Date) o).toLocalDate(); + } else { + throw new IllegalArgumentException( + "Unsupported date type: " + o.getClass().getName() + ". Expected java.time.LocalDate or java.sql.Date"); + } + + // Long to int, but we are sure that it fits + return (int) localDate.toEpochDay(); + } + + if (dataType instanceof ByteType) { + Validate.isInstanceOf(Byte.class, o, "Expected Integer, got: " + o.getClass().getName()); + return ((Byte) o).intValue(); + } + + if (dataType instanceof ShortType) { + Validate.isInstanceOf(Short.class, o, "Expected Integer, got: " + o.getClass().getName()); + return ((Short) o).intValue(); + } + + // Spark default Avro converter converts YearMonthIntervalType to int type + // This is not the type read by Spark's native Avro reader, but added to support YearMonthIntervalType + if (dataType instanceof YearMonthIntervalType) { + Validate.isInstanceOf(Period.class, o, "Expected Period, got: " + o.getClass().getName()); + return ((Period) o).getMonths(); + } + + throw new IllegalArgumentException("Unsupported data type: " + dataType); + } + + static Long convertToLong(Object o, DataType dataType, Schema schema) { + // LongType + if (dataType instanceof LongType) { + Validate.isInstanceOf(Long.class, o, "Expected Long, got: " + o.getClass().getName()); + return ((Long) o); + } + + // Avro logical types "timestamp-millis" and "timestamp-micros" are read as LongType in Spark + if (dataType instanceof TimestampType) { + LogicalType logicalType = + validateLogicalType(schema, false, LogicalTypes.timestampMicros(), LogicalTypes.timestampMillis()); + + Instant instant; + if (o instanceof java.time.Instant) { + instant = ((java.time.Instant) o); + } else if (o instanceof java.sql.Timestamp) { + instant = ((java.sql.Timestamp) o).toInstant(); + } else { + throw new IllegalArgumentException( + "Unsupported timestamp type: " + o.getClass().getName() + + ". Expected java.time.Instant or java.sql.Timestamp"); + } + + if (logicalType == null || logicalType == LogicalTypes.timestampMillis()) { + return ChronoUnit.MILLIS.between(Instant.EPOCH, instant); + } + + return ChronoUnit.MICROS.between(Instant.EPOCH, instant); + } + + // Spark default Avro converter converts TimestampNTZType to int type + // This is not the type read by Spark's native Avro reader, but added to support TimestampNTZType + // Avro logical types "local-timestamp-millis" and "local-timestamp-micros" are read as LongType in Spark + if (dataType instanceof TimestampNTZType) { + LogicalType logicalType = + validateLogicalType(schema, false, LogicalTypes.localTimestampMicros(), LogicalTypes.localTimestampMillis()); + Validate.isInstanceOf(java.time.LocalDateTime.class, o, "Expected LocalDateTime, got: " + o.getClass().getName()); + + LocalDateTime localDateTime = ((java.time.LocalDateTime) o); + LocalDateTime epoch = LocalDateTime.of(1970, 1, 1, 0, 0, 0); + + if (logicalType == null || logicalType == LogicalTypes.localTimestampMillis()) { + return ChronoUnit.MILLIS.between(epoch, localDateTime); + } + + return ChronoUnit.MICROS.between(epoch, localDateTime); + } + + // Spark default Avro converter converts DayTimeIntervalType to long type + // This is not the type read by Spark's native Avro reader, but added to support DayTimeIntervalType + if (dataType instanceof DayTimeIntervalType) { + Validate.isInstanceOf(Duration.class, o, "Expected Duration, got: " + o.getClass().getName()); + Duration duration = (Duration) o; + return TimeUnit.SECONDS.toMicros(duration.getSeconds()) + TimeUnit.NANOSECONDS.toMicros(duration.getNano()); + } + + throw new IllegalArgumentException("Unsupported data type: " + dataType); + } + + static Float convertToFloat(Object o, DataType dataType) { + Validate.isInstanceOf(FloatType.class, dataType, "Expected FloatType, got: " + dataType); + Validate.isInstanceOf(Float.class, o, "Expected Float, got: " + o.getClass().getName()); + return ((Float) o); + } + + static Double convertToDouble(Object o, DataType dataType) { + Validate.isInstanceOf(DoubleType.class, dataType, "Expected DoubleType, got: " + dataType); + Validate.isInstanceOf(Double.class, o, "Expected Double, got: " + o.getClass().getName()); + return ((Double) o); + } + + static CharSequence convertToString(Object o, DataType dataType) { + Validate.isInstanceOf(StringType.class, dataType, "Expected StringType, got: " + dataType); + Validate.isInstanceOf(CharSequence.class, o, "Expected CharSequence, got: " + o.getClass().getName()); + return ((CharSequence) o); + } + + static ByteBuffer convertToBytes(Object o, DataType dataType, Schema schema) { + if (dataType instanceof BinaryType) { + if (o instanceof byte[]) { + return ByteBuffer.wrap((byte[]) o); + } + + if (o instanceof ByteBuffer) { + return (ByteBuffer) o; + } + + throw new IllegalArgumentException("Unsupported byte array type: " + o.getClass().getName()); + } + + if (dataType instanceof DecimalType) { + DecimalType decimalType = (DecimalType) dataType; + validateLogicalType(schema, LogicalTypes.decimal(decimalType.precision(), decimalType.scale())); + Validate.isInstanceOf(BigDecimal.class, o, "Expected BigDecimal, got: " + o.getClass().getName()); + BigDecimal decimal = (BigDecimal) o; + LogicalTypes.Decimal l = (LogicalTypes.Decimal) schema.getLogicalType(); + return DECIMAL_CONVERTER.toBytes(decimal, schema, l); + } + + throw new IllegalArgumentException("Unsupported data type: " + dataType); + } + + static GenericFixed convertToFixed(Object o, DataType dataType, Schema schema) { + if (dataType instanceof BinaryType) { + if (o instanceof byte[]) { + byte[] bytes = (byte[]) o; + Validate.isTrue( + bytes.length == schema.getFixedSize(), + "Fixed size mismatch. Expected: " + schema.getFixedSize() + ", got: " + bytes.length); + return AvroCompatibilityHelper.newFixed(schema, bytes); + } + + if (o instanceof ByteBuffer) { + ByteBuffer bytes = (ByteBuffer) o; + Validate.isTrue( + bytes.remaining() == schema.getFixedSize(), + "Fixed size mismatch. Expected: " + schema.getFixedSize() + ", got: " + bytes.remaining()); + return AvroCompatibilityHelper.newFixed(schema, ByteUtils.extractByteArray(bytes)); + } + + throw new IllegalArgumentException("Unsupported byte array type: " + o.getClass().getName()); + } + + if (dataType instanceof DecimalType) { + DecimalType decimalType = (DecimalType) dataType; + validateLogicalType(schema, LogicalTypes.decimal(decimalType.precision(), decimalType.scale())); + Validate.isInstanceOf(BigDecimal.class, o, "Expected BigDecimal, got: " + o.getClass().getName()); + BigDecimal decimal = (BigDecimal) o; + Conversions.DecimalConversion DECIMAL_CONVERTER = new Conversions.DecimalConversion(); + LogicalTypes.Decimal l = (LogicalTypes.Decimal) schema.getLogicalType(); + return DECIMAL_CONVERTER.toFixed(decimal, schema, l); + } + + throw new IllegalArgumentException("Unsupported data type: " + dataType); + } + + static GenericEnumSymbol convertToEnum(Object o, DataType dataType, Schema schema) { + Validate.isInstanceOf(StringType.class, dataType, "Expected StringType, got: " + dataType); + Validate.isInstanceOf(CharSequence.class, o, "Expected CharSequence, got: " + o.getClass().getName()); + Validate.isTrue(schema.getEnumSymbols().contains(o.toString()), "Enum symbol not found: " + o); + return AvroCompatibilityHelper.newEnumSymbol(schema, ((CharSequence) o).toString()); + } + + static List convertToArray(Object o, DataType dataType, Schema schema) { + Validate.isInstanceOf(ArrayType.class, dataType, "Expected ArrayType, got: " + dataType); + + // Type of elements in the array + Schema elementType = schema.getElementType(); + + List inputList; + if (o instanceof List) { + inputList = (List) o; + } else if (o instanceof scala.collection.Seq) { + // If the input is a scala.collection.Seq, convert it to a List + inputList = JavaConverters.seqAsJavaList((scala.collection.Seq) o); + } else { + throw new IllegalArgumentException("Unsupported array type: " + o.getClass().getName()); + } + + List outputList = new ArrayList<>(inputList.size()); + + for (Object element: inputList) { + outputList.add(convertInternal(element, ((ArrayType) dataType).elementType(), elementType)); + } + + return outputList; + } + + static Map convertToMap(Object o, DataType dataType, Schema schema) { + Validate.isInstanceOf(MapType.class, dataType, "Expected MapType, got: " + dataType.getClass().getName()); + + MapType sType = ((MapType) dataType); + + Map inputMap; + if (o instanceof Map) { + inputMap = (Map) o; + } else if (o instanceof scala.collection.Map) { + inputMap = JavaConverters.mapAsJavaMap((scala.collection.Map) o); + } else { + throw new IllegalArgumentException("Unsupported map type: " + o.getClass().getName()); + } + + Map outputMap = new HashMap<>(inputMap.size()); + + for (Object entryObj: inputMap.entrySet()) { + Validate.isInstanceOf(Map.Entry.class, entryObj, "Expected Map.Entry, got: " + entryObj.getClass().getName()); + Map.Entry entry = (Map.Entry) entryObj; + outputMap.put( + // Key is always a String in Avro + convertToString(entry.getKey(), sType.keyType()), + convertInternal(entry.getValue(), sType.valueType(), schema.getValueType())); + } + + return outputMap; + } + + static Object convertToUnion(Object o, DataType dataType, Schema schema) { + if (o == null) { + Validate.isTrue(schema.isNullable(), "Field is not nullable: " + schema.getName()); + return null; + } + + // Now that we've checked for null explicitly, we should process everything else as a non-null value. + // This is consistent with the way Spark handles unions. + List types = + schema.getTypes().stream().filter(s -> s.getType() != Schema.Type.NULL).collect(Collectors.toList()); + Schema first = types.get(0); + // If there's only one branch, Spark will use that as the data type + if (types.size() == 1) { + return convertInternal(o, dataType, first); + } + + Schema second = types.get(1); + if (types.size() == 2) { + // A union of int and long is read as LongType. + // This is lossy because we cannot know what type was provided in the input + if ((first.getType() == Schema.Type.INT && second.getType() == Schema.Type.LONG) + || (first.getType() == Schema.Type.LONG && second.getType() == Schema.Type.INT)) { + return convertToLong(o, dataType, schema); + } + + // A union of float and double is read as DoubleType. + // This is lossy because we cannot know what type was provided in the input + if ((first.getType() == Schema.Type.FLOAT && second.getType() == Schema.Type.DOUBLE) + || (first.getType() == Schema.Type.DOUBLE && second.getType() == Schema.Type.FLOAT)) { + return convertToDouble(o, dataType); + } + } + + // Now, handle complex unions: member0, member1, ... + // If a branch of the union is "null", then it is skipped in the Catalyst schema. + // So, [ "null", "int", "string" ], [ "int", "null", "string" ], [ "int", "string", "null" ], will all be parsed as + // StructType { member0 -> IntegerType, member1 -> StringType }. + Validate.isInstanceOf(StructType.class, dataType, "Expected StructType, got: " + dataType.getClass().getName()); + Validate.isInstanceOf(Row.class, o, "Expected Row, got: " + o.getClass().getName()); + Row row = (Row) o; + + StructType structType = (StructType) dataType; + StructField[] structFields = structType.fields(); + int structFieldIndex = 0; + for (Schema type: types) { + Validate.isTrue(type.getType() != Schema.Type.NULL); + + Object unionField = row.get(structFieldIndex); + if (unionField != null) { + return convertInternal(unionField, structFields[structFieldIndex].dataType(), type); + } + structFieldIndex++; + } + + throw new IllegalArgumentException("At least one field of complex union must be non-null: " + types); + } + + private static Object convertInternal(Object o, DataType dataType, Schema schema) { + if (o == null) { + Validate.isTrue(schema.isNullable(), "Field is not nullable: " + schema.getName()); + return null; + } + + switch (schema.getType()) { + case BOOLEAN: + return convertToBoolean(o, dataType); + case INT: + return convertToInt(o, dataType, schema); + case LONG: + return convertToLong(o, dataType, schema); + case FLOAT: + return convertToFloat(o, dataType); + case DOUBLE: + return convertToDouble(o, dataType); + case STRING: + return convertToString(o, dataType); + case BYTES: + return convertToBytes(o, dataType, schema); + case FIXED: + return convertToFixed(o, dataType, schema); + case ENUM: + return convertToEnum(o, dataType, schema); + case ARRAY: + return convertToArray(o, dataType, schema); + case MAP: + return convertToMap(o, dataType, schema); + case RECORD: + return convertToRecord(o, dataType, schema); + case UNION: + return convertToUnion(o, dataType, schema); + default: + throw new IllegalArgumentException("Unsupported Avro type: " + schema.getType()); + } + } + + static LogicalType validateLogicalType(Schema schema, LogicalType... expectedTypes) { + return validateLogicalType(schema, true, expectedTypes); + } + + static LogicalType validateLogicalType(Schema schema, boolean needLogicalType, LogicalType... expectedTypes) { + LogicalType logicalType = schema.getLogicalType(); + if (logicalType == null) { + if (needLogicalType) { + throw new IllegalArgumentException("Expected Avro logical type to be present, got schema: " + schema); + } else { + return null; + } + } + + for (LogicalType expectedType: expectedTypes) { + if (logicalType.equals(expectedType)) { + return expectedType; + } + } + + throw new IllegalArgumentException( + "Expected Avro logical type to be one of: " + Arrays.toString(expectedTypes) + ", got: " + logicalType); + } +} diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/vpj/VenicePushJobConstants.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/vpj/VenicePushJobConstants.java index c6f62f4ee2..3e172755f5 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/vpj/VenicePushJobConstants.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/vpj/VenicePushJobConstants.java @@ -33,6 +33,9 @@ private VenicePushJobConstants() { public static final boolean DEFAULT_EXTENDED_SCHEMA_VALIDITY_CHECK_ENABLED = true; public static final String UPDATE_SCHEMA_STRING_PROP = "update.schema"; + // This is a temporary config used to rollout the native input format for Spark. This will be removed soon + public static final String SPARK_NATIVE_INPUT_FORMAT_ENABLED = "spark.native.input.format.enabled"; + // Vson input configs // Vson files store key/value schema on file header. key / value fields are optional // and should be specified only when key / value schema is the partial of the files. @@ -224,6 +227,7 @@ private VenicePushJobConstants() { * ignore hdfs files with prefix "_" and "." */ public static final PathFilter PATH_FILTER = p -> !p.getName().startsWith("_") && !p.getName().startsWith("."); + public static final String GLOB_FILTER_PATTERN = "[^_.]*"; // Configs to control temp paths and their permissions public static final String HADOOP_TMP_DIR = "hadoop.tmp.dir"; diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/utils/RowToAvroConverterTest.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/utils/RowToAvroConverterTest.java new file mode 100644 index 0000000000..da28c8982d --- /dev/null +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/spark/utils/RowToAvroConverterTest.java @@ -0,0 +1,1214 @@ +package com.linkedin.venice.spark.utils; + +import static org.apache.spark.sql.types.DataTypes.BinaryType; +import static org.apache.spark.sql.types.DataTypes.BooleanType; +import static org.apache.spark.sql.types.DataTypes.ByteType; +import static org.apache.spark.sql.types.DataTypes.DateType; +import static org.apache.spark.sql.types.DataTypes.DoubleType; +import static org.apache.spark.sql.types.DataTypes.FloatType; +import static org.apache.spark.sql.types.DataTypes.IntegerType; +import static org.apache.spark.sql.types.DataTypes.LongType; +import static org.apache.spark.sql.types.DataTypes.ShortType; +import static org.apache.spark.sql.types.DataTypes.StringType; +import static org.apache.spark.sql.types.DataTypes.TimestampType; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertThrows; +import static org.testng.Assert.assertTrue; + +import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; +import com.linkedin.venice.utils.Time; +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.nio.ByteBuffer; +import java.sql.Date; +import java.sql.Timestamp; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.Period; +import java.time.ZoneOffset; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericEnumSymbol; +import org.apache.avro.generic.GenericFixed; +import org.apache.avro.generic.GenericRecord; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.TimestampNTZType$; +import org.testng.annotations.Test; +import scala.collection.JavaConverters; + + +public class RowToAvroConverterTest { + private static final StructType COMPLEX_SUB_SCHEMA = DataTypes.createStructType( + new StructField[] { new StructField("int", IntegerType, false, Metadata.empty()), + new StructField("string", StringType, false, Metadata.empty()) }); + + private static final StructType UNION_STRUCT_STRING_INT = DataTypes.createStructType( + new StructField[] { new StructField("member0", StringType, true, Metadata.empty()), + new StructField("member1", IntegerType, true, Metadata.empty()) }); + + private static final StructType UNION_STRUCT_DOUBLE_FLOAT_STRING = DataTypes.createStructType( + new StructField[] { new StructField("member0", DoubleType, true, Metadata.empty()), + new StructField("member1", FloatType, true, Metadata.empty()), + new StructField("member2", StringType, true, Metadata.empty()) }); + + private static final StructType SPARK_STRUCT_SCHEMA = new StructType( + new StructField[] { new StructField("byteArr", BinaryType, false, Metadata.empty()), + new StructField("byteBuffer", BinaryType, false, Metadata.empty()), + new StructField("decimalBytes", DataTypes.createDecimalType(3, 2), false, Metadata.empty()), + new StructField("booleanTrue", BooleanType, false, Metadata.empty()), + new StructField("booleanFalse", BooleanType, false, Metadata.empty()), + new StructField("float", FloatType, false, Metadata.empty()), + new StructField("double", DoubleType, false, Metadata.empty()), + new StructField("string", StringType, false, Metadata.empty()), + new StructField("byteArrFixed", BinaryType, false, Metadata.empty()), + new StructField("byteBufferFixed", BinaryType, false, Metadata.empty()), + new StructField("decimalFixed", DataTypes.createDecimalType(3, 2), false, Metadata.empty()), + new StructField("enumType", StringType, false, Metadata.empty()), + new StructField("int", IntegerType, false, Metadata.empty()), + new StructField("date", DateType, false, Metadata.empty()), + new StructField("dateLocal", DateType, false, Metadata.empty()), + new StructField("byte", ByteType, false, Metadata.empty()), + new StructField("short", ShortType, false, Metadata.empty()), + new StructField("yearMonthInterval", DataTypes.createYearMonthIntervalType(), false, Metadata.empty()), + new StructField("long", LongType, false, Metadata.empty()), + new StructField("instantMicros", TimestampType, false, Metadata.empty()), + new StructField("instantMillis", TimestampType, false, Metadata.empty()), + new StructField("timestampMicros", TimestampType, false, Metadata.empty()), + new StructField("timestampMillis", TimestampType, false, Metadata.empty()), + new StructField("timestampNoLogical", TimestampType, false, Metadata.empty()), + new StructField("localTimestampMicros", TimestampNTZType$.MODULE$, false, Metadata.empty()), + new StructField("localTimestampMillis", TimestampNTZType$.MODULE$, false, Metadata.empty()), + new StructField("localTimestampNoLogical", TimestampNTZType$.MODULE$, false, Metadata.empty()), + new StructField("dayTimeInterval", DataTypes.createDayTimeIntervalType(), false, Metadata.empty()), + new StructField("arrayIntList", DataTypes.createArrayType(IntegerType), false, Metadata.empty()), + new StructField("arrayIntSeq", DataTypes.createArrayType(IntegerType), false, Metadata.empty()), + new StructField("arrayComplex", DataTypes.createArrayType(COMPLEX_SUB_SCHEMA), false, Metadata.empty()), + new StructField("mapIntJavaMap", DataTypes.createMapType(StringType, IntegerType), false, Metadata.empty()), + new StructField("mapIntScalaMap", DataTypes.createMapType(StringType, IntegerType), false, Metadata.empty()), + new StructField( + "mapComplex", + DataTypes.createMapType(StringType, COMPLEX_SUB_SCHEMA), + false, + Metadata.empty()), + new StructField("nullableUnion", IntegerType, true, Metadata.empty()), + new StructField("nullableUnion2", IntegerType, true, Metadata.empty()), + new StructField("singleElementUnion", IntegerType, false, Metadata.empty()), + new StructField("intLongUnion", LongType, false, Metadata.empty()), + new StructField("longIntUnion", LongType, false, Metadata.empty()), + new StructField("floatDoubleUnion", DoubleType, false, Metadata.empty()), + new StructField("doubleFloatUnion", DoubleType, false, Metadata.empty()), + new StructField("complexNonNullableUnion", UNION_STRUCT_DOUBLE_FLOAT_STRING, false, Metadata.empty()), + new StructField("complexNullableUnion1", UNION_STRUCT_STRING_INT, true, Metadata.empty()), + new StructField("complexNullableUnion2", UNION_STRUCT_STRING_INT, true, Metadata.empty()), + new StructField("complexNullableUnion3", UNION_STRUCT_STRING_INT, true, Metadata.empty()), }); + + private static final Schema DECIMAL_TYPE = LogicalTypes.decimal(3, 2).addToSchema(Schema.create(Schema.Type.BYTES)); + private static final Schema DECIMAL_FIXED_TYPE = + LogicalTypes.decimal(3, 2).addToSchema(Schema.createFixed("decimalFixed", null, null, 3)); + private static final Schema FIXED_TYPE_3 = Schema.createFixed("decimalFixed", null, null, 3); + private static final String STRING_VALUE = "PAX TIBI MARCE EVANGELISTA MEVS"; + private static final String STRING_VALUE_2 = + "It’s temples and palaces did seem like fabrics of enchantment piled to heaven"; + private static final String STRING_VALUE_3 = "Like eating an entire box of chocolate liqueurs in one go"; + private static final Schema DATE_TYPE = LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT)); + private static final Schema TIMESTAMP_MICROS = + LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + private static final Schema TIMESTAMP_MILLIS = + LogicalTypes.timestampMillis().addToSchema(Schema.create(Schema.Type.LONG)); + private static final long TEST_EPOCH_MILLIS = 1718860000000L; + private static final Instant TEST_EPOCH_INSTANT = Instant.ofEpochMilli(TEST_EPOCH_MILLIS); + private static final Schema LOCAL_TIMESTAMP_MICROS = + LogicalTypes.localTimestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + private static final Schema LOCAL_TIMESTAMP_MILLIS = + LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Schema.Type.LONG)); + private static final LocalDateTime TEST_LOCAL_DATE_TIME = + LocalDateTime.ofEpochSecond(TEST_EPOCH_MILLIS / 1000, 0, ZoneOffset.of("+02:00")); + // 2 hour offset to account for the local timezone + private static final long TEST_LOCAL_TIMESTAMP_MILLIS = TEST_EPOCH_MILLIS + 2 * Time.MS_PER_HOUR; + + private static final Schema COMPLEX_SUB_SCHEMA_AVRO = + SchemaBuilder.record("arrayComplex").fields().requiredInt("int").requiredString("string").endRecord(); + + private static final Schema AVRO_SCHEMA = SchemaBuilder.record("test") + .fields() + .name("byteArr") + .type() + .bytesType() + .noDefault() + .name("byteBuffer") + .type() + .bytesType() + .noDefault() + .name("decimalBytes") + .type(DECIMAL_TYPE) + .noDefault() + .name("booleanTrue") + .type() + .booleanType() + .noDefault() + .name("booleanFalse") + .type() + .booleanType() + .noDefault() + .name("float") + .type() + .floatType() + .noDefault() + .name("double") + .type() + .doubleType() + .noDefault() + .name("string") + .type() + .stringType() + .noDefault() + .name("byteArrFixed") + .type(FIXED_TYPE_3) + .noDefault() + .name("byteBufferFixed") + .type(FIXED_TYPE_3) + .noDefault() + .name("decimalFixed") + .type(DECIMAL_FIXED_TYPE) + .noDefault() + .name("enumType") + .type() + .enumeration("enumType") + .symbols("A", "B", "C") + .noDefault() + .name("int") + .type() + .intType() + .noDefault() + .name("date") + .type(DATE_TYPE) + .noDefault() + .name("dateLocal") + .type(DATE_TYPE) + .noDefault() + .name("byte") + .type() + .intType() + .noDefault() + .name("short") + .type() + .intType() + .noDefault() + .name("yearMonthInterval") + .type() + .intType() + .noDefault() + .name("long") + .type() + .longType() + .noDefault() + .name("instantMicros") + .type(TIMESTAMP_MICROS) + .noDefault() + .name("instantMillis") + .type(TIMESTAMP_MILLIS) + .noDefault() + .name("timestampMicros") + .type(TIMESTAMP_MICROS) + .noDefault() + .name("timestampMillis") + .type(TIMESTAMP_MILLIS) + .noDefault() + .name("timestampNoLogical") + .type() + .longType() + .noDefault() + .name("localTimestampMicros") + .type(LOCAL_TIMESTAMP_MICROS) + .noDefault() + .name("localTimestampMillis") + .type(LOCAL_TIMESTAMP_MILLIS) + .noDefault() + .name("localTimestampNoLogical") + .type() + .longType() + .noDefault() + .name("dayTimeInterval") + .type() + .longType() + .noDefault() + .name("arrayIntList") + .type() + .array() + .items() + .intType() + .noDefault() + .name("arrayIntSeq") + .type() + .array() + .items() + .intType() + .noDefault() + .name("arrayComplex") + .type() + .array() + .items(COMPLEX_SUB_SCHEMA_AVRO) + .noDefault() + .name("mapIntJavaMap") + .type() + .map() + .values() + .intType() + .noDefault() + .name("mapIntScalaMap") + .type() + .map() + .values() + .intType() + .noDefault() + .name("mapComplex") + .type() + .map() + .values(COMPLEX_SUB_SCHEMA_AVRO) + .noDefault() + .name("nullableUnion") + .type() + .unionOf() + .nullType() + .and() + .intType() + .endUnion() + .noDefault() + .name("nullableUnion2") + .type() + .unionOf() + .intType() + .and() + .nullType() + .endUnion() + .noDefault() + .name("singleElementUnion") + .type() + .unionOf() + .intType() + .endUnion() + .noDefault() + .name("intLongUnion") + .type() + .unionOf() + .intType() + .and() + .longType() + .endUnion() + .noDefault() + .name("longIntUnion") + .type() + .unionOf() + .longType() + .and() + .intType() + .endUnion() + .noDefault() + .name("floatDoubleUnion") + .type() + .unionOf() + .floatType() + .and() + .doubleType() + .endUnion() + .noDefault() + .name("doubleFloatUnion") + .type() + .unionOf() + .doubleType() + .and() + .floatType() + .endUnion() + .noDefault() + .name("complexNonNullableUnion") + .type() + .unionOf() + .doubleType() + .and() + .floatType() + .and() + .stringType() + .endUnion() + .noDefault() + .name("complexNullableUnion1") + .type() + .unionOf() + .nullType() + .and() + .stringType() + .and() + .intType() + .endUnion() + .noDefault() + .name("complexNullableUnion2") + .type() + .unionOf() + .stringType() + .and() + .nullType() + .and() + .intType() + .endUnion() + .noDefault() + .name("complexNullableUnion3") + .type() + .unionOf() + .stringType() + .and() + .intType() + .and() + .nullType() + .endUnion() + .noDefault() + .endRecord(); + + private static final Row SPARK_ROW = new GenericRowWithSchema( + new Object[] { new byte[] { 0x01, 0x02, 0x03 }, // byteArr + ByteBuffer.wrap(new byte[] { 0x04, 0x05, 0x06 }), // byteBuffer + new BigDecimal("0.456").setScale(2, RoundingMode.HALF_UP), // decimalBytes + true, // booleanTrue + false, // booleanFalse + 0.5f, // float + 0.7, // double + STRING_VALUE, // string + new byte[] { 0x01, 0x02, 0x03 }, // byteArrFixed + ByteBuffer.wrap(new byte[] { 0x04, 0x05, 0x06 }), // byteBufferFixed + new BigDecimal("0.456").setScale(2, RoundingMode.HALF_UP), // decimalFixed + "A", // enumType + 100, // int + Date.valueOf(LocalDate.of(2024, 6, 18)), // date + LocalDate.of(2024, 6, 18), // dateLocal + (byte) 100, // byte + (short) 100, // short + Period.ofMonths(5), // yearMonthInterval + 100L, // long + TEST_EPOCH_INSTANT, // instantMicros + TEST_EPOCH_INSTANT, // instantMillis + Timestamp.from(TEST_EPOCH_INSTANT), // timestampMicros + Timestamp.from(TEST_EPOCH_INSTANT), // timestampMillis + TEST_EPOCH_INSTANT, // timestampNoLogical + TEST_LOCAL_DATE_TIME, // localTimestampMicros + TEST_LOCAL_DATE_TIME, // localTimestampMillis + TEST_LOCAL_DATE_TIME, // localTimestampNoLogical + Duration.ofSeconds(100), // dayTimeInterval + Arrays.asList(1, 2, 3), // arrayIntList + JavaConverters.asScalaBuffer(Arrays.asList(1, 2, 3)).toList(), // arrayIntSeq + JavaConverters.asScalaBuffer( + Arrays.asList( + new GenericRowWithSchema(new Object[] { 10, STRING_VALUE_2 }, COMPLEX_SUB_SCHEMA), + new GenericRowWithSchema(new Object[] { 20, STRING_VALUE_3 }, COMPLEX_SUB_SCHEMA))) + .toList(), // arrayComplex + new HashMap() { + { + put("key1", 10); + put("key2", 20); + } + }, // mapIntJavaMap + JavaConverters.mapAsScalaMap(new HashMap() { + { + put("key1", 10); + put("key2", 20); + } + }), // mapIntScalaMap + new HashMap() { + { + put("key1", new GenericRowWithSchema(new Object[] { 10, STRING_VALUE_2 }, COMPLEX_SUB_SCHEMA)); + put("key2", new GenericRowWithSchema(new Object[] { 20, STRING_VALUE_3 }, COMPLEX_SUB_SCHEMA)); + } + }, // mapComplex + 10, // nullableUnion + null, // nullableUnion2 + 10, // singleElementUnion + 10L, // intLongUnion + 10L, // longIntUnion + 0.5, // floatDoubleUnion + 0.5, // doubleFloatUnion + new GenericRowWithSchema(new Object[] { null, 0.5f, null }, UNION_STRUCT_DOUBLE_FLOAT_STRING), // complexNonNullableUnion + new GenericRowWithSchema(new Object[] { null, 10 }, UNION_STRUCT_STRING_INT), // complexNullableUnion1 + new GenericRowWithSchema(new Object[] { STRING_VALUE, null }, UNION_STRUCT_STRING_INT), // complexNullableUnion2 + null, // complexNullableUnion3 + }, + SPARK_STRUCT_SCHEMA); + + @Test + public void testConvertToRecord() { + GenericRecord record = RowToAvroConverter.convertToRecord(SPARK_ROW, SPARK_STRUCT_SCHEMA, AVRO_SCHEMA); + assertEquals(record.get("byteArr"), ByteBuffer.wrap(new byte[] { 0x01, 0x02, 0x03 })); + assertEquals(record.get("byteBuffer"), ByteBuffer.wrap(new byte[] { 0x04, 0x05, 0x06 })); + assertEquals(record.get("decimalBytes"), ByteBuffer.wrap(new byte[] { 46 })); + assertEquals(record.get("booleanTrue"), true); + assertEquals(record.get("booleanFalse"), false); + assertEquals(record.get("float"), 0.5f); + assertEquals(record.get("double"), 0.7); + assertEquals(record.get("string"), STRING_VALUE); + assertEquals( + record.get("byteArrFixed"), + AvroCompatibilityHelper.newFixed(FIXED_TYPE_3, new byte[] { 0x01, 0x02, 0x03 })); + assertEquals( + record.get("byteBufferFixed"), + AvroCompatibilityHelper.newFixed(FIXED_TYPE_3, new byte[] { 0x04, 0x05, 0x06 })); + assertEquals(record.get("decimalFixed"), AvroCompatibilityHelper.newFixed(FIXED_TYPE_3, new byte[] { 0, 0, 46 })); + assertEquals( + record.get("enumType"), + AvroCompatibilityHelper.newEnumSymbol(AVRO_SCHEMA.getField("enumType").schema(), "A")); + assertEquals(record.get("int"), 100); + assertEquals(record.get("date"), (int) LocalDate.of(2024, 6, 18).toEpochDay()); + assertEquals(record.get("dateLocal"), (int) LocalDate.of(2024, 6, 18).toEpochDay()); + assertEquals(record.get("byte"), 100); + assertEquals(record.get("short"), 100); + assertEquals(record.get("yearMonthInterval"), 5); + assertEquals(record.get("long"), 100L); + assertEquals(record.get("instantMicros"), TEST_EPOCH_MILLIS * 1000); + assertEquals(record.get("instantMillis"), TEST_EPOCH_MILLIS); + assertEquals(record.get("timestampMicros"), TEST_EPOCH_MILLIS * 1000); + assertEquals(record.get("timestampMillis"), TEST_EPOCH_MILLIS); + assertEquals(record.get("timestampNoLogical"), TEST_EPOCH_MILLIS); + assertEquals(record.get("localTimestampMicros"), TEST_LOCAL_TIMESTAMP_MILLIS * 1000); + assertEquals(record.get("localTimestampMillis"), TEST_LOCAL_TIMESTAMP_MILLIS); + assertEquals(record.get("localTimestampNoLogical"), TEST_LOCAL_TIMESTAMP_MILLIS); + assertEquals(record.get("dayTimeInterval"), 100L * 1000 * 1000); + assertEquals(record.get("arrayIntList"), Arrays.asList(1, 2, 3)); + assertEquals(record.get("arrayIntSeq"), Arrays.asList(1, 2, 3)); + + GenericRecord complex_record_1 = new GenericData.Record(COMPLEX_SUB_SCHEMA_AVRO); + complex_record_1.put("int", 10); + complex_record_1.put("string", STRING_VALUE_2); + + GenericRecord complex_record_2 = new GenericData.Record(COMPLEX_SUB_SCHEMA_AVRO); + complex_record_2.put("int", 20); + complex_record_2.put("string", STRING_VALUE_3); + + assertEquals(record.get("arrayComplex"), Arrays.asList(complex_record_1, complex_record_2)); + + Map expectedIntMap = new HashMap() { + { + put("key1", 10); + put("key2", 20); + } + }; + assertEquals(record.get("mapIntJavaMap"), expectedIntMap); + assertEquals(record.get("mapIntScalaMap"), expectedIntMap); + + Map expectedComplexMap = new HashMap() { + { + put("key1", complex_record_1); + put("key2", complex_record_2); + } + }; + assertEquals(record.get("mapComplex"), expectedComplexMap); + + assertEquals(record.get("nullableUnion"), 10); + + assertNull(record.get("nullableUnion2")); + + assertEquals(record.get("singleElementUnion"), 10); + + assertEquals(record.get("intLongUnion"), 10L); + + assertEquals(record.get("longIntUnion"), 10L); + + assertEquals(record.get("floatDoubleUnion"), 0.5); + + assertEquals(record.get("doubleFloatUnion"), 0.5); + + Object complexNonNullableUnion = record.get("complexNonNullableUnion"); + assertTrue(complexNonNullableUnion instanceof Float); + assertEquals((Float) complexNonNullableUnion, 0.5f, 0.001f); + + Object complexNullableUnion1 = record.get("complexNullableUnion1"); + assertTrue(complexNullableUnion1 instanceof Integer); + assertEquals(((Integer) complexNullableUnion1).intValue(), 10); + + Object complexNullableUnion2 = record.get("complexNullableUnion2"); + assertTrue(complexNullableUnion2 instanceof CharSequence); + assertEquals(complexNullableUnion2, STRING_VALUE); + + assertNull(record.get("complexNullableUnion3")); + } + + @Test + public void testConvertToBoolean() { + Boolean trueObj = RowToAvroConverter.convertToBoolean(true, BooleanType); + assertNotNull(trueObj); + assertTrue(trueObj); + + Boolean falseObj = RowToAvroConverter.convertToBoolean(false, BooleanType); + assertNotNull(falseObj); + assertFalse(falseObj); + + // Type must be BooleanType + assertThrows(IllegalArgumentException.class, () -> RowToAvroConverter.convertToBoolean(true, ByteType)); + + // Data must be Boolean + assertThrows(IllegalArgumentException.class, () -> RowToAvroConverter.convertToBoolean(10, BooleanType)); + } + + @Test + public void testConvertToInt() { + Integer integer = + RowToAvroConverter.convertToInt(SPARK_ROW.getAs("int"), IntegerType, AVRO_SCHEMA.getField("int").schema()); + assertNotNull(integer); + assertEquals(integer.intValue(), 100); + + Integer date = + RowToAvroConverter.convertToInt(SPARK_ROW.getAs("date"), DateType, AVRO_SCHEMA.getField("date").schema()); + assertNotNull(date); + assertEquals(date.intValue(), (int) LocalDate.of(2024, 6, 18).toEpochDay()); + + Integer dateLocal = RowToAvroConverter + .convertToInt(SPARK_ROW.getAs("dateLocal"), DateType, AVRO_SCHEMA.getField("dateLocal").schema()); + assertNotNull(dateLocal); + assertEquals(dateLocal.intValue(), (int) LocalDate.of(2024, 6, 18).toEpochDay()); + + Integer byteInt = + RowToAvroConverter.convertToInt(SPARK_ROW.getAs("byte"), ByteType, AVRO_SCHEMA.getField("byte").schema()); + assertNotNull(byteInt); + assertEquals(byteInt.intValue(), 100); + + Integer shortInt = + RowToAvroConverter.convertToInt(SPARK_ROW.getAs("short"), ShortType, AVRO_SCHEMA.getField("short").schema()); + assertNotNull(shortInt); + assertEquals(shortInt.intValue(), 100); + + Integer yearMonthInterval = RowToAvroConverter.convertToInt( + SPARK_ROW.getAs("yearMonthInterval"), + DataTypes.createYearMonthIntervalType(), + AVRO_SCHEMA.getField("yearMonthInterval").schema()); + assertNotNull(yearMonthInterval); + assertEquals(yearMonthInterval.intValue(), 5); + + // Type must be IntegerType, ByteType, ShortType, DateType or YearMonthIntervalType + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToInt(0.5f, StringType, AVRO_SCHEMA.getField("int").schema())); + + // When using IntegerType, data must be an Integer + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToInt(10.0, IntegerType, AVRO_SCHEMA.getField("int").schema())); + + // When using ByteType, data must be a Byte + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToInt(10.0, ByteType, AVRO_SCHEMA.getField("byte").schema())); + + // When using ShortType, data must be a Short + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToInt(10.0, ShortType, AVRO_SCHEMA.getField("short").schema())); + + // When using DateType, data must be a java.time.LocalDate or java.sql.Date + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToInt(10.0, DateType, AVRO_SCHEMA.getField("date").schema())); + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToInt(10.0, DateType, AVRO_SCHEMA.getField("dateLocal").schema())); + + // When using DateType, the Avro schema must have a logical type of Date + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter + .convertToInt(LocalDate.of(2024, 6, 18), DateType, AVRO_SCHEMA.getField("int").schema())); + + // When using YearMonthIntervalType, data must be a Period + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToInt( + 10.0, + DataTypes.createYearMonthIntervalType(), + AVRO_SCHEMA.getField("yearMonthInterval").schema())); + } + + @Test + public void testConvertToLong() { + Long longType = + RowToAvroConverter.convertToLong(SPARK_ROW.getAs("long"), LongType, AVRO_SCHEMA.getField("long").schema()); + assertNotNull(longType); + assertEquals(longType.intValue(), 100); + + Long instantMicros = RowToAvroConverter + .convertToLong(SPARK_ROW.getAs("instantMicros"), TimestampType, AVRO_SCHEMA.getField("instantMicros").schema()); + assertNotNull(instantMicros); + assertEquals(instantMicros.longValue(), TEST_EPOCH_MILLIS * 1000); + + Long instantMillis = RowToAvroConverter + .convertToLong(SPARK_ROW.getAs("instantMillis"), TimestampType, AVRO_SCHEMA.getField("instantMillis").schema()); + assertNotNull(instantMillis); + assertEquals(instantMillis.longValue(), TEST_EPOCH_MILLIS); + + Long timestampMicros = RowToAvroConverter.convertToLong( + SPARK_ROW.getAs("timestampMicros"), + TimestampType, + AVRO_SCHEMA.getField("timestampMicros").schema()); + assertNotNull(timestampMicros); + assertEquals(timestampMicros.longValue(), TEST_EPOCH_MILLIS * 1000); + + Long timestampMillis = RowToAvroConverter.convertToLong( + SPARK_ROW.getAs("timestampMillis"), + TimestampType, + AVRO_SCHEMA.getField("timestampMillis").schema()); + assertNotNull(timestampMillis); + assertEquals(timestampMillis.longValue(), TEST_EPOCH_MILLIS); + + // When using TimestampType, and there is no logical type on the Avro schema, convert to millis by default + Long timestampNoLogical = RowToAvroConverter.convertToLong( + SPARK_ROW.getAs("timestampNoLogical"), + TimestampType, + AVRO_SCHEMA.getField("timestampNoLogical").schema()); + assertNotNull(timestampNoLogical); + assertEquals(timestampNoLogical.longValue(), TEST_EPOCH_MILLIS); + + Long localTimestampMicros = RowToAvroConverter.convertToLong( + SPARK_ROW.getAs("localTimestampMicros"), + TimestampNTZType$.MODULE$, + AVRO_SCHEMA.getField("localTimestampMicros").schema()); + assertNotNull(localTimestampMicros); + assertEquals(localTimestampMicros.longValue(), TEST_LOCAL_TIMESTAMP_MILLIS * 1000); + + Long localTimestampMillis = RowToAvroConverter.convertToLong( + SPARK_ROW.getAs("localTimestampMillis"), + TimestampNTZType$.MODULE$, + AVRO_SCHEMA.getField("localTimestampMillis").schema()); + assertNotNull(localTimestampMillis); + assertEquals(localTimestampMillis.longValue(), TEST_LOCAL_TIMESTAMP_MILLIS); + + // When using TimestampNTZType, and there is no logical type on the Avro schema, convert to millis by default + Long localTimestampNoLogical = RowToAvroConverter.convertToLong( + SPARK_ROW.getAs("localTimestampNoLogical"), + TimestampNTZType$.MODULE$, + AVRO_SCHEMA.getField("localTimestampNoLogical").schema()); + assertNotNull(localTimestampNoLogical); + assertEquals(localTimestampNoLogical.longValue(), TEST_LOCAL_TIMESTAMP_MILLIS); + + Long dayTimeInterval = RowToAvroConverter.convertToLong( + SPARK_ROW.getAs("dayTimeInterval"), + DataTypes.createDayTimeIntervalType(), + AVRO_SCHEMA.getField("dayTimeInterval").schema()); + assertNotNull(dayTimeInterval); + assertEquals(dayTimeInterval.longValue(), 100L * 1000 * 1000); + + // Type must be LongType, TimestampType, TimestampNTZType or DayTimeIntervalType + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToLong(0.5f, StringType, AVRO_SCHEMA.getField("long").schema())); + + // When using LongType, data must be a Long + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToLong(10.0, LongType, AVRO_SCHEMA.getField("long").schema())); + + // When using TimestampType, data must be a java.time.Instant or java.sql.Timestamp + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToLong(10.0, TimestampType, AVRO_SCHEMA.getField("instantMicros").schema())); + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToLong(10.0, TimestampType, AVRO_SCHEMA.getField("instantMillis").schema())); + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToLong(10.0, TimestampType, AVRO_SCHEMA.getField("timestampMicros").schema())); + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToLong(10.0, TimestampType, AVRO_SCHEMA.getField("timestampMillis").schema())); + + // When using TimestampNTZType, data must be a java.time.LocalDateTime + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter + .convertToLong(10.0, TimestampNTZType$.MODULE$, AVRO_SCHEMA.getField("localTimestampNoLogical").schema())); + + // When using DayTimeIntervalType, data must be a Duration + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToLong( + 10.0, + DataTypes.createDayTimeIntervalType(), + AVRO_SCHEMA.getField("dayTimeInterval").schema())); + } + + @Test + public void testConvertToFloat() { + Float floatObj = RowToAvroConverter.convertToFloat(0.5f, FloatType); + assertNotNull(floatObj); + assertEquals(floatObj, 0.5f, 0.0001f); + + // Type must be FloatType + assertThrows(IllegalArgumentException.class, () -> RowToAvroConverter.convertToFloat(0.5f, ByteType)); + + // Data must be Float + assertThrows(IllegalArgumentException.class, () -> RowToAvroConverter.convertToFloat(10, FloatType)); + } + + @Test + public void testConvertToDouble() { + Double doubleObj = RowToAvroConverter.convertToDouble(0.7, DoubleType); + assertNotNull(doubleObj); + assertEquals(doubleObj, 0.7, 0.0001); + + // Type must be DoubleType + assertThrows(IllegalArgumentException.class, () -> RowToAvroConverter.convertToDouble(0.7, ByteType)); + + // Data must be Double + assertThrows(IllegalArgumentException.class, () -> RowToAvroConverter.convertToDouble(true, DoubleType)); + } + + @Test + public void testConvertToString() { + CharSequence strObj = RowToAvroConverter.convertToString(STRING_VALUE, StringType); + assertNotNull(strObj); + assertEquals(strObj, STRING_VALUE); + + // Type must be StringType + assertThrows(IllegalArgumentException.class, () -> RowToAvroConverter.convertToString(STRING_VALUE, ByteType)); + + // Data must be String + assertThrows(IllegalArgumentException.class, () -> RowToAvroConverter.convertToString(100, StringType)); + } + + @Test + public void testConvertToBytes() { + ByteBuffer byteArrObj = RowToAvroConverter + .convertToBytes(SPARK_ROW.getAs("byteArr"), BinaryType, AVRO_SCHEMA.getField("byteArr").schema()); + assertNotNull(byteArrObj); + assertEquals(byteArrObj, ByteBuffer.wrap(new byte[] { 0x01, 0x02, 0x03 })); + + ByteBuffer byteBufferObj = RowToAvroConverter + .convertToBytes(SPARK_ROW.getAs("byteBuffer"), BinaryType, AVRO_SCHEMA.getField("byteBuffer").schema()); + assertNotNull(byteBufferObj); + assertEquals(byteBufferObj, ByteBuffer.wrap(new byte[] { 0x04, 0x05, 0x06 })); + + ByteBuffer decimalObj = RowToAvroConverter.convertToBytes( + SPARK_ROW.getAs("decimalBytes"), + DataTypes.createDecimalType(3, 2), + AVRO_SCHEMA.getField("decimalBytes").schema()); + assertNotNull(decimalObj); + assertEquals(decimalObj, ByteBuffer.wrap(new byte[] { 46 })); + + // The scale of the actual BigDecimal object shouldn't matter + ByteBuffer decimalObj2 = RowToAvroConverter.convertToBytes( + new BigDecimal("0.456").setScale(1, RoundingMode.HALF_UP), + DataTypes.createDecimalType(3, 2), + DECIMAL_TYPE); + assertNotNull(decimalObj2); + assertEquals(decimalObj2, ByteBuffer.wrap(new byte[] { 50 })); + + // Type must be BinaryType or DecimalType + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter + .convertToBytes(SPARK_ROW.getAs("byteArr"), ByteType, AVRO_SCHEMA.getField("byteArr").schema())); + + // Data must be byte[], ByteBuffer or BigDecimal + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter + .convertToBytes(SPARK_ROW.getAs("booleanTrue"), BinaryType, AVRO_SCHEMA.getField("booleanTrue").schema())); + + // Logical type scale must match the Spark type scale + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToBytes( + new BigDecimal("0.456").setScale(2, RoundingMode.HALF_UP), + DataTypes.createDecimalType(3, 2), + LogicalTypes.decimal(3, 3).addToSchema(Schema.create(Schema.Type.BYTES)))); + + // Logical type precision must match the Spark type precision + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToBytes( + new BigDecimal("0.456").setScale(4, RoundingMode.HALF_UP), + DataTypes.createDecimalType(4, 3), + LogicalTypes.decimal(3, 3).addToSchema(Schema.create(Schema.Type.BYTES)))); + } + + @Test + public void testConvertToFixed() { + GenericFixed byteArrObj = RowToAvroConverter + .convertToFixed(SPARK_ROW.getAs("byteArrFixed"), BinaryType, AVRO_SCHEMA.getField("byteArrFixed").schema()); + assertNotNull(byteArrObj); + assertNotNull(byteArrObj.bytes()); + assertEquals(byteArrObj.bytes().length, 3); + assertEquals(byteArrObj.bytes(), new byte[] { 0x01, 0x02, 0x03 }); + + GenericFixed byteBufferObj = RowToAvroConverter.convertToFixed( + SPARK_ROW.getAs("byteBufferFixed"), + BinaryType, + AVRO_SCHEMA.getField("byteBufferFixed").schema()); + assertNotNull(byteBufferObj); + assertNotNull(byteBufferObj.bytes()); + assertEquals(byteBufferObj.bytes().length, 3); + assertEquals(byteBufferObj.bytes(), new byte[] { 0x04, 0x05, 0x06 }); + + GenericFixed decimalObj = RowToAvroConverter.convertToFixed( + SPARK_ROW.getAs("decimalFixed"), + DataTypes.createDecimalType(3, 2), + AVRO_SCHEMA.getField("decimalFixed").schema()); + assertNotNull(decimalObj); + assertNotNull(decimalObj.bytes()); + assertEquals(decimalObj.bytes().length, 3); + assertEquals(decimalObj.bytes(), new byte[] { 0, 0, 46 }); + + // The scale of the actual BigDecimal object shouldn't matter + GenericFixed decimalObj2 = RowToAvroConverter.convertToFixed( + new BigDecimal("0.456").setScale(1, RoundingMode.HALF_UP), + DataTypes.createDecimalType(3, 2), + DECIMAL_FIXED_TYPE); + assertNotNull(decimalObj2); + assertNotNull(decimalObj2.bytes()); + assertEquals(decimalObj2.bytes().length, 3); + assertEquals(decimalObj2.bytes(), new byte[] { 0, 0, 50 }); + + // The byte array must have the correct length + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter + .convertToFixed(new byte[] { 0x00, 0x01 }, BinaryType, AVRO_SCHEMA.getField("byteArrFixed").schema())); + + // The byte buffer must have the correct length + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToFixed( + ByteBuffer.wrap(new byte[] { 0x00, 0x01 }), + BinaryType, + AVRO_SCHEMA.getField("byteBufferFixed").schema())); + + // Type must be BinaryType or DecimalType + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter + .convertToFixed(SPARK_ROW.getAs("byteArrFixed"), ByteType, AVRO_SCHEMA.getField("byteArrFixed").schema())); + + // Data must be byte[], ByteBuffer or BigDecimal + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter + .convertToFixed(SPARK_ROW.getAs("booleanTrue"), BinaryType, AVRO_SCHEMA.getField("booleanTrue").schema())); + + // Logical type scale must match the Spark type scale + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToFixed( + new BigDecimal("0.456").setScale(2, RoundingMode.HALF_UP), + DataTypes.createDecimalType(3, 2), + LogicalTypes.decimal(3, 3).addToSchema(Schema.createFixed("decimalFixed", null, null, 3)))); + + // Logical type precision must match the Spark type precision + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToFixed( + new BigDecimal("0.456").setScale(4, RoundingMode.HALF_UP), + DataTypes.createDecimalType(4, 3), + LogicalTypes.decimal(3, 3).addToSchema(Schema.createFixed("decimalFixed", null, null, 3)))); + } + + @Test + public void testConvertToEnum() { + GenericEnumSymbol enumObj = RowToAvroConverter + .convertToEnum(SPARK_ROW.getAs("enumType"), StringType, AVRO_SCHEMA.getField("enumType").schema()); + assertNotNull(enumObj); + assertEquals(enumObj, AvroCompatibilityHelper.newEnumSymbol(AVRO_SCHEMA.getField("enumType").schema(), "A")); + + // String value must be a valid symbol + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToEnum("D", StringType, AVRO_SCHEMA.getField("enumType").schema())); + + // Type must be StringType + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter + .convertToEnum(SPARK_ROW.getAs("enumType"), ByteType, AVRO_SCHEMA.getField("enumType").schema())); + + // Data must be String + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter + .convertToEnum(SPARK_ROW.getAs("booleanTrue"), StringType, AVRO_SCHEMA.getField("enumType").schema())); + } + + @Test + public void testConvertToArray() { + Schema arrayIntSchema = SchemaBuilder.array().items().intType(); + List arrayIntList = RowToAvroConverter + .convertToArray(SPARK_ROW.getAs("arrayIntList"), DataTypes.createArrayType(IntegerType), arrayIntSchema); + assertNotNull(arrayIntList); + assertEquals(arrayIntList, Arrays.asList(1, 2, 3)); + + List arrayIntSeq = RowToAvroConverter + .convertToArray(SPARK_ROW.getAs("arrayIntSeq"), DataTypes.createArrayType(IntegerType), arrayIntSchema); + assertNotNull(arrayIntSeq); + assertEquals(arrayIntSeq, Arrays.asList(1, 2, 3)); + + Schema arrayComplexSchema = SchemaBuilder.array().items(COMPLEX_SUB_SCHEMA_AVRO); + List arrayComplex = RowToAvroConverter.convertToArray( + SPARK_ROW.getAs("arrayComplex"), + DataTypes.createArrayType(COMPLEX_SUB_SCHEMA), + arrayComplexSchema); + assertNotNull(arrayComplex); + + GenericRecord complex_record_1 = new GenericData.Record(COMPLEX_SUB_SCHEMA_AVRO); + complex_record_1.put("int", 10); + complex_record_1.put("string", STRING_VALUE_2); + + GenericRecord complex_record_2 = new GenericData.Record(COMPLEX_SUB_SCHEMA_AVRO); + complex_record_2.put("int", 20); + complex_record_2.put("string", STRING_VALUE_3); + + assertEquals(arrayComplex, Arrays.asList(complex_record_1, complex_record_2)); + + // Type must be ArrayType + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToArray(SPARK_ROW.getAs("arrayIntList"), ByteType, arrayIntSchema)); + + // Data must be scala.collection.Seq or java.util.List + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToArray(100, DataTypes.createArrayType(IntegerType), arrayIntSchema)); + } + + @Test + public void testConvertToMap() { + Schema mapIntSchema = SchemaBuilder.map().values().intType(); + Map expectedIntMap = new HashMap() { + { + put("key1", 10); + put("key2", 20); + } + }; + + Map mapIntJavaMap = RowToAvroConverter + .convertToMap(SPARK_ROW.getAs("mapIntJavaMap"), DataTypes.createMapType(StringType, IntegerType), mapIntSchema); + assertNotNull(mapIntJavaMap); + assertEquals(mapIntJavaMap, expectedIntMap); + + Map mapIntScalaMap = RowToAvroConverter.convertToMap( + SPARK_ROW.getAs("mapIntScalaMap"), + DataTypes.createMapType(StringType, IntegerType), + mapIntSchema); + assertNotNull(mapIntScalaMap); + assertEquals(mapIntScalaMap, expectedIntMap); + + Schema mapComplexSchema = SchemaBuilder.map().values(COMPLEX_SUB_SCHEMA_AVRO); + Map mapComplex = RowToAvroConverter.convertToMap( + SPARK_ROW.getAs("mapComplex"), + DataTypes.createMapType(StringType, COMPLEX_SUB_SCHEMA), + mapComplexSchema); + assertNotNull(mapComplex); + + GenericRecord complex_record_1 = new GenericData.Record(COMPLEX_SUB_SCHEMA_AVRO); + complex_record_1.put("int", 10); + complex_record_1.put("string", STRING_VALUE_2); + + GenericRecord complex_record_2 = new GenericData.Record(COMPLEX_SUB_SCHEMA_AVRO); + complex_record_2.put("int", 20); + complex_record_2.put("string", STRING_VALUE_3); + + Map expectedComplexMap = new HashMap() { + { + put("key1", complex_record_1); + put("key2", complex_record_2); + } + }; + + assertEquals(mapComplex, expectedComplexMap); + + // Maps with keys that are not String are not supported + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToMap( + SPARK_ROW.getAs("mapIntJavaMap"), + DataTypes.createMapType(ByteType, IntegerType), + mapIntSchema)); + + // Type must be MapType + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToMap(SPARK_ROW.getAs("mapIntJavaMap"), ByteType, mapIntSchema)); + + // Data must be scala.collection.Map or java.util.Map + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToMap(100, DataTypes.createMapType(StringType, IntegerType), mapIntSchema)); + } + + @Test + public void testConvertToUnion() { + // null is allowed for nullable unions + assertNull(RowToAvroConverter.convertToUnion(null, IntegerType, AVRO_SCHEMA.getField("nullableUnion").schema())); + + // null is not allowed for non-nullable unions + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToUnion(null, LongType, AVRO_SCHEMA.getField("longIntUnion").schema())); + + // Test union with only 1 branch + Object singleElementUnion = + RowToAvroConverter.convertToUnion(10, IntegerType, AVRO_SCHEMA.getField("singleElementUnion").schema()); + assertTrue(singleElementUnion instanceof Integer); + assertEquals(((Integer) singleElementUnion).intValue(), 10); + + // Test union with two branches: null + something else + assertNull(RowToAvroConverter.convertToUnion(null, IntegerType, AVRO_SCHEMA.getField("nullableUnion").schema())); + Object nullableUnionValue = + RowToAvroConverter.convertToUnion(10, IntegerType, AVRO_SCHEMA.getField("nullableUnion").schema()); + assertTrue(nullableUnionValue instanceof Integer); + assertEquals(((Integer) nullableUnionValue).intValue(), 10); + + // Test union with two branches: something else + null + assertNull(RowToAvroConverter.convertToUnion(null, IntegerType, AVRO_SCHEMA.getField("nullableUnion2").schema())); + Object nullableUnion2Value = + RowToAvroConverter.convertToUnion(10, IntegerType, AVRO_SCHEMA.getField("nullableUnion2").schema()); + assertTrue(nullableUnion2Value instanceof Integer); + assertEquals(((Integer) nullableUnion2Value).intValue(), 10); + + // Test union with two branches: int + long + Object intLongUnion = + RowToAvroConverter.convertToUnion(10L, LongType, AVRO_SCHEMA.getField("intLongUnion").schema()); + assertTrue(intLongUnion instanceof Long); + assertEquals(((Long) intLongUnion).longValue(), 10L); + + // Test union with two branches: long + int + Object longIntUnion = + RowToAvroConverter.convertToUnion(10L, LongType, AVRO_SCHEMA.getField("intLongUnion").schema()); + assertTrue(longIntUnion instanceof Long); + assertEquals(((Long) longIntUnion).longValue(), 10L); + + // Test union with two branches: float + double + Object floatDoubleUnion = + RowToAvroConverter.convertToUnion(0.5, DoubleType, AVRO_SCHEMA.getField("floatDoubleUnion").schema()); + assertTrue(floatDoubleUnion instanceof Double); + assertEquals((Double) floatDoubleUnion, 0.5, 0.001); + + // Test union with two branches: double + float + Object doubleFloatUnion = + RowToAvroConverter.convertToUnion(0.5, DoubleType, AVRO_SCHEMA.getField("doubleFloatUnion").schema()); + assertTrue(doubleFloatUnion instanceof Double); + assertEquals((Double) doubleFloatUnion, 0.5, 0.001); + + // Test complex union without null + Object complexNonNullableUnion1 = RowToAvroConverter.convertToUnion( + new GenericRowWithSchema(new Object[] { null, 0.5f, null }, UNION_STRUCT_DOUBLE_FLOAT_STRING), + UNION_STRUCT_DOUBLE_FLOAT_STRING, + AVRO_SCHEMA.getField("complexNonNullableUnion").schema()); + assertTrue(complexNonNullableUnion1 instanceof Float); + assertEquals((Float) complexNonNullableUnion1, 0.5f, 0.001f); + + Object complexNonNullableUnion2 = RowToAvroConverter.convertToUnion( + new GenericRowWithSchema(new Object[] { 0.5, null, null }, UNION_STRUCT_DOUBLE_FLOAT_STRING), + UNION_STRUCT_DOUBLE_FLOAT_STRING, + AVRO_SCHEMA.getField("complexNonNullableUnion").schema()); + assertTrue(complexNonNullableUnion2 instanceof Double); + assertEquals((Double) complexNonNullableUnion2, 0.5, 0.001); + + Object complexNonNullableUnion3 = RowToAvroConverter.convertToUnion( + new GenericRowWithSchema(new Object[] { null, null, STRING_VALUE }, UNION_STRUCT_DOUBLE_FLOAT_STRING), + UNION_STRUCT_DOUBLE_FLOAT_STRING, + AVRO_SCHEMA.getField("complexNonNullableUnion").schema()); + assertTrue(complexNonNullableUnion3 instanceof String); + assertEquals(complexNonNullableUnion3, STRING_VALUE); + + // Test complex union with null in first branch + Object complexNullableUnion1_1 = RowToAvroConverter.convertToUnion( + new GenericRowWithSchema(new Object[] { null, 10 }, UNION_STRUCT_STRING_INT), + UNION_STRUCT_STRING_INT, + AVRO_SCHEMA.getField("complexNullableUnion1").schema()); + assertTrue(complexNullableUnion1_1 instanceof Integer); + assertEquals(((Integer) complexNullableUnion1_1).intValue(), 10); + + Object complexNullableUnion1_2 = RowToAvroConverter.convertToUnion( + new GenericRowWithSchema(new Object[] { STRING_VALUE, null }, UNION_STRUCT_STRING_INT), + UNION_STRUCT_STRING_INT, + AVRO_SCHEMA.getField("complexNullableUnion1").schema()); + assertTrue(complexNullableUnion1_2 instanceof String); + assertEquals(complexNullableUnion1_2, STRING_VALUE); + + Object complexNullableUnion1_3 = RowToAvroConverter + .convertToUnion(null, UNION_STRUCT_STRING_INT, AVRO_SCHEMA.getField("complexNullableUnion1").schema()); + assertNull(complexNullableUnion1_3); + + // Test complex union with null in second branch + Object complexNullableUnion2_1 = RowToAvroConverter.convertToUnion( + new GenericRowWithSchema(new Object[] { null, 10 }, UNION_STRUCT_STRING_INT), + UNION_STRUCT_STRING_INT, + AVRO_SCHEMA.getField("complexNullableUnion2").schema()); + assertTrue(complexNullableUnion2_1 instanceof Integer); + assertEquals(((Integer) complexNullableUnion2_1).intValue(), 10); + + Object complexNullableUnion2_2 = RowToAvroConverter.convertToUnion( + new GenericRowWithSchema(new Object[] { STRING_VALUE, null }, UNION_STRUCT_STRING_INT), + UNION_STRUCT_STRING_INT, + AVRO_SCHEMA.getField("complexNullableUnion2").schema()); + assertTrue(complexNullableUnion2_2 instanceof String); + assertEquals(complexNullableUnion2_2, STRING_VALUE); + + Object complexNullableUnion2_3 = RowToAvroConverter + .convertToUnion(null, UNION_STRUCT_STRING_INT, AVRO_SCHEMA.getField("complexNullableUnion2").schema()); + assertNull(complexNullableUnion2_3); + + // Test complex union with null in third branch + Object complexNullableUnion3_1 = RowToAvroConverter.convertToUnion( + new GenericRowWithSchema(new Object[] { null, 10 }, UNION_STRUCT_STRING_INT), + UNION_STRUCT_STRING_INT, + AVRO_SCHEMA.getField("complexNullableUnion3").schema()); + assertTrue(complexNullableUnion3_1 instanceof Integer); + assertEquals(((Integer) complexNullableUnion3_1).intValue(), 10); + + Object complexNullableUnion3_2 = RowToAvroConverter.convertToUnion( + new GenericRowWithSchema(new Object[] { STRING_VALUE, null }, UNION_STRUCT_STRING_INT), + UNION_STRUCT_STRING_INT, + AVRO_SCHEMA.getField("complexNullableUnion3").schema()); + assertTrue(complexNullableUnion3_2 instanceof String); + assertEquals(complexNullableUnion3_2, STRING_VALUE); + + Object complexNullableUnion3_3 = RowToAvroConverter + .convertToUnion(null, UNION_STRUCT_STRING_INT, AVRO_SCHEMA.getField("complexNullableUnion3").schema()); + assertNull(complexNullableUnion3_3); + + // At least one branch must be non-null + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.convertToUnion( + new GenericRowWithSchema(new Object[] { null, null }, UNION_STRUCT_STRING_INT), + UNION_STRUCT_STRING_INT, + AVRO_SCHEMA.getField("complexNullableUnion3").schema())); + } + + @Test + public void testValidateLogicalType() { + assertEquals( + RowToAvroConverter.validateLogicalType(DATE_TYPE, LogicalTypes.date(), LogicalTypes.timeMillis()), + LogicalTypes.date()); + + // Logical type must match the Spark type if it is mandatory + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.validateLogicalType(DATE_TYPE, LogicalTypes.timeMillis())); + + // Logical type must be present in the Avro schema if it is mandatory + assertThrows( + IllegalArgumentException.class, + () -> RowToAvroConverter.validateLogicalType(Schema.create(Schema.Type.LONG), LogicalTypes.timeMillis())); + + // Logical type might not be present in the Avro schema if it is optional + assertNull( + RowToAvroConverter.validateLogicalType(Schema.create(Schema.Type.LONG), false, LogicalTypes.timeMillis())); + } +} 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 b6930ba933..b6dd7bd164 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 @@ -34,6 +34,7 @@ import static com.linkedin.venice.vpj.VenicePushJobConstants.REPUSH_TTL_START_TIMESTAMP; import static com.linkedin.venice.vpj.VenicePushJobConstants.REWIND_TIME_IN_SECONDS_OVERRIDE; import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_KAFKA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SPARK_NATIVE_INPUT_FORMAT_ENABLED; import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -399,6 +400,7 @@ public void testIncrementalPushPartialUpdateNewFormat(boolean useSparkCompute) t vpjProperties.put(INCREMENTAL_PUSH, true); if (useSparkCompute) { vpjProperties.setProperty(DATA_WRITER_COMPUTE_JOB_CLASS, DataWriterSparkJob.class.getCanonicalName()); + vpjProperties.setProperty(SPARK_NATIVE_INPUT_FORMAT_ENABLED, String.valueOf(true)); } try (ControllerClient parentControllerClient = new ControllerClient(CLUSTER_NAME, parentControllerUrl)) { diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestBatch.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestBatch.java index 81a6945c05..d95accd2d2 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestBatch.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestBatch.java @@ -47,6 +47,7 @@ import static com.linkedin.venice.vpj.VenicePushJobConstants.SEND_CONTROL_MESSAGES_DIRECTLY; import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_ETL; import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_KAFKA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SPARK_NATIVE_INPUT_FORMAT_ENABLED; import static com.linkedin.venice.vpj.VenicePushJobConstants.USE_MAPPER_TO_BUILD_DICTIONARY; import static com.linkedin.venice.vpj.VenicePushJobConstants.VENICE_STORE_NAME_PROP; import static com.linkedin.venice.vpj.VenicePushJobConstants.ZSTD_COMPRESSION_LEVEL; @@ -769,7 +770,7 @@ public void testBatchFromETLWithNullDefaultValue() throws Exception { } @Test(timeOut = TEST_TIMEOUT) - public void testBatchFromETLWithForUnionWithNullSchema() throws Exception { + public void testBatchFromETLForUnionWithNullSchema() throws Exception { testBatchStore(inputDir -> { writeETLFileWithUnionWithNullSchema(inputDir); return new KeyAndValueSchemas(ETL_KEY_SCHEMA, ETL_UNION_VALUE_WITH_NULL_SCHEMA); @@ -796,7 +797,7 @@ public void testBatchFromETLWithForUnionWithNullSchema() throws Exception { } @Test(timeOut = TEST_TIMEOUT) - public void testBatchFromETLWithForUnionWithoutNullSchema() throws Exception { + public void testBatchFromETLForUnionWithoutNullSchema() throws Exception { testBatchStore(inputDir -> { writeETLFileWithUnionWithoutNullSchema(inputDir); return new KeyAndValueSchemas(ETL_KEY_SCHEMA, ETL_UNION_VALUE_WITHOUT_NULL_SCHEMA); @@ -895,6 +896,7 @@ private String testBatchStore( String inputDirPath = "file://" + inputDir.getAbsolutePath(); Properties props = defaultVPJProps(veniceCluster, inputDirPath, storeName); props.setProperty(DATA_WRITER_COMPUTE_JOB_CLASS, DataWriterSparkJob.class.getCanonicalName()); + props.setProperty(SPARK_NATIVE_INPUT_FORMAT_ENABLED, String.valueOf(true)); extraProps.accept(props); if (StringUtils.isEmpty(existingStore)) { @@ -1419,7 +1421,7 @@ public void testKafkaInputBatchJobSucceedsWhenSourceTopicIsEmpty() throws Except } @Test(timeOut = TEST_TIMEOUT, dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) - public void testBatchJobSnapshots(Boolean isKakfaPush) throws Exception { + public void testBatchJobSnapshots(Boolean isKafkaPush) throws Exception { VPJValidator validator = (avroClient, vsonClient, metricsRepository) -> { for (int i = 1; i <= 100; i++) { @@ -1437,7 +1439,7 @@ public void testBatchJobSnapshots(Boolean isKakfaPush) throws Exception { deleteDirectory(Paths.get(BASE_DATA_PATH_1).toFile()); deleteDirectory(Paths.get(BASE_DATA_PATH_2).toFile()); - if (isKakfaPush) { + if (isKafkaPush) { testRepush(storeName, validator); } else { testBatchStore( diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestVsonStoreBatch.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestVsonStoreBatch.java index 2b2642c730..16f97ddb98 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestVsonStoreBatch.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestVsonStoreBatch.java @@ -15,6 +15,7 @@ import static com.linkedin.venice.vpj.VenicePushJobConstants.KAFKA_INPUT_TOPIC; import static com.linkedin.venice.vpj.VenicePushJobConstants.KEY_FIELD_PROP; import static com.linkedin.venice.vpj.VenicePushJobConstants.SOURCE_KAFKA; +import static com.linkedin.venice.vpj.VenicePushJobConstants.SPARK_NATIVE_INPUT_FORMAT_ENABLED; import static com.linkedin.venice.vpj.VenicePushJobConstants.VALUE_FIELD_PROP; import com.linkedin.venice.client.store.AvroGenericStoreClient; @@ -322,6 +323,7 @@ private String testBatchStore( String inputDirPath = "file://" + inputDir.getAbsolutePath(); Properties props = defaultVPJPropsWithoutD2Routing(veniceCluster, inputDirPath, storeName); props.setProperty(DATA_WRITER_COMPUTE_JOB_CLASS, DataWriterSparkJob.class.getCanonicalName()); + props.setProperty(SPARK_NATIVE_INPUT_FORMAT_ENABLED, String.valueOf(true)); extraProps.accept(props); if (!storeNameOptional.isPresent()) { From 1df15f8e7d3fcbd00faeef2c7ec5a7dd18f509d7 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Tue, 24 Sep 2024 16:31:55 -0700 Subject: [PATCH 10/16] [controller] Set rebalancing strategy to Waged instead of Crush when adding a resource to controller cluster (#1197) --- .../com/linkedin/venice/controller/ZkHelixAdminClient.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/ZkHelixAdminClient.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/ZkHelixAdminClient.java index 38afcaecd5..89c60e7236 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/ZkHelixAdminClient.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/ZkHelixAdminClient.java @@ -16,7 +16,7 @@ import org.apache.helix.cloud.constants.CloudProvider; import org.apache.helix.controller.rebalancer.DelayedAutoRebalancer; import org.apache.helix.controller.rebalancer.strategy.AutoRebalanceStrategy; -import org.apache.helix.controller.rebalancer.strategy.CrushRebalanceStrategy; +import org.apache.helix.controller.rebalancer.waged.WagedRebalancer; import org.apache.helix.manager.zk.ZKHelixAdmin; import org.apache.helix.manager.zk.ZKHelixManager; import org.apache.helix.manager.zk.ZNRecordSerializer; @@ -167,8 +167,7 @@ public void addVeniceStorageClusterToControllerCluster(String clusterName) { VeniceControllerClusterConfig config = multiClusterConfigs.getControllerConfig(clusterName); IdealState idealState = helixAdmin.getResourceIdealState(controllerClusterName, clusterName); idealState.setMinActiveReplicas(controllerClusterReplicaCount); - idealState.setRebalancerClassName(DelayedAutoRebalancer.class.getName()); - idealState.setRebalanceStrategy(CrushRebalanceStrategy.class.getName()); + idealState.setRebalancerClassName(WagedRebalancer.class.getName()); String instanceGroupTag = config.getControllerResourceInstanceGroupTag(); if (!instanceGroupTag.isEmpty()) { From 939d655324d757c790092d3158dbcbabbb73a528 Mon Sep 17 00:00:00 2001 From: Hao Xu Date: Tue, 24 Sep 2024 18:41:25 -0700 Subject: [PATCH 11/16] [controller][compat] Controller part change for supporting separate real-time topic functionality for hybrid stores. (#1172) This change add store version level config separateRealTimeTopicEnabled to allow creating separate real-time topic for isolate bulk real-time traffic in short time range from incremental push. There is cluster level config for enabling this config: ENABLE_SEPARATE_REAL_TIME_TOPIC_FOR_STORE_WITH_INCREMENTAL_PUSH for all newly converted hybrid store with incremental push enabled. Creation: it will be automatically created by checking whether incremental push and separateRealTimeTopicEnabled is enabled when hybrid enabled store version is added and normal real-time topic is created. Deletion: separate real time topic will be cleaned up when normal real-time topic is deleted due to all hybrid store version has been deleted. Added Integration test to see if incremental push job would send expected traffic to the separate real-time topic after new store is turned this feature. PS: skip the flaky test: testDaVinciMemoryLimitShouldFailLargeDataPushAndResumeHybridStore. Co-authored-by: Hao Xu --- .../java/com/linkedin/venice/AdminTool.java | 1 + .../main/java/com/linkedin/venice/Arg.java | 4 + .../java/com/linkedin/venice/Command.java | 3 +- .../java/com/linkedin/venice/ConfigKeys.java | 8 + .../controllerapi/ControllerApiConstants.java | 1 + .../controllerapi/UpdateStoreQueryParams.java | 9 + .../linkedin/venice/meta/AbstractStore.java | 2 + .../linkedin/venice/meta/ReadOnlyStore.java | 20 + .../java/com/linkedin/venice/meta/Store.java | 4 + .../com/linkedin/venice/meta/SystemStore.java | 10 + .../com/linkedin/venice/meta/Version.java | 17 +- .../com/linkedin/venice/meta/VersionImpl.java | 11 + .../com/linkedin/venice/meta/ZKStore.java | 11 + .../avro/AvroProtocolDefinition.java | 4 +- .../StoreMetaValue/v24/StoreMetaValue.avsc | 399 ++++++ ...VeniceHelixAdminWithSharedEnvironment.java | 12 +- .../DaVinciClientMemoryLimitTest.java | 1 - ...TestActiveActiveReplicationForIncPush.java | 127 +- .../com/linkedin/venice/controller/Admin.java | 2 + .../VeniceControllerClusterConfig.java | 13 + .../venice/controller/VeniceHelixAdmin.java | 52 +- .../controller/VeniceParentHelixAdmin.java | 18 + .../kafka/consumer/AdminExecutionTask.java | 1 + .../controller/server/CreateVersion.java | 7 +- .../control/RealTimeTopicSwitcher.java | 36 +- .../AdminOperation/v81/AdminOperation.avsc | 1134 +++++++++++++++++ .../controller/server/CreateVersionTest.java | 11 +- 27 files changed, 1867 insertions(+), 51 deletions(-) create mode 100644 internal/venice-common/src/main/resources/avro/StoreMetaValue/v24/StoreMetaValue.avsc create mode 100644 services/venice-controller/src/main/resources/avro/AdminOperation/v81/AdminOperation.avsc diff --git a/clients/venice-admin-tool/src/main/java/com/linkedin/venice/AdminTool.java b/clients/venice-admin-tool/src/main/java/com/linkedin/venice/AdminTool.java index 31abdd397b..49a3fda801 100644 --- a/clients/venice-admin-tool/src/main/java/com/linkedin/venice/AdminTool.java +++ b/clients/venice-admin-tool/src/main/java/com/linkedin/venice/AdminTool.java @@ -1116,6 +1116,7 @@ static UpdateStoreQueryParams getUpdateStoreQueryParams(CommandLine cmd) { integerParam(cmd, Arg.BATCH_GET_LIMIT, p -> params.setBatchGetLimit(p), argSet); integerParam(cmd, Arg.NUM_VERSIONS_TO_PRESERVE, p -> params.setNumVersionsToPreserve(p), argSet); booleanParam(cmd, Arg.INCREMENTAL_PUSH_ENABLED, p -> params.setIncrementalPushEnabled(p), argSet); + booleanParam(cmd, Arg.SEPARATE_REALTIME_TOPIC_ENABLED, p -> params.setSeparateRealTimeTopicEnabled(p), argSet); booleanParam(cmd, Arg.WRITE_COMPUTATION_ENABLED, p -> params.setWriteComputationEnabled(p), argSet); booleanParam(cmd, Arg.READ_COMPUTATION_ENABLED, p -> params.setReadComputationEnabled(p), argSet); integerParam( diff --git a/clients/venice-admin-tool/src/main/java/com/linkedin/venice/Arg.java b/clients/venice-admin-tool/src/main/java/com/linkedin/venice/Arg.java index 89e48d6f4a..c9b7c0940d 100644 --- a/clients/venice-admin-tool/src/main/java/com/linkedin/venice/Arg.java +++ b/clients/venice-admin-tool/src/main/java/com/linkedin/venice/Arg.java @@ -100,6 +100,10 @@ public enum Arg { ), INCREMENTAL_PUSH_ENABLED( "incremental-push-enabled", "ipe", true, "a flag to see if the store supports incremental push or not" + ), + SEPARATE_REALTIME_TOPIC_ENABLED( + "separate-realtime-topic-enabled", "srte", true, + "a flag to see if the store supports separate real-time topic or not" ), BATCH_GET_LIMIT("batch-get-limit", "bgl", true, "Key number limit inside one batch-get request"), NUM_VERSIONS_TO_PRESERVE("num-versions-to-preserve", "nvp", true, "Number of version that store should preserve."), KAFKA_BOOTSTRAP_SERVERS("kafka-bootstrap-servers", "kbs", true, "Kafka bootstrap server URL(s)"), diff --git a/clients/venice-admin-tool/src/main/java/com/linkedin/venice/Command.java b/clients/venice-admin-tool/src/main/java/com/linkedin/venice/Command.java index b22a9fc3c7..cc10836b08 100644 --- a/clients/venice-admin-tool/src/main/java/com/linkedin/venice/Command.java +++ b/clients/venice-admin-tool/src/main/java/com/linkedin/venice/Command.java @@ -99,6 +99,7 @@ import static com.linkedin.venice.Arg.REPLICATION_FACTOR; import static com.linkedin.venice.Arg.RETRY; import static com.linkedin.venice.Arg.RMD_CHUNKING_ENABLED; +import static com.linkedin.venice.Arg.SEPARATE_REALTIME_TOPIC_ENABLED; import static com.linkedin.venice.Arg.SERVER_KAFKA_FETCH_QUOTA_RECORDS_PER_SECOND; import static com.linkedin.venice.Arg.SERVER_URL; import static com.linkedin.venice.Arg.SKIP_DIV; @@ -267,7 +268,7 @@ public enum Command { ACTIVE_ACTIVE_REPLICATION_ENABLED, REGIONS_FILTER, DISABLE_META_STORE, DISABLE_DAVINCI_PUSH_STATUS_STORE, STORAGE_PERSONA, STORE_VIEW_CONFIGS, LATEST_SUPERSET_SCHEMA_ID, MIN_COMPACTION_LAG_SECONDS, MAX_COMPACTION_LAG_SECONDS, MAX_RECORD_SIZE_BYTES, MAX_NEARLINE_RECORD_SIZE_BYTES, - UNUSED_SCHEMA_DELETION_ENABLED, BLOB_TRANSFER_ENABLED } + UNUSED_SCHEMA_DELETION_ENABLED, BLOB_TRANSFER_ENABLED, SEPARATE_REALTIME_TOPIC_ENABLED } ), UPDATE_CLUSTER_CONFIG( "update-cluster-config", "Update live cluster configs", new Arg[] { URL, CLUSTER }, 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 e25fc8fa0e..9960a61097 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 @@ -1192,6 +1192,14 @@ private ConfigKeys() { */ public static final String ENABLE_INCREMENTAL_PUSH_FOR_HYBRID_ACTIVE_ACTIVE_USER_STORES = "enable.incremental.push.for.hybrid.active.active.user.stores"; + + /** + * We will use this config to determine whether we should enable separate real-time topic for incremental push enabled stores. + * If this config is set to true, we will enable separate real-time topic for incremental push enabled stores. + */ + public static final String ENABLE_SEPARATE_REAL_TIME_TOPIC_FOR_STORE_WITH_INCREMENTAL_PUSH = + "enable.separate.real.time.topic.for.store.with.incremental.push"; + /** * We will use this config to determine whether we should enable partial update for hybrid active-active user stores. * If this config is set to true, we will enable partial update for hybrid active-active user stores whose latest value diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/ControllerApiConstants.java b/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/ControllerApiConstants.java index d6caeed561..e647ddd3e7 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/ControllerApiConstants.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/ControllerApiConstants.java @@ -56,6 +56,7 @@ public class ControllerApiConstants { public static final String CHUNKING_ENABLED = "chunking_enabled"; public static final String RMD_CHUNKING_ENABLED = "rmd_chunking_enabled"; public static final String INCREMENTAL_PUSH_ENABLED = "incremental_push_enabled"; + public static final String SEPARATE_REAL_TIME_TOPIC_ENABLED = "separate_realtime_topic_enabled"; public static final String SINGLE_GET_ROUTER_CACHE_ENABLED = "single_get_router_cache_enabled"; public static final String BATCH_GET_ROUTER_CACHE_ENABLED = "batch_get_router_cache_enabled"; public static final String BATCH_GET_LIMIT = "batch_get_limit"; diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/UpdateStoreQueryParams.java b/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/UpdateStoreQueryParams.java index 71573848dd..168d2ee27e 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/UpdateStoreQueryParams.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/controllerapi/UpdateStoreQueryParams.java @@ -50,6 +50,7 @@ import static com.linkedin.venice.controllerapi.ControllerApiConstants.REPLICATION_METADATA_PROTOCOL_VERSION_ID; import static com.linkedin.venice.controllerapi.ControllerApiConstants.REWIND_TIME_IN_SECONDS; import static com.linkedin.venice.controllerapi.ControllerApiConstants.RMD_CHUNKING_ENABLED; +import static com.linkedin.venice.controllerapi.ControllerApiConstants.SEPARATE_REAL_TIME_TOPIC_ENABLED; import static com.linkedin.venice.controllerapi.ControllerApiConstants.STORAGE_NODE_READ_QUOTA_ENABLED; import static com.linkedin.venice.controllerapi.ControllerApiConstants.STORAGE_QUOTA_IN_BYTE; import static com.linkedin.venice.controllerapi.ControllerApiConstants.STORE_MIGRATION; @@ -406,6 +407,14 @@ public Optional getIncrementalPushEnabled() { return getBoolean(INCREMENTAL_PUSH_ENABLED); } + public UpdateStoreQueryParams setSeparateRealTimeTopicEnabled(boolean separateRealTimeTopicEnabled) { + return putBoolean(SEPARATE_REAL_TIME_TOPIC_ENABLED, separateRealTimeTopicEnabled); + } + + public Optional getSeparateRealTimeTopicEnabled() { + return getBoolean(SEPARATE_REAL_TIME_TOPIC_ENABLED); + } + public UpdateStoreQueryParams setBatchGetLimit(int batchGetLimit) { return putInteger(BATCH_GET_LIMIT, batchGetLimit); } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/meta/AbstractStore.java b/internal/venice-common/src/main/java/com/linkedin/venice/meta/AbstractStore.java index d3c0a21934..62c318899d 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/meta/AbstractStore.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/meta/AbstractStore.java @@ -152,6 +152,8 @@ private void addVersion(Version version, boolean checkDisableWrite, boolean isCl version.setIncrementalPushEnabled(isIncrementalPushEnabled()); + version.setSeparateRealTimeTopicEnabled(isSeparateRealTimeTopicEnabled()); + version.setBlobTransferEnabled(isBlobTransferEnabled()); version.setUseVersionLevelIncrementalPushEnabled(true); diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/meta/ReadOnlyStore.java b/internal/venice-common/src/main/java/com/linkedin/venice/meta/ReadOnlyStore.java index f393701010..1679bb53b7 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/meta/ReadOnlyStore.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/meta/ReadOnlyStore.java @@ -483,6 +483,16 @@ public void setIncrementalPushEnabled(boolean incrementalPushEnabled) { throw new UnsupportedOperationException(); } + @Override + public boolean isSeparateRealTimeTopicEnabled() { + return this.delegate.isSeparateRealTimeTopicEnabled(); + } + + @Override + public void setSeparateRealTimeTopicEnabled(boolean separateRealTimeTopicEnabled) { + throw new UnsupportedOperationException(); + } + @Override public boolean isBlobTransferEnabled() { return this.delegate.isBlobTransferEnabled(); @@ -955,6 +965,16 @@ public void setIncrementalPushEnabled(boolean incrementalPushEnabled) { throw new UnsupportedOperationException(); } + @Override + public boolean isSeparateRealTimeTopicEnabled() { + return this.delegate.isSeparateRealTimeTopicEnabled(); + } + + @Override + public void setSeparateRealTimeTopicEnabled(boolean separateRealTimeTopicEnabled) { + throw new UnsupportedOperationException(); + } + @Override public boolean isAccessControlled() { return this.delegate.isAccessControlled(); diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/meta/Store.java b/internal/venice-common/src/main/java/com/linkedin/venice/meta/Store.java index af27977609..75eded6878 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/meta/Store.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/meta/Store.java @@ -150,6 +150,10 @@ static boolean isSystemStore(String storeName) { void setIncrementalPushEnabled(boolean incrementalPushEnabled); + boolean isSeparateRealTimeTopicEnabled(); + + void setSeparateRealTimeTopicEnabled(boolean separateRealTimeTopicEnabled); + boolean isAccessControlled(); void setAccessControlled(boolean accessControlled); diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/meta/SystemStore.java b/internal/venice-common/src/main/java/com/linkedin/venice/meta/SystemStore.java index c56691e058..ace64de3c1 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/meta/SystemStore.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/meta/SystemStore.java @@ -339,6 +339,16 @@ public void setIncrementalPushEnabled(boolean incrementalPushEnabled) { throwUnsupportedOperationException("setIncrementalPushEnabled"); } + @Override + public boolean isSeparateRealTimeTopicEnabled() { + return zkSharedStore.isSeparateRealTimeTopicEnabled(); + } + + @Override + public void setSeparateRealTimeTopicEnabled(boolean separateRealTimeTopicEnabled) { + throwUnsupportedOperationException("setSeparateRealTimeTopicEnabled"); + } + @Override public boolean isAccessControlled() { return zkSharedStore.isAccessControlled(); diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/meta/Version.java b/internal/venice-common/src/main/java/com/linkedin/venice/meta/Version.java index e23d5a1466..0b4ba9f5f0 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/meta/Version.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/meta/Version.java @@ -25,7 +25,7 @@ public interface Version extends Comparable, DataModelBackedStructure REWIND_FROM_EOP (replay from 'EOP - rewindTimeInSeconds'), 1 => REWIND_FROM_SOP (replay from 'SOP - rewindTimeInSeconds')", + "default": 0 + } + ] + } + ], + "default": null + }, + { + "name": "views", + "doc": "A map of views which describe and configure a downstream view of a venice store. Keys in this map are for convenience of managing configs.", + "type": { + "type":"map", + "values": { + "name": "StoreViewConfig", + "type": "record", + "doc": "A configuration for a particular view. This config should inform Venice leaders how to transform and transmit data to destination views.", + "fields": [ + { + "name": "viewClassName", + "type": "string", + "doc": "This informs what kind of view we are materializing. This then informs what kind of parameters are passed to parse this input. This is expected to be a fully formed class path name for materialization.", + "default": "" + }, + { + "name": "viewParameters", + "doc": "Optional parameters to be passed to the given view config.", + "type": ["null", + { + "type": "map", + "java-key-class": "java.lang.String", + "avro.java.string": "String", + "values": { "type": "string", "avro.java.string": "String" } + } + ], + "default": null + } + ] + } + }, + "default": {} + }, + {"name": "accessControlled", "type": "boolean", "default": true, "doc": "Store-level ACL switch. When disabled, Venice Router should accept every request."}, + {"name": "compressionStrategy", "type": "int", "default": 0, "doc": "Strategy used to compress/decompress Record's value, and default is 'NO_OP'"}, + {"name": "clientDecompressionEnabled", "type": "boolean", "default": true, "doc": "le/Disable client-side record decompression (default: true)"}, + {"name": "chunkingEnabled", "type": "boolean", "default": false, "doc": "Whether current store supports large value (typically more than 1MB). By default, the chunking feature is disabled."}, + {"name": "rmdChunkingEnabled", "type": "boolean", "default": false, "doc": "Whether current store supports large replication metadata (typically more than 1MB). By default, the chunking feature is disabled."}, + {"name": "batchGetLimit", "type": "int", "default": -1, "doc": "Batch get key number limit, and Venice will use cluster-level config if it is not positive."}, + {"name": "numVersionsToPreserve", "type": "int", "default": 0, "doc": "How many versions this store preserve at most. By default it's 0 means we use the cluster level config to determine how many version is preserved."}, + {"name": "incrementalPushEnabled", "type": "boolean", "default": false, "doc": "Flag to see if the store supports incremental push or not"}, + {"name": "separateRealTimeTopicEnabled", "type": "boolean", "default": false, "doc": "Flag to see if the store supports separate real-time topic for incremental push."}, + {"name": "migrating", "type": "boolean", "default": false, "doc": "Whether or not the store is in the process of migration."}, + {"name": "writeComputationEnabled", "type": "boolean", "default": false, "doc": "Whether or not write-path computation feature is enabled for this store."}, + {"name": "readComputationEnabled", "type": "boolean", "default": false, "doc": "Whether read-path computation is enabled for this store."}, + {"name": "bootstrapToOnlineTimeoutInHours", "type": "int", "default": 24, "doc": "Maximum number of hours allowed for the store to transition from bootstrap to online state."}, + {"name": "leaderFollowerModelEnabled", "type": "boolean", "default": false, "doc": "Whether or not to use leader follower state transition model for upcoming version."}, + {"name": "nativeReplicationEnabled", "type": "boolean", "default": false, "doc": "Whether or not native should be enabled for this store. Will only successfully apply if leaderFollowerModelEnabled is also true either in this update or a previous version of the store."}, + {"name": "replicationMetadataVersionID", "type": "int", "default": -1, "doc": "RMD (Replication metadata) version ID on the store-level. Default -1 means NOT_SET and the cluster-level RMD version ID should be used for stores."}, + {"name": "pushStreamSourceAddress", "type": "string", "default": "", "doc": "Address to the kafka broker which holds the source of truth topic for this store version."}, + {"name": "backupStrategy", "type": "int", "default": 1, "doc": "Strategies to store backup versions, and default is 'DELETE_ON_NEW_PUSH_START'"}, + {"name": "schemaAutoRegisteFromPushJobEnabled", "type": "boolean", "default": false, "doc": "Whether or not value schema auto registration enabled from push job for this store."}, + {"name": "latestSuperSetValueSchemaId", "type": "int", "default": -1, "doc": "For read compute stores with auto super-set schema enabled, stores the latest super-set value schema ID."}, + {"name": "hybridStoreDiskQuotaEnabled", "type": "boolean", "default": false, "doc": "Whether or not storage disk quota is enabled for a hybrid store. This store config cannot be enabled until the routers and servers in the corresponding cluster are upgraded to the right version: 0.2.249 or above for routers and servers."}, + {"name": "storeMetadataSystemStoreEnabled", "type": "boolean", "default": false, "doc": "Whether or not the store metadata system store is enabled for this store."}, + { + "name": "etlConfig", + "doc": "Properties related to ETL Store behavior.", + "type": [ + "null", + { + "name": "StoreETLConfig", + "type": "record", + "fields": [ + {"name": "etledUserProxyAccount", "type": "string", "doc": "If enabled regular ETL or future version ETL, this account name is part of path for where the ETLed snapshots will go. for example, for user account veniceetl001, snapshots will be published to HDFS /jobs/veniceetl001/storeName."}, + {"name": "regularVersionETLEnabled", "type": "boolean", "doc": "Whether or not enable regular version ETL for this store."}, + {"name": "futureVersionETLEnabled", "type": "boolean", "doc": "Whether or not enable future version ETL - the version that might come online in future - for this store."} + ] + } + ], + "default": null + }, + { + "name": "partitionerConfig", + "doc": "", + "type": [ + "null", + { + "name": "StorePartitionerConfig", + "type": "record", + "fields": [ + {"name": "partitionerClass", "type": "string"}, + {"name": "partitionerParams", "type": {"type": "map", "values": "string"}}, + {"name": "amplificationFactor", "type": "int"} + ] + } + ], + "default": null + }, + {"name": "incrementalPushPolicy", "type": "int", "default": 0, "doc": "Incremental Push Policy to reconcile with real time pushes, and default is 'PUSH_TO_VERSION_TOPIC'"}, + {"name": "latestVersionPromoteToCurrentTimestamp", "type": "long", "default": -1, "doc": "This is used to track the time when a new version is promoted to current version. For now, it is mostly to decide whether a backup version can be removed or not based on retention. For the existing store before this code change, it will be set to be current timestamp."}, + {"name": "backupVersionRetentionMs", "type": "long", "default": -1, "doc": "Backup retention time, and if it is not set (-1), Venice Controller will use the default configured retention. {@link com.linkedin.venice.ConfigKeys#CONTROLLER_BACKUP_VERSION_DEFAULT_RETENTION_MS}."}, + {"name": "replicationFactor", "type": "int", "default": 3, "doc": "The number of replica each store version will keep."}, + {"name": "migrationDuplicateStore", "type": "boolean", "default": false, "doc": "Whether or not the store is a duplicate store in the process of migration."}, + {"name": "nativeReplicationSourceFabric", "type": "string", "default": "", "doc": "The source fabric name to be uses in native replication. Remote consumption will happen from kafka in this fabric."}, + {"name": "daVinciPushStatusStoreEnabled", "type": "boolean", "default": false, "doc": "Whether or not davinci push status store is enabled."}, + {"name": "storeMetaSystemStoreEnabled", "type": "boolean", "default": false, "doc": "Whether or not the store meta system store is enabled for this store."}, + {"name": "activeActiveReplicationEnabled", "type": "boolean", "default": false, "doc": "Whether or not active/active replication is enabled for hybrid stores; eventually this config will replace native replication flag, when all stores are on A/A"}, + {"name": "applyTargetVersionFilterForIncPush", "type": "boolean", "default": false, "doc": "Whether or not the target version field in Kafka messages will be used in increment push to RT policy"}, + {"name": "minCompactionLagSeconds", "type": "long", "default": -1, "doc": "Store level min compaction lag config and if not specified, it will use the global config for version topics"}, + {"name": "maxCompactionLagSeconds", "type": "long", "default": -1, "doc": "Store level max compaction lag config and if not specified, 'max.compaction.lag.ms' config won't be setup in the corresponding version topics"}, + {"name": "maxRecordSizeBytes", "type": "int", "default": -1, "doc": "Store-level max record size in bytes. If not specified (-1), the controller config 'default.max.record.size.bytes' (100MB default) will be backfilled"}, + {"name": "maxNearlineRecordSizeBytes", "type": "int", "default": -1, "doc": "Store-level max record size in bytes for nearline jobs with partial updates. If not specified (-1), the server config 'default.max.record.size.bytes' (100MB default) will be backfilled. This may converge with maxRecordSizeBytes in the future"}, + {"name": "unusedSchemaDeletionEnabled", "type": "boolean", "default": false, "doc": "Store level config to indicate whether unused schema deletion is enabled or not."}, + { + "name": "versions", + "doc": "List of non-retired versions. It's currently sorted and there is code run under the assumption that the last element in the list is the largest. Check out {VeniceHelixAdmin#getIncrementalPushVersion}, and please make it in mind if you want to change this logic", + "type": { + "type": "array", + "items": { + "name": "StoreVersion", + "type": "record", + "doc": "Type describes all the version attributes", + "fields": [ + {"name": "storeName", "type": "string", "doc": "Name of the store which this version belong to."}, + {"name": "number", "type": "int", "doc": "Version number."}, + {"name": "createdTime", "type": "long", "doc": "Time when this version was created."}, + {"name": "status", "type": "int", "default": 1, "doc": "Status of version, and default is 'STARTED'"}, + {"name": "pushJobId", "type": "string", "default": ""}, + {"name": "compressionStrategy", "type": "int", "default": 0, "doc": "strategies used to compress/decompress Record's value, and default is 'NO_OP'"}, + {"name": "leaderFollowerModelEnabled", "type": "boolean", "default": false, "doc": "Whether or not to use leader follower state transition."}, + {"name": "nativeReplicationEnabled", "type": "boolean", "default": false, "doc": "Whether or not native replication is enabled."}, + {"name": "pushStreamSourceAddress", "type": "string", "default": "", "doc": "Address to the kafka broker which holds the source of truth topic for this store version."}, + {"name": "bufferReplayEnabledForHybrid", "type": "boolean", "default": true, "doc": "Whether or not to enable buffer replay for hybrid."}, + {"name": "chunkingEnabled", "type": "boolean", "default": false, "doc": "Whether or not large values are supported (via chunking)."}, + {"name": "rmdChunkingEnabled", "type": "boolean", "default": false, "doc": "Whether or not large replication metadata are supported (via chunking)."}, + {"name": "pushType", "type": "int", "default": 0, "doc": "Producer type for this version, and default is 'BATCH'"}, + {"name": "partitionCount", "type": "int", "default": 0, "doc": "Partition count of this version."}, + { + "name": "partitionerConfig", + "type": [ + "null", + "com.linkedin.venice.systemstore.schemas.StorePartitionerConfig" + ], + "default": null, + "doc": "Config for custom partitioning." + }, + {"name": "incrementalPushPolicy", "type": "int", "default": 0, "doc": "Incremental Push Policy to reconcile with real time pushes., and default is 'PUSH_TO_VERSION_TOPIC'"}, + {"name": "replicationFactor", "type": "int", "default": 3, "doc": "The number of replica this store version is keeping."}, + {"name": "nativeReplicationSourceFabric", "type": "string", "default": "", "doc": "The source fabric name to be uses in native replication. Remote consumption will happen from kafka in this fabric."}, + {"name": "incrementalPushEnabled", "type": "boolean", "default": false, "doc": "Flag to see if the store supports incremental push or not"}, + {"name": "separateRealTimeTopicEnabled", "type": "boolean", "default": false, "doc": "Flag to see if the store supports separate real-time topic for incremental push."}, + {"name": "blobTransferEnabled", "type": "boolean", "default": false, "doc": "Flag to indicate if the blob transfer is allowed or not"}, + {"name": "useVersionLevelIncrementalPushEnabled", "type": "boolean", "default": false, "doc": "Flag to see if incrementalPushEnabled config at StoreVersion should be used. This is needed during migration of this config from Store level to Version level. We can deprecate this field later."}, + { + "name": "hybridConfig", + "type": [ + "null", + "com.linkedin.venice.systemstore.schemas.StoreHybridConfig" + ], + "default": null, + "doc": "Properties related to Hybrid Store behavior. If absent (null), then the store is not hybrid." + }, + {"name": "useVersionLevelHybridConfig", "type": "boolean", "default": false, "doc": "Flag to see if hybridConfig at StoreVersion should be used. This is needed during migration of this config from Store level to Version level. We can deprecate this field later."}, + {"name": "activeActiveReplicationEnabled", "type": "boolean", "default": false, "doc": "Whether or not active/active replication is enabled for hybrid stores; eventually this config will replace native replication flag, when all stores are on A/A"}, + {"name": "timestampMetadataVersionId", "type": "int", "default": -1, "doc": "The A/A timestamp metadata schema version ID that will be used to deserialize metadataPayload."}, + { + "name": "dataRecoveryConfig", + "type": [ + "null", + { + "name": "DataRecoveryConfig", + "type": "record", + "fields": [ + {"name": "dataRecoverySourceFabric", "type": "string", "doc": "The fabric name to be used as the source for data recovery."}, + {"name": "isDataRecoveryComplete", "type": "boolean", "doc": "Whether or not data recovery is complete."}, + {"name": "dataRecoverySourceVersionNumber", "type": "int", "default": 0, "doc": "The store version number to be used as the source for data recovery."} + ] + } + ], + "default": null, + "doc": "Properties related to data recovery mode behavior for this version. If absent (null), then the version never went go through data recovery." + }, + {"name": "deferVersionSwap", "type": "boolean", "default": false, "doc": "flag that informs venice controller to defer marking this version as the serving version after instances report ready to serve. This version must be marked manually as the current version in order to serve traffic from it."}, + { + "name": "views", + "doc": "A list of views which describe and configure a downstream view of a venice store.", + "type": { + "type": "map", + "java-key-class": "java.lang.String", + "avro.java.string": "String", + "values": "com.linkedin.venice.systemstore.schemas.StoreViewConfig" + }, + "default": {} + }, + {"name": "repushSourceVersion", "type": "int", "default": -1, "doc": "For store version created from repush, indicates the source store version its created from."} + ] + } + }, + "default": [] + }, + { + "name": "systemStores", + "doc": "This field is used to maintain a mapping between each type of system store and the corresponding distinct properties", + "type": { + "type": "map", + "values": { + "name": "SystemStoreProperties", + "type": "record", + "doc": "This type describes all the distinct properties", + "fields": [ + {"name": "largestUsedVersionNumber", "type": "int", "default": 0}, + {"name": "currentVersion", "type": "int", "default": 0}, + {"name": "latestVersionPromoteToCurrentTimestamp", "type": "long", "default": -1}, + {"name": "versions", "type": {"type": "array", "items": "com.linkedin.venice.systemstore.schemas.StoreVersion"}, "default": []} + ] + } + }, + "default": {} + }, + {"name": "storageNodeReadQuotaEnabled", "type": "boolean", "default": false, "doc": "Controls the storage node read quota enforcement for the given Venice store"}, + {"name": "blobTransferEnabled", "type": "boolean", "default": false, "doc": "Flag to indicate if the blob transfer is allowed or not"} + ] + } + ], + "default": null + }, + { + "name": "storeKeySchemas", + "doc": "", + "type": [ + "null", + { + "name": "StoreKeySchemas", + "doc": "This type describes the key schemas of the store", + "type": "record", + "fields": [ + { + "name": "keySchemaMap", + "doc": "A string to string map representing the mapping from id to key schema.", + "type": { + "type": "map", + "values": "string" + } + } + ] + } + ], + "default": null + }, + { + "name": "storeValueSchemas", + "doc": "", + "type": [ + "null", + { + "name": "StoreValueSchemas", + "doc": "This type describes the value schemas of the store.", + "type": "record", + "fields": [ + { + "name": "valueSchemaMap", + "doc": "A string to string map representing the mapping from schema id to value schema string. The value could be an empty string indicating the value schema is stored in another field.", + "type": { + "type": "map", + "values": "string" + } + } + ] + } + ], + "default": null + }, + { + "name": "storeValueSchema", + "doc": "", + "type": [ + "null", + { + "name": "StoreValueSchema", + "doc": "This type describes a single version of the value schema of the store.", + "type": "record", + "fields": [ + { + "name": "valueSchema", + "doc": "Store value schema string.", + "type": "string", + "default": "" + } + ] + } + ], + "default": null + }, + { + "name": "storeReplicaStatuses", + "doc": "This field describes the replica statuses per version per partition, and the mapping is 'host_port' -> 'replica status'", + "type": [ + "null", + { + "type": "map", + "values": { + "name": "StoreReplicaStatus", + "type": "record", + "doc": "This structure will contain all kinds of info related to one replica", + "fields": [ + {"name": "status", "type": "int", "doc": "replica status"} + ] + } + } + ], + "default": null + }, + { + "name": "storeValueSchemaIdsWrittenPerStoreVersion", + "doc": "This field described the set of value schemas id written by a store version.", + "type": [ + "null", + { + "name": "StoreValueSchemaIdsWrittenPerStoreVersion", + "doc": "This type describes value schema IDs written by the store version.", + "type": "array", + "items": "int" + } + ], + "default": null + }, + { + "name": "storeClusterConfig", + "doc": "This is the Zk's StoreConfig equivalent which contains various Venice cluster information", + "type": [ + "null", + { + "name": "StoreClusterConfig", + "doc": "This type describes the various Venice cluster information for a store", + "type": "record", + "fields": [ + {"name": "cluster", "type": "string", "default": "", "doc": "The Venice cluster of the store."}, + {"name": "deleting", "type": "boolean", "default": false, "doc": "Is the store undergoing deletion."}, + {"name": "migrationDestCluster", "type": ["null", "string"], "default": null, "doc": "The destination cluster for store migration"}, + {"name": "migrationSrcCluster", "type": ["null", "string"], "default": null, "doc": "The source cluster for store migration"}, + {"name": "storeName", "type": "string", "default": "", "doc": "The name of the store"} + ] + } + ], + "default": null + } + ] +} diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestVeniceHelixAdminWithSharedEnvironment.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestVeniceHelixAdminWithSharedEnvironment.java index 40dc8d98e6..1fabff7a5a 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestVeniceHelixAdminWithSharedEnvironment.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestVeniceHelixAdminWithSharedEnvironment.java @@ -1845,14 +1845,22 @@ public void testHybridStoreToBatchOnly() { veniceAdmin.updateStore( clusterName, storeName, - new UpdateStoreQueryParams().setHybridOffsetLagThreshold(1).setHybridRewindSeconds(1)); + new UpdateStoreQueryParams().setHybridOffsetLagThreshold(1) + .setHybridRewindSeconds(1) + .setSeparateRealTimeTopicEnabled(true)); veniceAdmin.incrementVersionIdempotent(clusterName, storeName, Version.guidBasedDummyPushId(), 1, 1); TestUtils.waitForNonDeterministicCompletion( TOTAL_TIMEOUT_FOR_SHORT_TEST_MS, TimeUnit.MILLISECONDS, () -> veniceAdmin.getCurrentVersion(clusterName, storeName) == 1); + Assert.assertTrue(veniceAdmin.getStore(clusterName, storeName).isHybrid()); + Assert.assertTrue(veniceAdmin.getStore(clusterName, storeName).isSeparateRealTimeTopicEnabled()); + Assert.assertTrue(veniceAdmin.getStore(clusterName, storeName).getVersion(1).isSeparateRealTimeTopicEnabled()); + String rtTopic = veniceAdmin.getRealTimeTopic(clusterName, storeName); + String incrementalPushRealTimeTopic = veniceAdmin.getSeparateRealTimeTopic(clusterName, storeName); Assert.assertFalse(veniceAdmin.isTopicTruncated(rtTopic)); + Assert.assertFalse(veniceAdmin.isTopicTruncated(incrementalPushRealTimeTopic)); veniceAdmin.updateStore( clusterName, storeName, @@ -1860,6 +1868,7 @@ public void testHybridStoreToBatchOnly() { .setHybridRewindSeconds(-1) .setHybridTimeLagThreshold(-1)); Assert.assertFalse(veniceAdmin.isTopicTruncated(rtTopic)); + Assert.assertFalse(veniceAdmin.isTopicTruncated(incrementalPushRealTimeTopic)); // Perform two new pushes and the RT should be deleted upon the completion of the new pushes. veniceAdmin.incrementVersionIdempotent(clusterName, storeName, Version.guidBasedDummyPushId(), 1, 1); TestUtils.waitForNonDeterministicCompletion( @@ -1872,6 +1881,7 @@ public void testHybridStoreToBatchOnly() { TimeUnit.MILLISECONDS, () -> veniceAdmin.getCurrentVersion(clusterName, storeName) == 3); Assert.assertTrue(veniceAdmin.isTopicTruncated(rtTopic)); + Assert.assertTrue(veniceAdmin.isTopicTruncated(incrementalPushRealTimeTopic)); } @Test(timeOut = TOTAL_TIMEOUT_FOR_LONG_TEST_MS) diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientMemoryLimitTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientMemoryLimitTest.java index 1d7f7a0959..b8ab2c13ef 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientMemoryLimitTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/DaVinciClientMemoryLimitTest.java @@ -257,7 +257,6 @@ public void testDaVinciMemoryLimitShouldFailLargeDataPush( } } - @Test(timeOut = TEST_TIMEOUT, dataProviderClass = DataProviderUtils.class, dataProvider = "Two-True-and-False") public void testDaVinciMemoryLimitShouldFailLargeDataPushAndResumeHybridStore( boolean ingestionIsolationEnabledInDaVinci, boolean useDaVinciSpecificExecutionStatusForError) throws Exception { diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveReplicationForIncPush.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveReplicationForIncPush.java index 430a28dff2..a4216b42b7 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveReplicationForIncPush.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestActiveActiveReplicationForIncPush.java @@ -8,6 +8,7 @@ import static com.linkedin.venice.ConfigKeys.SERVER_DATABASE_SYNC_BYTES_INTERNAL_FOR_DEFERRED_WRITE_MODE; import static com.linkedin.venice.ConfigKeys.SERVER_DEDICATED_CONSUMER_POOL_FOR_AA_WC_LEADER_ENABLED; import static com.linkedin.venice.ConfigKeys.SERVER_PROMOTION_TO_LEADER_REPLICA_DELAY_SECONDS; +import static com.linkedin.venice.pubsub.PubSubConstants.PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE; import static com.linkedin.venice.utils.TestUtils.assertCommand; import static com.linkedin.venice.utils.TestUtils.waitForNonDeterministicAssertion; import static com.linkedin.venice.utils.TestWriteUtils.getTempDataDirectory; @@ -33,15 +34,22 @@ import com.linkedin.venice.meta.Store; import com.linkedin.venice.meta.StoreInfo; import com.linkedin.venice.meta.Version; +import com.linkedin.venice.pubsub.PubSubTopicPartitionImpl; +import com.linkedin.venice.pubsub.PubSubTopicRepository; +import com.linkedin.venice.pubsub.api.PubSubTopicPartition; +import com.linkedin.venice.pubsub.manager.TopicManager; +import com.linkedin.venice.utils.DataProviderUtils; import com.linkedin.venice.utils.IntegrationTestPushUtils; import com.linkedin.venice.utils.TestUtils; import com.linkedin.venice.utils.TestWriteUtils; import com.linkedin.venice.utils.Time; import com.linkedin.venice.utils.Utils; import java.io.File; +import java.util.ArrayList; import java.util.List; import java.util.Optional; import java.util.Properties; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.function.Function; import org.apache.avro.Schema; @@ -63,9 +71,10 @@ public class TestActiveActiveReplicationForIncPush { private String[] clusterNames; private String parentRegionName; private String[] dcNames; - + private String clusterName; private List childDatacenters; private VeniceTwoLayerMultiRegionMultiClusterWrapper multiRegionMultiClusterWrapper; + private static final PubSubTopicRepository PUB_SUB_TOPIC_REPOSITORY = new PubSubTopicRepository(); PubSubBrokerWrapper veniceParentDefaultKafka; @@ -102,9 +111,9 @@ public void setUp() { false); childDatacenters = multiRegionMultiClusterWrapper.getChildRegions(); clusterNames = multiRegionMultiClusterWrapper.getClusterNames(); + clusterName = this.clusterNames[0]; parentRegionName = multiRegionMultiClusterWrapper.getParentRegionName(); dcNames = multiRegionMultiClusterWrapper.getChildRegionNames().toArray(new String[0]); - veniceParentDefaultKafka = multiRegionMultiClusterWrapper.getParentKafkaBrokerWrapper(); } @@ -117,9 +126,8 @@ public void cleanUp() { * The purpose of this test is to verify that incremental push with RT policy succeeds when A/A is enabled in all * regions. And also incremental push can push to the closes kafka cluster from the grid using the SOURCE_GRID_CONFIG. */ - @Test(timeOut = TEST_TIMEOUT) - public void testAAReplicationForIncrementalPushToRT() throws Exception { - String clusterName = this.clusterNames[0]; + @Test(timeOut = TEST_TIMEOUT, dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class) + public void testAAReplicationForIncrementalPushToRT(Boolean isSeparateRealTimeTopicEnabled) throws Exception { File inputDirBatch = getTempDataDirectory(); File inputDirInc1 = getTempDataDirectory(); File inputDirInc2 = getTempDataDirectory(); @@ -174,7 +182,8 @@ public void testAAReplicationForIncrementalPushToRT() throws Exception { .setPartitionCount(1) .setHybridOffsetLagThreshold(TEST_TIMEOUT / 2) .setHybridRewindSeconds(2L) - .setNativeReplicationSourceFabric("dc-2"); + .setNativeReplicationSourceFabric("dc-2") + .setSeparateRealTimeTopicEnabled(isSeparateRealTimeTopicEnabled); TestUtils.assertCommand(parentControllerClient.updateStore(storeName, updateStoreParams)); @@ -208,29 +217,97 @@ public void testAAReplicationForIncrementalPushToRT() throws Exception { job.run(); Assert.assertEquals(job.getKafkaUrl(), childDatacenters.get(2).getKafkaBrokerWrapper().getAddress()); } - // Run inc push with source fabric preference taking effect. - try (VenicePushJob job = new VenicePushJob("Test push job incremental with NR + A/A from dc-2", propsInc1)) { - job.run(); - Assert.assertEquals(job.getKafkaUrl(), childDatacenters.get(2).getKafkaBrokerWrapper().getAddress()); + if (isSeparateRealTimeTopicEnabled) { + verifyForSeparateIncrementalPushTopic(storeName, propsInc1, 2); + } else { + verifyForRealTimeIncrementalPushTopic(storeName, propsInc1, propsInc2); } + } + } - // Verify - for (int i = 0; i < childDatacenters.size(); i++) { - VeniceMultiClusterWrapper childDataCenter = childDatacenters.get(i); - // Verify the current version should be 1. - Version version = - childDataCenter.getRandomController().getVeniceAdmin().getStore(clusterName, storeName).getVersion(1); - Assert.assertNotNull(version, "Version 1 is not present for DC: " + dcNames[i]); - } - NativeReplicationTestUtils.verifyIncrementalPushData(childDatacenters, clusterName, storeName, 150, 2); + private void verifyForSeparateIncrementalPushTopic( + String storeName, + Properties propsInc1, + int dcIndexForSourceRegion) { + // Prepare TopicManagers + List topicManagers = new ArrayList<>(); + for (VeniceMultiClusterWrapper childDataCenter: childDatacenters) { + PubSubTopicRepository pubSubTopicRepository = + childDataCenter.getClusters().get(clusterNames[0]).getPubSubTopicRepository(); + topicManagers.add( + IntegrationTestPushUtils + .getTopicManagerRepo( + PUBSUB_OPERATION_TIMEOUT_MS_DEFAULT_VALUE, + 100, + 0l, + childDataCenter.getKafkaBrokerWrapper(), + pubSubTopicRepository) + .getLocalTopicManager()); + } + // Run inc push with source fabric preference taking effect. + PubSubTopicPartition separateRealTimeTopicPartition = new PubSubTopicPartitionImpl( + PUB_SUB_TOPIC_REPOSITORY.getTopic(Version.composeSeparateRealTimeTopic(storeName)), + 0); + PubSubTopicPartition realTimeTopicPartition = + new PubSubTopicPartitionImpl(PUB_SUB_TOPIC_REPOSITORY.getTopic(Version.composeRealTimeTopic(storeName)), 0); + try (VenicePushJob job = new VenicePushJob("Test push job incremental with NR + A/A from dc-2", propsInc1)) { + // TODO: Once server part separate topic ingestion logic is ready, we should avoid runAsync here and add extra + // check + CompletableFuture.runAsync(job::run); + TestUtils.waitForNonDeterministicAssertion(60, TimeUnit.SECONDS, true, () -> { + Assert.assertEquals( + job.getKafkaUrl(), + childDatacenters.get(dcIndexForSourceRegion).getKafkaBrokerWrapper().getAddress()); + for (int dcIndex = 0; dcIndex < childDatacenters.size(); dcIndex++) { + long separateTopicOffset = + topicManagers.get(dcIndex).getLatestOffsetWithRetries(separateRealTimeTopicPartition, 3); + long realTimeTopicOffset = topicManagers.get(dcIndex).getLatestOffsetWithRetries(realTimeTopicPartition, 3); + // Real-time topic will have heartbeat messages, so the offset will be non-zero but smaller than the record + // count. + // DC 2 separeate real-time topic should get enough data. + if (dcIndex == dcIndexForSourceRegion) { + Assert.assertTrue( + separateTopicOffset > TestWriteUtils.DEFAULT_USER_DATA_RECORD_COUNT, + "Records # is not enough: " + separateTopicOffset); + Assert.assertTrue( + realTimeTopicOffset < TestWriteUtils.DEFAULT_USER_DATA_RECORD_COUNT / 10, + "Records # is more than expected: " + realTimeTopicOffset); + } else { + assertEquals(separateTopicOffset, 0, "Records # is not enough: " + separateTopicOffset); + Assert.assertTrue( + realTimeTopicOffset < TestWriteUtils.DEFAULT_USER_DATA_RECORD_COUNT / 10, + "Records # is more than expected: " + realTimeTopicOffset); + } + } + }); + job.cancel(); + } + } - // Run another inc push with a different source fabric preference taking effect. - try (VenicePushJob job = new VenicePushJob("Test push job incremental with NR + A/A from dc-1", propsInc2)) { - job.run(); - Assert.assertEquals(job.getKafkaUrl(), childDatacenters.get(1).getKafkaBrokerWrapper().getAddress()); - } - NativeReplicationTestUtils.verifyIncrementalPushData(childDatacenters, clusterName, storeName, 200, 3); + private void verifyForRealTimeIncrementalPushTopic(String storeName, Properties propsInc1, Properties propsInc2) + throws Exception { + // Run inc push with source fabric preference taking effect. + try (VenicePushJob job = new VenicePushJob("Test push job incremental with NR + A/A from dc-2", propsInc1)) { + job.run(); + Assert.assertEquals(job.getKafkaUrl(), childDatacenters.get(2).getKafkaBrokerWrapper().getAddress()); + } + + // Verify + for (int i = 0; i < childDatacenters.size(); i++) { + VeniceMultiClusterWrapper childDataCenter = childDatacenters.get(i); + // Verify the current version should be 1. + Version version = + childDataCenter.getRandomController().getVeniceAdmin().getStore(clusterName, storeName).getVersion(1); + Assert.assertNotNull(version, "Version 1 is not present for DC: " + dcNames[i]); + } + NativeReplicationTestUtils.verifyIncrementalPushData(childDatacenters, clusterName, storeName, 150, 2); + + // Run another inc push with a different source fabric preference taking effect. + try (VenicePushJob job = new VenicePushJob("Test push job incremental with NR + A/A from dc-1", propsInc2)) { + job.run(); + Assert.assertEquals(job.getKafkaUrl(), childDatacenters.get(1).getKafkaBrokerWrapper().getAddress()); } + NativeReplicationTestUtils.verifyIncrementalPushData(childDatacenters, clusterName, storeName, 200, 3); } public static void verifyHybridAndIncPushConfig( diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/Admin.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/Admin.java index 6547752ad2..7269235874 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/Admin.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/Admin.java @@ -290,6 +290,8 @@ Version incrementVersionIdempotent( String getRealTimeTopic(String clusterName, String storeName); + String getSeparateRealTimeTopic(String clusterName, String storeName); + /** * Right now, it will return the latest version recorded in parent controller. There are a couple of edge cases. * 1. If a push fails in some colos, the version will be inconsistent among colos diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java index 50bdfaec33..e0c1881e05 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java @@ -99,6 +99,7 @@ import static com.linkedin.venice.ConfigKeys.ENABLE_PARTIAL_UPDATE_FOR_HYBRID_ACTIVE_ACTIVE_USER_STORES; import static com.linkedin.venice.ConfigKeys.ENABLE_PARTIAL_UPDATE_FOR_HYBRID_NON_ACTIVE_ACTIVE_USER_STORES; import static com.linkedin.venice.ConfigKeys.ENABLE_PARTITION_COUNT_ROUND_UP; +import static com.linkedin.venice.ConfigKeys.ENABLE_SEPARATE_REAL_TIME_TOPIC_FOR_STORE_WITH_INCREMENTAL_PUSH; import static com.linkedin.venice.ConfigKeys.ERROR_PARTITION_AUTO_RESET_LIMIT; import static com.linkedin.venice.ConfigKeys.ERROR_PARTITION_PROCESSING_CYCLE_DELAY; import static com.linkedin.venice.ConfigKeys.FATAL_DATA_VALIDATION_FAILURE_TOPIC_RETENTION_MS; @@ -439,6 +440,12 @@ public class VeniceControllerClusterConfig { */ private final boolean enabledIncrementalPushForHybridActiveActiveUserStores; + /** + * When the following option is enabled, new user hybrid store with incremental push enabled will automatically + * have separate real time topic enabled. + */ + private final boolean enabledSeparateRealTimeTopicForStoreWithIncrementalPush; + private final boolean enablePartialUpdateForHybridActiveActiveUserStores; private final boolean enablePartialUpdateForHybridNonActiveActiveUserStores; @@ -571,6 +578,8 @@ public VeniceControllerClusterConfig(VeniceProperties props) { this.controllerSchemaValidationEnabled = props.getBoolean(CONTROLLER_SCHEMA_VALIDATION_ENABLED, true); this.enabledIncrementalPushForHybridActiveActiveUserStores = props.getBoolean(ENABLE_INCREMENTAL_PUSH_FOR_HYBRID_ACTIVE_ACTIVE_USER_STORES, false); + this.enabledSeparateRealTimeTopicForStoreWithIncrementalPush = + props.getBoolean(ENABLE_SEPARATE_REAL_TIME_TOPIC_FOR_STORE_WITH_INCREMENTAL_PUSH, false); this.enablePartialUpdateForHybridActiveActiveUserStores = props.getBoolean(ENABLE_PARTIAL_UPDATE_FOR_HYBRID_ACTIVE_ACTIVE_USER_STORES, false); this.enablePartialUpdateForHybridNonActiveActiveUserStores = @@ -1141,6 +1150,10 @@ public boolean enabledIncrementalPushForHybridActiveActiveUserStores() { return enabledIncrementalPushForHybridActiveActiveUserStores; } + public boolean enabledSeparateRealTimeTopicForStoreWithIncrementalPush() { + return enabledSeparateRealTimeTopicForStoreWithIncrementalPush; + } + public boolean isEnablePartialUpdateForHybridActiveActiveUserStores() { return enablePartialUpdateForHybridActiveActiveUserStores; } 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 55f03147a6..fe8170c501 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 @@ -2740,6 +2740,17 @@ private Pair addVersion( // Note: do not enable RT compaction! Might make jobs in Online/Offline model stuck clusterConfig.getMinInSyncReplicasRealTimeTopics(), false); + if (version.isSeparateRealTimeTopicEnabled()) { + getTopicManager().createTopic( + pubSubTopicRepository.getTopic(Version.composeSeparateRealTimeTopic(storeName)), + numberOfPartitions, + clusterConfig.getKafkaReplicationFactorRTTopics(), + StoreUtils.getExpectedRetentionTimeInMs(store, store.getHybridStoreConfig()), + false, + // Note: do not enable RT compaction! Might make jobs in Online/Offline model stuck + clusterConfig.getMinInSyncReplicasRealTimeTopics(), + false); + } } else { // If real-time topic already exists, check whether its retention time is correct. PubSubTopicConfiguration pubSubTopicConfiguration = @@ -3103,14 +3114,29 @@ private Optional getVersionWithPushId(String clusterName, String storeN @Override public String getRealTimeTopic(String clusterName, String storeName) { checkControllerLeadershipFor(clusterName); - TopicManager topicManager = getTopicManager(); PubSubTopic realTimeTopic = pubSubTopicRepository.getTopic(Version.composeRealTimeTopic(storeName)); + ensureRealTimeTopicIsReady(clusterName, realTimeTopic); + return realTimeTopic.getName(); + } + + @Override + public String getSeparateRealTimeTopic(String clusterName, String storeName) { + checkControllerLeadershipFor(clusterName); + PubSubTopic incrementalPushRealTimeTopic = + pubSubTopicRepository.getTopic(Version.composeSeparateRealTimeTopic(storeName)); + ensureRealTimeTopicIsReady(clusterName, incrementalPushRealTimeTopic); + return incrementalPushRealTimeTopic.getName(); + } + + private void ensureRealTimeTopicIsReady(String clusterName, PubSubTopic realTimeTopic) { + TopicManager topicManager = getTopicManager(); + String storeName = realTimeTopic.getStoreName(); if (!topicManager.containsTopic(realTimeTopic)) { HelixVeniceClusterResources resources = getHelixVeniceClusterResources(clusterName); try (AutoCloseableLock ignore = resources.getClusterLockManager().createStoreWriteLock(storeName)) { // The topic might be created by another thread already. Check before creating. if (topicManager.containsTopic(realTimeTopic)) { - return realTimeTopic.getName(); + return; } ReadWriteStoreRepository repository = resources.getStoreMetadataRepository(); Store store = repository.getStore(storeName); @@ -3154,7 +3180,6 @@ public String getRealTimeTopic(String clusterName, String storeName) { storeName); } } - return realTimeTopic.getName(); } /** @@ -3513,6 +3538,15 @@ private void safeDeleteRTTopic(String clusterName, String storeName, Store store for (ControllerClient controllerClient: controllerClientMap.values()) { controllerClient.deleteKafkaTopic(rtTopicToDelete); } + // Check if there is incremental push topic exist. If yes, delete it and send out to let other controller to + // delete it. + String incrementalPushRTTopicToDelete = Version.composeSeparateRealTimeTopic(storeName); + if (getTopicManager().containsTopic(pubSubTopicRepository.getTopic(incrementalPushRTTopicToDelete))) { + truncateKafkaTopic(incrementalPushRTTopicToDelete); + for (ControllerClient controllerClient: controllerClientMap.values()) { + controllerClient.deleteKafkaTopic(incrementalPushRTTopicToDelete); + } + } } } @@ -4420,6 +4454,13 @@ void setIncrementalPushEnabled(String clusterName, String storeName, boolean inc }); } + void setSeparateRealTimeTopicEnabled(String clusterName, String storeName, boolean separateRealTimeTopicEnabled) { + storeMetadataUpdate(clusterName, storeName, store -> { + store.setSeparateRealTimeTopicEnabled(separateRealTimeTopicEnabled); + return store; + }); + } + private void setReplicationFactor(String clusterName, String storeName, int replicaFactor) { storeMetadataUpdate(clusterName, storeName, store -> { store.setReplicationFactor(replicaFactor); @@ -4699,6 +4740,7 @@ private void internalUpdateStore(String clusterName, String storeName, UpdateSto Optional batchGetLimit = params.getBatchGetLimit(); Optional numVersionsToPreserve = params.getNumVersionsToPreserve(); Optional incrementalPushEnabled = params.getIncrementalPushEnabled(); + Optional separateRealTimeTopicEnabled = params.getSeparateRealTimeTopicEnabled(); Optional storeMigration = params.getStoreMigration(); Optional writeComputationEnabled = params.getWriteComputationEnabled(); Optional replicationMetadataVersionID = params.getReplicationMetadataVersionID(); @@ -4887,6 +4929,10 @@ private void internalUpdateStore(String clusterName, String storeName, UpdateSto setIncrementalPushEnabled(clusterName, storeName, incrementalPushEnabled.get()); } + if (separateRealTimeTopicEnabled.isPresent()) { + setSeparateRealTimeTopicEnabled(clusterName, storeName, separateRealTimeTopicEnabled.get()); + } + if (replicationFactor.isPresent()) { setReplicationFactor(clusterName, storeName, replicationFactor.get()); } diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceParentHelixAdmin.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceParentHelixAdmin.java index 8610718213..aaf3fa1ed8 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceParentHelixAdmin.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceParentHelixAdmin.java @@ -49,6 +49,7 @@ import static com.linkedin.venice.controllerapi.ControllerApiConstants.REPLICATION_METADATA_PROTOCOL_VERSION_ID; import static com.linkedin.venice.controllerapi.ControllerApiConstants.REWIND_TIME_IN_SECONDS; import static com.linkedin.venice.controllerapi.ControllerApiConstants.RMD_CHUNKING_ENABLED; +import static com.linkedin.venice.controllerapi.ControllerApiConstants.SEPARATE_REAL_TIME_TOPIC_ENABLED; import static com.linkedin.venice.controllerapi.ControllerApiConstants.STORAGE_NODE_READ_QUOTA_ENABLED; import static com.linkedin.venice.controllerapi.ControllerApiConstants.STORAGE_QUOTA_IN_BYTE; import static com.linkedin.venice.controllerapi.ControllerApiConstants.STORE_MIGRATION; @@ -1704,6 +1705,11 @@ public String getRealTimeTopic(String clusterName, String storeName) { return getVeniceHelixAdmin().getRealTimeTopic(clusterName, storeName); } + @Override + public String getSeparateRealTimeTopic(String clusterName, String storeName) { + return getVeniceHelixAdmin().getSeparateRealTimeTopic(clusterName, storeName); + } + /** * A couple of extra checks are needed in parent controller * 1. check batch job statuses across child controllers. (We cannot only check the version status @@ -2229,6 +2235,7 @@ public void updateStore(String clusterName, String storeName, UpdateStoreQueryPa Optional batchGetLimit = params.getBatchGetLimit(); Optional numVersionsToPreserve = params.getNumVersionsToPreserve(); Optional incrementalPushEnabled = params.getIncrementalPushEnabled(); + Optional separateRealTimeTopicEnabled = params.getSeparateRealTimeTopicEnabled(); Optional storeMigration = params.getStoreMigration(); Optional writeComputationEnabled = params.getWriteComputationEnabled(); Optional replicationMetadataVersionID = params.getReplicationMetadataVersionID(); @@ -2460,6 +2467,13 @@ public void updateStore(String clusterName, String storeName, UpdateStoreQueryPa setStore.incrementalPushEnabled = true; updatedConfigsList.add(INCREMENTAL_PUSH_ENABLED); } + // Enable separate real-time topic automatically when incremental push is enabled and cluster config allows it. + if (setStore.incrementalPushEnabled + && controllerConfig.enabledSeparateRealTimeTopicForStoreWithIncrementalPush()) { + setStore.separateRealTimeTopicEnabled = true; + updatedConfigsList.add(SEPARATE_REAL_TIME_TOPIC_ENABLED); + } + // When turning off hybrid store, we will also turn off incremental store config. if (storeBeingConvertedToBatch && setStore.incrementalPushEnabled) { setStore.incrementalPushEnabled = false; @@ -2584,6 +2598,10 @@ public void updateStore(String clusterName, String storeName, UpdateStoreQueryPa .map(addToUpdatedConfigList(updatedConfigsList, BLOB_TRANSFER_ENABLED)) .orElseGet(currStore::isBlobTransferEnabled); + setStore.separateRealTimeTopicEnabled = + separateRealTimeTopicEnabled.map(addToUpdatedConfigList(updatedConfigsList, SEPARATE_REAL_TIME_TOPIC_ENABLED)) + .orElseGet(currStore::isSeparateRealTimeTopicEnabled); + // Check whether the passed param is valid or not if (latestSupersetSchemaId.isPresent()) { if (latestSupersetSchemaId.get() != SchemaData.INVALID_VALUE_SCHEMA_ID) { diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/kafka/consumer/AdminExecutionTask.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/kafka/consumer/AdminExecutionTask.java index d2e3094a58..f0188c257b 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/kafka/consumer/AdminExecutionTask.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/kafka/consumer/AdminExecutionTask.java @@ -486,6 +486,7 @@ private void handleSetStore(UpdateStore message) { .setBatchGetLimit(message.batchGetLimit) .setNumVersionsToPreserve(message.numVersionsToPreserve) .setIncrementalPushEnabled(message.incrementalPushEnabled) + .setSeparateRealTimeTopicEnabled(message.separateRealTimeTopicEnabled) .setStoreMigration(message.isMigrating) .setWriteComputationEnabled(message.writeComputationEnabled) .setReadComputationEnabled(message.readComputationEnabled) diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/CreateVersion.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/CreateVersion.java index 44afaa81fb..c22d6c3b3e 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/CreateVersion.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/server/CreateVersion.java @@ -306,7 +306,12 @@ public Route requestTopicForPushing(Admin admin) { responseTopic = Version.composeStreamReprocessingTopic(storeName, version.getNumber()); } else if (pushType.isIncremental()) { isTopicRT = true; - responseTopic = Version.composeRealTimeTopic(storeName); + if (version.isSeparateRealTimeTopicEnabled()) { + admin.getSeparateRealTimeTopic(clusterName, storeName); + responseTopic = Version.composeSeparateRealTimeTopic(storeName); + } else { + responseTopic = Version.composeRealTimeTopic(storeName); + } // disable amplificationFactor logic on real-time topic responseObject.setAmplificationFactor(1); diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/ingestion/control/RealTimeTopicSwitcher.java b/services/venice-controller/src/main/java/com/linkedin/venice/ingestion/control/RealTimeTopicSwitcher.java index f12180fb7d..f20209b903 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/ingestion/control/RealTimeTopicSwitcher.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/ingestion/control/RealTimeTopicSwitcher.java @@ -131,6 +131,8 @@ void ensurePreconditions( if (!hybridStoreConfig.isPresent()) { throw new VeniceException("Topic switching is only supported for Hybrid Stores."); } + Version version = + store.getVersion(Version.parseVersionFromKafkaTopicName(topicWhereToSendTheTopicSwitch.getName())); /** * TopicReplicator is used in child fabrics to create real-time (RT) topic when a child fabric * is ready to start buffer replay but RT topic doesn't exist. This scenario could happen for a @@ -146,35 +148,47 @@ void ensurePreconditions( * doesn't have any existing version or a correct storage quota, we cannot decide the partition * number for it. */ - if (!getTopicManager().containsTopicAndAllPartitionsAreOnline(srcTopicName)) { + createRealTimeTopicIfNeeded(store, version, srcTopicName, hybridStoreConfig.get()); + if (version != null && version.isSeparateRealTimeTopicEnabled()) { + PubSubTopic separateRealTimeTopic = + pubSubTopicRepository.getTopic(Version.composeSeparateRealTimeTopic(store.getName())); + createRealTimeTopicIfNeeded(store, version, separateRealTimeTopic, hybridStoreConfig.get()); + } + } + + void createRealTimeTopicIfNeeded( + Store store, + Version version, + PubSubTopic realTimeTopic, + HybridStoreConfig hybridStoreConfig) { + if (!getTopicManager().containsTopicAndAllPartitionsAreOnline(realTimeTopic)) { int partitionCount; - Version version = - store.getVersion(Version.parseVersionFromKafkaTopicName(topicWhereToSendTheTopicSwitch.getName())); if (version != null) { partitionCount = version.getPartitionCount(); } else { partitionCount = store.getPartitionCount(); } - int replicationFactor = srcTopicName.isRealTime() ? kafkaReplicationFactorForRTTopics : kafkaReplicationFactor; - Optional minISR = srcTopicName.isRealTime() ? minSyncReplicasForRTTopics : Optional.empty(); + int replicationFactor = realTimeTopic.isRealTime() ? kafkaReplicationFactorForRTTopics : kafkaReplicationFactor; + Optional minISR = realTimeTopic.isRealTime() ? minSyncReplicasForRTTopics : Optional.empty(); getTopicManager().createTopic( - srcTopicName, + realTimeTopic, partitionCount, replicationFactor, - StoreUtils.getExpectedRetentionTimeInMs(store, hybridStoreConfig.get()), - false, // Note: do not enable RT compaction! Might make jobs in Online/Offline model stuck + StoreUtils.getExpectedRetentionTimeInMs(store, hybridStoreConfig), + false, minISR, false); } else { /** * If real-time topic already exists, check whether its retention time is correct. */ - long topicRetentionTimeInMs = getTopicManager().getTopicRetention(srcTopicName); - long expectedRetentionTimeMs = StoreUtils.getExpectedRetentionTimeInMs(store, hybridStoreConfig.get()); + long topicRetentionTimeInMs = getTopicManager().getTopicRetention(realTimeTopic); + long expectedRetentionTimeMs = StoreUtils.getExpectedRetentionTimeInMs(store, hybridStoreConfig); if (topicRetentionTimeInMs != expectedRetentionTimeMs) { - getTopicManager().updateTopicRetention(srcTopicName, expectedRetentionTimeMs); + getTopicManager().updateTopicRetention(realTimeTopic, expectedRetentionTimeMs); } } + } long getRewindStartTime( diff --git a/services/venice-controller/src/main/resources/avro/AdminOperation/v81/AdminOperation.avsc b/services/venice-controller/src/main/resources/avro/AdminOperation/v81/AdminOperation.avsc new file mode 100644 index 0000000000..0987529807 --- /dev/null +++ b/services/venice-controller/src/main/resources/avro/AdminOperation/v81/AdminOperation.avsc @@ -0,0 +1,1134 @@ +{ + "name": "AdminOperation", + "namespace": "com.linkedin.venice.controller.kafka.protocol.admin", + "type": "record", + "fields": [ + { + "name": "operationType", + "doc": "0 => StoreCreation, 1 => ValueSchemaCreation, 2 => PauseStore, 3 => ResumeStore, 4 => KillOfflinePushJob, 5 => DisableStoreRead, 6 => EnableStoreRead, 7=> DeleteAllVersions, 8=> SetStoreOwner, 9=> SetStorePartitionCount, 10=> SetStoreCurrentVersion, 11=> UpdateStore, 12=> DeleteStore, 13=> DeleteOldVersion, 14=> MigrateStore, 15=> AbortMigration, 16=>AddVersion, 17=> DerivedSchemaCreation, 18=>SupersetSchemaCreation, 19=>EnableNativeReplicationForCluster, 20=>MetadataSchemaCreation, 21=>EnableActiveActiveReplicationForCluster, 25=>CreatePersona, 26=>DeletePersona, 27=>UpdatePersona, 28=>RollbackCurrentVersion, 29=>RollforwardCurrentVersion", + "type": "int" + }, { + "name": "executionId", + "doc": "ID of a command execution which is used to query the status of this command.", + "type": "long", + "default": 0 + }, { + "name": "payloadUnion", + "doc": "This contains the main payload of the admin operation", + "type": [ + { + "name": "StoreCreation", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + }, + { + "name": "owner", + "type": "string" + }, + { + "name": "keySchema", + "type": { + "type": "record", + "name": "SchemaMeta", + "fields": [ + {"name": "schemaType", "type": "int", "doc": "0 => Avro-1.4, and we can add more if necessary"}, + {"name": "definition", "type": "string"} + ] + } + }, + { + "name": "valueSchema", + "type": "SchemaMeta" + } + ] + }, + { + "name": "ValueSchemaCreation", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + }, + { + "name": "schema", + "type": "SchemaMeta" + }, + { + "name": "schemaId", + "type": "int" + }, + { + "name": "doUpdateSupersetSchemaID", + "type": "boolean", + "doc": "Whether this superset schema ID should be updated to be the value schema ID for this store.", + "default": false + } + ] + }, + { + "name": "PauseStore", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + } + ] + }, + { + "name": "ResumeStore", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + } + ] + }, + { + "name": "KillOfflinePushJob", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "kafkaTopic", + "type": "string" + } + ] + }, + { + "name": "DisableStoreRead", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + } + ] + }, + { + "name": "EnableStoreRead", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + } + ] + }, + { + "name": "DeleteAllVersions", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + } + ] + }, + { + "name": "SetStoreOwner", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + }, + { + "name": "owner", + "type": "string" + } + ] + }, + { + "name": "SetStorePartitionCount", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + }, + { + "name": "partitionNum", + "type": "int" + } + ] + }, + { + "name": "SetStoreCurrentVersion", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + }, + { + "name": "currentVersion", + "type": "int" + } + ] + }, + { + "name": "UpdateStore", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + }, + { + "name": "owner", + "type": "string" + }, + { + "name": "partitionNum", + "type": "int" + }, + { + "name": "currentVersion", + "type": "int" + }, + { + "name": "enableReads", + "type": "boolean" + }, + { + "name": "enableWrites", + "type": "boolean" + }, + { + "name": "storageQuotaInByte", + "type": "long", + "default": 21474836480 + }, + { + "name": "readQuotaInCU", + "type": "long", + "default": 1800 + }, + { + "name": "hybridStoreConfig", + "type": [ + "null", + { + "name": "HybridStoreConfigRecord", + "type": "record", + "fields": [ + { + "name": "rewindTimeInSeconds", + "type": "long" + }, + { + "name": "offsetLagThresholdToGoOnline", + "type": "long" + }, + { + "name": "producerTimestampLagThresholdToGoOnlineInSeconds", + "type": "long", + "default": -1 + }, + { + "name": "dataReplicationPolicy", + "doc": "Real-time Samza job data replication policy. Using int because Avro Enums are not evolvable 0 => NON_AGGREGATE, 1 => AGGREGATE, 2 => NONE, 3 => ACTIVE_ACTIVE", + "type": "int", + "default": 0 + }, + { + "name": "bufferReplayPolicy", + "type": "int", + "doc": "Policy that will be used during buffer replay. rewindTimeInSeconds defines the delta. 0 => REWIND_FROM_EOP (replay from 'EOP - rewindTimeInSeconds'), 1 => REWIND_FROM_SOP (replay from 'SOP - rewindTimeInSeconds')", + "default": 0 + } + ] + } + ], + "default": null + }, + { + "name": "accessControlled", + "type": "boolean", + "default": false + }, + { + "name": "compressionStrategy", + "doc": "Using int because Avro Enums are not evolvable", + "type": "int", + "default": 0 + }, + { + "name": "chunkingEnabled", + "type": "boolean", + "default": false + }, + { + "name": "rmdChunkingEnabled", + "type": "boolean", + "default": false + }, + { + "name": "singleGetRouterCacheEnabled", + "aliases": ["routerCacheEnabled"], + "type": "boolean", + "default": false + }, + { + "name": "batchGetRouterCacheEnabled", + "type": "boolean", + "default": false + }, + { + "name": "batchGetLimit", + "doc": "The max key number allowed in batch get request, and Venice will use cluster-level config if the limit (not positive) is not valid", + "type": "int", + "default": -1 + }, + { + "name": "numVersionsToPreserve", + "doc": "The max number of versions the store should preserve. Venice will use cluster-level config if the number is 0 here.", + "type": "int", + "default": 0 + }, + { + "name": "incrementalPushEnabled", + "doc": "a flag to see if the store supports incremental push or not", + "type": "boolean", + "default": false + }, + { + "name": "separateRealTimeTopicEnabled", + "doc": "Flag to see if the store supports separate real-time topic for incremental push.", + "type": "boolean", + "default": false + }, + { + "name": "isMigrating", + "doc": "Whether or not the store is in the process of migration", + "type": "boolean", + "default": false + }, + { + "name": "writeComputationEnabled", + "doc": "Whether write-path computation feature is enabled for this store", + "type": "boolean", + "default": false + }, + { + "name": "replicationMetadataVersionID", + "doc": "RMD (Replication metadata) version ID on the store-level. Default -1 means NOT_SET and the cluster-level RMD version ID should be used for stores.", + "type": "int", + "default": -1 + }, + { + "name": "readComputationEnabled", + "doc": "Whether read-path computation feature is enabled for this store", + "type": "boolean", + "default": false + }, + { + "name": "bootstrapToOnlineTimeoutInHours", + "doc": "Maximum number of hours allowed for the store to transition from bootstrap to online state", + "type": "int", + "default": 24 + }, + { + "name": "leaderFollowerModelEnabled", + "doc": "Whether or not to use leader follower state transition model for upcoming version", + "type": "boolean", + "default": false + }, + { + "name": "backupStrategy", + "doc": "Strategies to store backup versions.", + "type": "int", + "default": 0 + }, + { + "name": "clientDecompressionEnabled", + "type": "boolean", + "default": true + }, + { + "name": "schemaAutoRegisterFromPushJobEnabled", + "type": "boolean", + "default": false + }, + { + "name": "hybridStoreOverheadBypass", + "type": "boolean", + "default": false + }, + { + "name": "hybridStoreDiskQuotaEnabled", + "doc": "Whether or not to enable disk storage quota for a hybrid store", + "type": "boolean", + "default": false + }, + { + "name": "ETLStoreConfig", + "type": [ + "null", + { + "name": "ETLStoreConfigRecord", + "type": "record", + "fields": [ + { + "name": "etledUserProxyAccount", + "type": ["null", "string"] + }, + { + "name": "regularVersionETLEnabled", + "type": "boolean" + }, + { + "name": "futureVersionETLEnabled", + "type": "boolean" + } + ] + } + ], + "default": null + }, + { + "name": "partitionerConfig", + "type": [ + "null", + { + "name": "PartitionerConfigRecord", + "type": "record", + "fields": [ + { + "name": "partitionerClass", + "type": "string" + }, + { + "name": "partitionerParams", + "type": { + "type": "map", + "values": "string" + } + }, + { + "name": "amplificationFactor", + "type": "int" + } + ] + } + ], + "default": null + }, + { + "name": "nativeReplicationEnabled", + "type": "boolean", + "default": false + }, + { + "name": "pushStreamSourceAddress", + "type": ["null", "string"], + "default": null + }, + { + "name": "largestUsedVersionNumber", + "type": ["null", "int"], + "default": null + }, + { + "name": "incrementalPushPolicy", + "doc": "Incremental Push Policy to reconcile with real time pushes. Using int because Avro Enums are not evolvable 0 => PUSH_TO_VERSION_TOPIC, 1 => INCREMENTAL_PUSH_SAME_AS_REAL_TIME", + "type": "int", + "default": 0 + }, + { + "name": "backupVersionRetentionMs", + "type": "long", + "doc": "Backup version retention time after a new version is promoted to the current version, if not specified, Venice will use the configured retention as the default policy", + "default": -1 + }, + { + "name": "replicationFactor", + "doc": "number of replica each store version will have", + "type": "int", + "default": 3 + }, + { + "name": "migrationDuplicateStore", + "doc": "Whether or not the store is a duplicate store in the process of migration", + "type": "boolean", + "default": false + }, + { + "name": "nativeReplicationSourceFabric", + "doc": "The source fabric to be used when the store is running in Native Replication mode.", + "type": ["null", "string"], + "default": null + }, + { + "name": "activeActiveReplicationEnabled", + "doc": "A command option to enable/disable Active/Active replication feature for a store", + "type": "boolean", + "default": false + }, + { + "name": "disableMetaStore", + "doc": "An UpdateStore command option to disable the companion meta system store", + "type": "boolean", + "default": false + }, + { + "name": "disableDavinciPushStatusStore", + "doc": "An UpdateStore command option to disable the companion davinci push status store", + "type": "boolean", + "default": false + }, + { + "name": "applyTargetVersionFilterForIncPush", + "doc": "An UpdateStore command option to enable/disable applying the target version filter for incremental pushes", + "type": "boolean", + "default": false + }, + { + "name": "updatedConfigsList", + "doc": "The list that contains all updated configs by the UpdateStore command. Most of the fields in UpdateStore are not optional, and changing those fields to Optional (Union) is not a backward compatible change, so we have to add an addition array field to record all updated configs in parent controller.", + "type": { + "type": "array", + "items": "string" + }, + "default": [] + }, + { + "name": "replicateAllConfigs", + "doc": "A flag to indicate whether all store configs in parent cluster will be replicated to child clusters; true by default, so that existing UpdateStore messages in Admin topic will behave the same as before.", + "type": "boolean", + "default": true + }, + { + "name": "regionsFilter", + "doc": "A list of regions that will be impacted by the UpdateStore command", + "type": ["null", "string"], + "default": null + }, + { + "name": "storagePersona", + "doc": "The name of the StoragePersona to add to the store", + "type": ["null", "string"], + "default": null + }, + { + "name": "views", + "doc": "A map of views which describe and configure a downstream view of a venice store. Keys in this map are for convenience of managing configs.", + "type": ["null", + { + "type":"map", + "java-key-class": "java.lang.String", + "avro.java.string": "String", + "values": { + "name": "StoreViewConfigRecord", + "type": "record", + "doc": "A configuration for a particular view. This config should inform Venice leaders how to transform and transmit data to destination views.", + "fields": [ + { + "name": "viewClassName", + "type": "string", + "doc": "This informs what kind of view we are materializing. This then informs what kind of parameters are passed to parse this input. This is expected to be a fully formed class path name for materialization.", + "default": "" + }, + { + "name": "viewParameters", + "doc": "Optional parameters to be passed to the given view config.", + "type": ["null", + { + "type": "map", + "java-key-class": "java.lang.String", + "avro.java.string": "String", + "values": { "type": "string", "avro.java.string": "String" } + } + ], + "default": null + } + ] + } + }], + "default": null + }, + { + "name": "latestSuperSetValueSchemaId", + "doc": "The schema id for the latest superset schema", + "type" : "int", + "default": -1 + }, + { + "name": "storageNodeReadQuotaEnabled", + "doc": "Whether storage node read quota is enabled for this store", + "type": "boolean", + "default": false + }, + { + "name": "minCompactionLagSeconds", + "doc": "Store-level version topic min compaction lag", + "type": "long", + "default": -1 + }, + { + "name": "maxCompactionLagSeconds", + "doc": "Store-level version topic max compaction lag", + "type": "long", + "default": -1 + }, + { + "name": "maxRecordSizeBytes", + "doc": "Store-level maximum size of any record in bytes for batch push jobs", + "type": "int", + "default": -1 + }, + { + "name": "maxNearlineRecordSizeBytes", + "doc": "Store-level maximum size of any record in bytes for nearline jobs with partial updates", + "type": "int", + "default": -1 + }, + { + "name": "unusedSchemaDeletionEnabled", + "doc": "Whether unused schema deletion is enabled or not.", + "type": "boolean", + "default": false + }, + { + "name": "blobTransferEnabled", + "doc": "Flag to indicate if the blob transfer is allowed or not", + "type": "boolean", + "default": false + } + ] + }, + { + "name": "DeleteStore", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + }, + { + "name": "largestUsedVersionNumber", + "type": "int" + } + ] + }, + { + "name": "DeleteOldVersion", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + }, + { + "name": "versionNum", + "type": "int" + } + ] + }, + { + "name": "MigrateStore", + "type": "record", + "fields": [ + { + "name": "srcClusterName", + "type": "string" + }, + { + "name": "destClusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + } + ] + }, + { + "name": "AbortMigration", + "type": "record", + "fields": [ + { + "name": "srcClusterName", + "type": "string" + }, + { + "name": "destClusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + } + ] + }, + { + "name": "AddVersion", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + }, + { + "name": "pushJobId", + "type": "string" + }, + { + "name": "versionNum", + "type": "int" + }, + { + "name": "numberOfPartitions", + "type": "int" + }, + { + "name": "pushType", + "doc": "The push type of the new version, 0 => BATCH, 1 => STREAM_REPROCESSING. Previous add version messages will default to BATCH and this is a safe because they were created when BATCH was the only version type", + "type": "int", + "default": 0 + }, + { + "name": "pushStreamSourceAddress", + "type": ["null", "string"], + "default": null + }, + { + "name": "rewindTimeInSecondsOverride", + "doc": "The overridable rewind time config for this specific version of a hybrid store, and if it is not specified, the new version will use the store-level rewind time config", + "type": "long", + "default": -1 + }, + { + "name": "timestampMetadataVersionId", + "doc": "The A/A metadata schema version ID that will be used to deserialize metadataPayload.", + "type": "int", + "default": -1 + }, + { + "name": "versionSwapDeferred", + "doc": "Indicates if swapping this version to current version after push completion should be initiated or not", + "type": "boolean", + "default": false + }, + { + "name": "targetedRegions", + "doc": "The list of regions that is separated by comma for targeted region push. If set, this admin message should only be consumed by the targeted regions", + "type": [ + "null", + { + "type": "array", + "items": "string" + } + ], + "default": null + }, + { + "name": "repushSourceVersion", + "doc": "Indicates the source version from which a repush version is created", + "type": "int", + "default": -1 + } + ] + }, + { + "name": "DerivedSchemaCreation", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + }, + { + "name": "schema", + "type": "SchemaMeta" + }, + { + "name": "valueSchemaId", + "type": "int" + }, + { + "name": "derivedSchemaId", + "type": "int" + } + ] + }, + { + "name": "SupersetSchemaCreation", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + }, + { + "name": "valueSchema", + "type": "SchemaMeta" + }, + { + "name": "valueSchemaId", + "type": "int" + }, + { + "name": "supersetSchema", + "type": "SchemaMeta" + }, + { + "name": "supersetSchemaId", + "type": "int" + } + ] + }, + { + "name": "ConfigureNativeReplicationForCluster", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeType", + "type": "string" + }, + { + "name": "enabled", + "type": "boolean" + }, + { + "name": "nativeReplicationSourceRegion", + "doc": "The source region to be used when the store is running in Native Replication mode.", + "type": ["null", "string"], + "default": null + }, + { + "name": "regionsFilter", + "type": ["null", "string"], + "default": null + } + ] + }, + { + "name": "MetadataSchemaCreation", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + }, + { + "name": "valueSchemaId", + "type": "int" + }, + { + "name": "metadataSchema", + "type": "SchemaMeta" + }, + { + "name": "timestampMetadataVersionId", + "type": "int", + "aliases": ["metadataVersionId"], + "default": -1 + } + ] + }, + { + "name": "ConfigureActiveActiveReplicationForCluster", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeType", + "type": "string" + }, + { + "name": "enabled", + "type": "boolean" + }, + { + "name": "regionsFilter", + "type": ["null", "string"], + "default": null + } + ] + }, { + "name": "ConfigureIncrementalPushForCluster", + "doc": "A command to migrate all incremental push stores in a cluster to a specific incremental push policy.", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "incrementalPushPolicyToFilter", + "doc": "If this batch update command is trying to configure existing incremental push store type, their incremental push policy should also match this filter before the batch update command applies any change to them. Default value is -1, meaning there is no filter.", + "type": "int", + "default": -1 + }, + { + "name": "incrementalPushPolicyToApply", + "doc": "This field will determine what incremental push policy will be applied to the selected stores. Default value is 1, which is the INCREMENTAL_PUSH_SAME_AS_REAL_TIME policy", + "type": "int", + "default": 1 + }, + { + "name": "regionsFilter", + "type": ["null", "string"], + "default": null + } + ] + }, { + "name": "MetaSystemStoreAutoCreationValidation", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + } + ] + }, { + "name": "PushStatusSystemStoreAutoCreationValidation", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + } + ] + }, { + "name": "CreateStoragePersona", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "name", + "type": "string" + }, + { + "name": "quotaNumber", + "type": "long" + }, + { + "name": "storesToEnforce", + "type": { + "type": "array", + "items": "string", + "default": [] + } + }, + { + "name": "owners", + "type": { + "type": "array", + "items": "string", + "default": [] + } + } + ] + }, { + "name": "DeleteStoragePersona", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "name", + "type": "string" + } + ] + }, { + "name": "UpdateStoragePersona", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, { + "name": "name", + "type": "string" + }, { + "name": "quotaNumber", + "type": ["null","long"], + "default": null + }, { + "name": "storesToEnforce", + "type": [ + "null", + { + "type": "array", + "items": "string" + } + ], + "default": null + }, { + "name": "owners", + "type": [ + "null", + { + "type": "array", + "items": "string" + } + ], + "default": null + } + ] + }, + { + "name": "DeleteUnusedValueSchemas", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + }, + { + "name": "schemaIds", + "type": { + "type": "array", + "items": "int", + "default": [] + } + } + ] + }, + { + "name": "RollbackCurrentVersion", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + }, + { + "name": "regionsFilter", + "doc": "A list of regions that will be impacted by the RollbackCurrentVersion command", + "type": ["null", "string"], + "default": null + } + ] + }, + { + "name": "RollForwardCurrentVersion", + "type": "record", + "fields": [ + { + "name": "clusterName", + "type": "string" + }, + { + "name": "storeName", + "type": "string" + }, + { + "name": "regionsFilter", + "doc": "A list of regions that will be impacted by the RollForwardCurrentVersion command", + "type": ["null", "string"], + "default": null + } + ] + } + ] + } + ] +} diff --git a/services/venice-controller/src/test/java/com/linkedin/venice/controller/server/CreateVersionTest.java b/services/venice-controller/src/test/java/com/linkedin/venice/controller/server/CreateVersionTest.java index a0776fb802..8220f19f0c 100644 --- a/services/venice-controller/src/test/java/com/linkedin/venice/controller/server/CreateVersionTest.java +++ b/services/venice-controller/src/test/java/com/linkedin/venice/controller/server/CreateVersionTest.java @@ -145,8 +145,8 @@ public void testCreateVersionWithACL(boolean checkReadMethod) throws Exception { } } - @Test(description = "requestTopicForPushing should return an RT topic when store is hybrid and inc-push is enabled") - public void testRequestTopicForHybridIncPushEnabled() throws Exception { + @Test(dataProvider = "True-and-False", dataProviderClass = DataProviderUtils.class, description = "requestTopicForPushing should return an RT topic when store is hybrid and inc-push is enabled") + public void testRequestTopicForHybridIncPushEnabled(boolean isSeparateTopicEnabled) throws Exception { doReturn(true).when(admin).whetherEnableBatchPushFromAdmin(STORE_NAME); doCallRealMethod().when(request).queryParamOrDefault(any(), any()); doReturn(true).when(accessClient).isAllowlistUsers(certificate, STORE_NAME, HTTP_GET); @@ -156,6 +156,7 @@ public void testRequestTopicForHybridIncPushEnabled() throws Exception { doReturn(store).when(admin).getStore(CLUSTER_NAME, STORE_NAME); Version version = new VersionImpl(STORE_NAME, 1, JOB_ID); + version.setSeparateRealTimeTopicEnabled(isSeparateTopicEnabled); doReturn(version).when(admin) .incrementVersionIdempotent( CLUSTER_NAME, @@ -186,7 +187,11 @@ public void testRequestTopicForHybridIncPushEnabled() throws Exception { assertNotNull(result); VersionCreationResponse versionCreateResponse = OBJECT_MAPPER.readValue(result.toString(), VersionCreationResponse.class); - assertEquals(versionCreateResponse.getKafkaTopic(), "test_store_rt"); + if (isSeparateTopicEnabled) { + assertEquals(versionCreateResponse.getKafkaTopic(), Version.composeSeparateRealTimeTopic(STORE_NAME)); + } else { + assertEquals(versionCreateResponse.getKafkaTopic(), Version.composeRealTimeTopic(STORE_NAME)); + } } // A store should never end up in the state where inc-push is enabled but hybrid configs are not set, nevertheless From 25d90a5436545058955860d38b8a461426d4a78a Mon Sep 17 00:00:00 2001 From: Jingyan Li Date: Wed, 25 Sep 2024 13:41:39 -0700 Subject: [PATCH 12/16] [server][dvc] transfer offsetRecord and storeVersionState during blob transfer (#1187) --- .../com/linkedin/davinci/DaVinciBackend.java | 7 +- .../blobtransfer/BlobTransferManager.java | 2 +- .../BlobTransferPartitionMetadata.java | 66 +++ .../blobtransfer/BlobTransferPayload.java | 10 +- .../blobtransfer/BlobTransferUtil.java | 34 +- .../blobtransfer/BlobTransferUtils.java | 27 ++ .../NettyP2PBlobTransferManager.java | 8 +- .../blobtransfer/P2PBlobTransferManager.java | 2 +- .../client/MetadataAggregator.java | 30 ++ .../client/NettyFileTransferClient.java | 16 +- .../client/P2PFileTransferClientHandler.java | 23 +- .../client/P2PMetadataTransferHandler.java | 117 ++++++ .../BlobTransferNettyChannelInitializer.java | 9 +- .../server/P2PBlobTransferService.java | 7 +- .../server/P2PFileTransferServerHandler.java | 79 +++- .../helix/HelixParticipationService.java | 2 +- .../ingestion/DefaultIngestionBackend.java | 2 +- .../ingestion/IsolatedIngestionBackend.java | 2 +- .../TestNettyP2PBlobTransferManager.java | 44 +- .../TestP2PFileTransferClientHandler.java | 378 ++++++++++++++++++ .../TestP2PFileTransferServerHandler.java | 124 +++++- .../DefaultIngestionBackendTest.java | 2 +- .../blobtransfer/BlobTransferUtils.java | 6 - .../TestP2PFileTransferClientHandler.java | 206 ---------- .../linkedin/venice/server/VeniceServer.java | 7 +- 25 files changed, 936 insertions(+), 274 deletions(-) rename {internal/venice-common/src/main/java/com/linkedin/venice => clients/da-vinci-client/src/main/java/com/linkedin/davinci}/blobtransfer/BlobTransferManager.java (97%) create mode 100644 clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/BlobTransferPartitionMetadata.java rename {internal/venice-common/src/main/java/com/linkedin/venice => clients/da-vinci-client/src/main/java/com/linkedin/davinci}/blobtransfer/BlobTransferPayload.java (85%) rename {internal/venice-common/src/main/java/com/linkedin/venice => clients/da-vinci-client/src/main/java/com/linkedin/davinci}/blobtransfer/BlobTransferUtil.java (76%) create mode 100644 clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/BlobTransferUtils.java rename {internal/venice-common/src/main/java/com/linkedin/venice => clients/da-vinci-client/src/main/java/com/linkedin/davinci}/blobtransfer/NettyP2PBlobTransferManager.java (90%) rename {internal/venice-common/src/main/java/com/linkedin/venice => clients/da-vinci-client/src/main/java/com/linkedin/davinci}/blobtransfer/P2PBlobTransferManager.java (95%) create mode 100644 clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/client/MetadataAggregator.java rename {internal/venice-common/src/main/java/com/linkedin/venice => clients/da-vinci-client/src/main/java/com/linkedin/davinci}/blobtransfer/client/NettyFileTransferClient.java (76%) rename {internal/venice-common/src/main/java/com/linkedin/venice => clients/da-vinci-client/src/main/java/com/linkedin/davinci}/blobtransfer/client/P2PFileTransferClientHandler.java (90%) create mode 100644 clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/client/P2PMetadataTransferHandler.java rename {internal/venice-common/src/main/java/com/linkedin/venice => clients/da-vinci-client/src/main/java/com/linkedin/davinci}/blobtransfer/server/BlobTransferNettyChannelInitializer.java (74%) rename {internal/venice-common/src/main/java/com/linkedin/venice => clients/da-vinci-client/src/main/java/com/linkedin/davinci}/blobtransfer/server/P2PBlobTransferService.java (91%) rename {internal/venice-common/src/main/java/com/linkedin/venice => clients/da-vinci-client/src/main/java/com/linkedin/davinci}/blobtransfer/server/P2PFileTransferServerHandler.java (67%) rename {internal/venice-common/src/test/java/com/linkedin/venice => clients/da-vinci-client/src/test/java/com/linkedin/davinci}/blobtransfer/TestNettyP2PBlobTransferManager.java (74%) create mode 100644 clients/da-vinci-client/src/test/java/com/linkedin/davinci/blobtransfer/TestP2PFileTransferClientHandler.java rename {internal/venice-common/src/test/java/com/linkedin/venice/blobtransfer/server => clients/da-vinci-client/src/test/java/com/linkedin/davinci/blobtransfer}/TestP2PFileTransferServerHandler.java (51%) delete mode 100644 internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/BlobTransferUtils.java delete mode 100644 internal/venice-common/src/test/java/com/linkedin/venice/blobtransfer/client/TestP2PFileTransferClientHandler.java diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/DaVinciBackend.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/DaVinciBackend.java index 29922abdc8..8acc5d3535 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/DaVinciBackend.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/DaVinciBackend.java @@ -7,6 +7,8 @@ import static com.linkedin.venice.pushmonitor.ExecutionStatus.DVC_INGESTION_ERROR_OTHER; import static java.lang.Thread.currentThread; +import com.linkedin.davinci.blobtransfer.BlobTransferManager; +import com.linkedin.davinci.blobtransfer.BlobTransferUtil; import com.linkedin.davinci.client.DaVinciRecordTransformer; import com.linkedin.davinci.compression.StorageEngineBackedCompressorFactory; import com.linkedin.davinci.config.StoreBackendConfig; @@ -30,8 +32,6 @@ import com.linkedin.davinci.store.AbstractStorageEngine; import com.linkedin.davinci.store.cache.backend.ObjectCacheBackend; import com.linkedin.davinci.store.cache.backend.ObjectCacheConfig; -import com.linkedin.venice.blobtransfer.BlobTransferManager; -import com.linkedin.venice.blobtransfer.BlobTransferUtil; import com.linkedin.venice.client.exceptions.VeniceClientException; import com.linkedin.venice.client.schema.StoreSchemaFetcher; import com.linkedin.venice.client.store.ClientConfig; @@ -293,7 +293,8 @@ public DaVinciBackend( configLoader.getVeniceServerConfig().getDvcP2pBlobTransferServerPort(), configLoader.getVeniceServerConfig().getDvcP2pBlobTransferClientPort(), configLoader.getVeniceServerConfig().getRocksDBPath(), - clientConfig); + clientConfig, + storageMetadataService); } else { blobTransferManager = null; } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/BlobTransferManager.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/BlobTransferManager.java similarity index 97% rename from internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/BlobTransferManager.java rename to clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/BlobTransferManager.java index 95935fa76b..1d3b7e6ac9 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/BlobTransferManager.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/BlobTransferManager.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.blobtransfer; +package com.linkedin.davinci.blobtransfer; import com.linkedin.venice.annotation.Experimental; import com.linkedin.venice.exceptions.VenicePeersNotFoundException; diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/BlobTransferPartitionMetadata.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/BlobTransferPartitionMetadata.java new file mode 100644 index 0000000000..f0ea249a97 --- /dev/null +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/BlobTransferPartitionMetadata.java @@ -0,0 +1,66 @@ +package com.linkedin.davinci.blobtransfer; + +import com.fasterxml.jackson.annotation.JsonProperty; + + +/** + * This class is the metadata of a partition in the blob transfer client + */ +public class BlobTransferPartitionMetadata { + public String topicName; + public int partitionId; + public java.nio.ByteBuffer offsetRecord; + public java.nio.ByteBuffer storeVersionState; + + public BlobTransferPartitionMetadata() { + } + + public BlobTransferPartitionMetadata( + @JsonProperty("topicName") String topicName, + @JsonProperty("partitionId") int partitionId, + @JsonProperty("offsetRecord") java.nio.ByteBuffer offsetRecord, + @JsonProperty("storeVersionState") java.nio.ByteBuffer storeVersionState) { + this.topicName = topicName; + this.partitionId = partitionId; + this.offsetRecord = offsetRecord; + this.storeVersionState = storeVersionState; + } + + public String getTopicName() { + return topicName; + } + + public void setTopicName(String topicName) { + this.topicName = topicName; + } + + public int getPartitionId() { + return partitionId; + } + + public void setPartitionId(int partitionId) { + this.partitionId = partitionId; + } + + public java.nio.ByteBuffer getOffsetRecord() { + return offsetRecord; + } + + public void setOffsetRecord(java.nio.ByteBuffer offsetRecord) { + this.offsetRecord = offsetRecord; + } + + public void setStoreVersionState(java.nio.ByteBuffer storeVersionState) { + this.storeVersionState = storeVersionState; + } + + public java.nio.ByteBuffer getStoreVersionState() { + return storeVersionState; + } + + @Override + public String toString() { + return "BlobTransferPartitionMetadata {" + " topicName='" + topicName + ", partitionId=" + partitionId + + ", offsetRecord=" + offsetRecord + ", storeVersionState=" + storeVersionState + " }"; + } +} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/BlobTransferPayload.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/BlobTransferPayload.java similarity index 85% rename from internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/BlobTransferPayload.java rename to clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/BlobTransferPayload.java index df9c76517a..d39b9a29a9 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/BlobTransferPayload.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/BlobTransferPayload.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.blobtransfer; +package com.linkedin.davinci.blobtransfer; import static com.linkedin.venice.store.rocksdb.RocksDBUtils.composePartitionDbDir; import static com.linkedin.venice.store.rocksdb.RocksDBUtils.composeSnapshotDir; @@ -31,4 +31,12 @@ public String getSnapshotDir() { public String getFullResourceName() { return Utils.getReplicaId(topicName, partition); } + + public String getTopicName() { + return topicName; + } + + public int getPartition() { + return partition; + } } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/BlobTransferUtil.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/BlobTransferUtil.java similarity index 76% rename from internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/BlobTransferUtil.java rename to clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/BlobTransferUtil.java index fae636887a..7a18d402da 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/BlobTransferUtil.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/BlobTransferUtil.java @@ -1,9 +1,12 @@ -package com.linkedin.venice.blobtransfer; +package com.linkedin.davinci.blobtransfer; import static com.linkedin.venice.client.store.ClientFactory.getTransportClient; -import com.linkedin.venice.blobtransfer.client.NettyFileTransferClient; -import com.linkedin.venice.blobtransfer.server.P2PBlobTransferService; +import com.linkedin.davinci.blobtransfer.client.NettyFileTransferClient; +import com.linkedin.davinci.blobtransfer.server.P2PBlobTransferService; +import com.linkedin.davinci.storage.StorageMetadataService; +import com.linkedin.venice.blobtransfer.DaVinciBlobFinder; +import com.linkedin.venice.blobtransfer.ServerBlobFinder; import com.linkedin.venice.client.store.AbstractAvroStoreClient; import com.linkedin.venice.client.store.AvroGenericStoreClientImpl; import com.linkedin.venice.client.store.ClientConfig; @@ -21,27 +24,35 @@ public class BlobTransferUtil { * @param p2pTransferPort, the port used by the P2P transfer server and client * @param baseDir, the base directory of the underlying storage * @param clientConfig, the client config to start up a transport client + * @param storageMetadataService, the storage metadata service * @return the blob transfer manager * @throws Exception */ public static BlobTransferManager getP2PBlobTransferManagerForDVCAndStart( int p2pTransferPort, String baseDir, - ClientConfig clientConfig) { - return getP2PBlobTransferManagerForDVCAndStart(p2pTransferPort, p2pTransferPort, baseDir, clientConfig); + ClientConfig clientConfig, + StorageMetadataService storageMetadataService) { + return getP2PBlobTransferManagerForDVCAndStart( + p2pTransferPort, + p2pTransferPort, + baseDir, + clientConfig, + storageMetadataService); } public static BlobTransferManager getP2PBlobTransferManagerForDVCAndStart( int p2pTransferServerPort, int p2pTransferClientPort, String baseDir, - ClientConfig clientConfig) { + ClientConfig clientConfig, + StorageMetadataService storageMetadataService) { try { AbstractAvroStoreClient storeClient = new AvroGenericStoreClientImpl<>(getTransportClient(clientConfig), false, clientConfig); BlobTransferManager manager = new NettyP2PBlobTransferManager( - new P2PBlobTransferService(p2pTransferServerPort, baseDir), - new NettyFileTransferClient(p2pTransferClientPort, baseDir), + new P2PBlobTransferService(p2pTransferServerPort, baseDir, storageMetadataService), + new NettyFileTransferClient(p2pTransferClientPort, baseDir, storageMetadataService), new DaVinciBlobFinder(storeClient)); manager.start(); return manager; @@ -64,11 +75,12 @@ public static BlobTransferManager getP2PBlobTransferManagerForServerAndSta int p2pTransferServerPort, int p2pTransferClientPort, String baseDir, - CompletableFuture customizedViewFuture) { + CompletableFuture customizedViewFuture, + StorageMetadataService storageMetadataService) { try { BlobTransferManager manager = new NettyP2PBlobTransferManager( - new P2PBlobTransferService(p2pTransferServerPort, baseDir), - new NettyFileTransferClient(p2pTransferClientPort, baseDir), + new P2PBlobTransferService(p2pTransferServerPort, baseDir, storageMetadataService), + new NettyFileTransferClient(p2pTransferClientPort, baseDir, storageMetadataService), new ServerBlobFinder(customizedViewFuture)); manager.start(); return manager; diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/BlobTransferUtils.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/BlobTransferUtils.java new file mode 100644 index 0000000000..4a568a42b4 --- /dev/null +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/BlobTransferUtils.java @@ -0,0 +1,27 @@ +package com.linkedin.davinci.blobtransfer; + +import io.netty.handler.codec.http.HttpResponse; + + +public class BlobTransferUtils { + public static final String BLOB_TRANSFER_STATUS = "X-Blob-Transfer-Status"; + public static final String BLOB_TRANSFER_COMPLETED = "Completed"; + public static final String BLOB_TRANSFER_TYPE = "X-Blob-Transfer-Type"; + + public enum BlobTransferType { + FILE, METADATA + } + + /** + * Check if the HttpResponse message is for metadata. + * @param msg the HttpResponse message + * @return true if the message is a metadata message, false otherwise + */ + public static boolean isMetadataMessage(HttpResponse msg) { + String metadataHeader = msg.headers().get(BlobTransferUtils.BLOB_TRANSFER_TYPE); + if (metadataHeader == null) { + return false; + } + return metadataHeader.equals(BlobTransferUtils.BlobTransferType.METADATA.name()); + } +} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/NettyP2PBlobTransferManager.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/NettyP2PBlobTransferManager.java similarity index 90% rename from internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/NettyP2PBlobTransferManager.java rename to clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/NettyP2PBlobTransferManager.java index 8dedd6337c..15465dab49 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/NettyP2PBlobTransferManager.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/NettyP2PBlobTransferManager.java @@ -1,7 +1,9 @@ -package com.linkedin.venice.blobtransfer; +package com.linkedin.davinci.blobtransfer; -import com.linkedin.venice.blobtransfer.client.NettyFileTransferClient; -import com.linkedin.venice.blobtransfer.server.P2PBlobTransferService; +import com.linkedin.davinci.blobtransfer.client.NettyFileTransferClient; +import com.linkedin.davinci.blobtransfer.server.P2PBlobTransferService; +import com.linkedin.venice.blobtransfer.BlobFinder; +import com.linkedin.venice.blobtransfer.BlobPeersDiscoveryResponse; import com.linkedin.venice.exceptions.VenicePeersNotFoundException; import java.io.InputStream; import java.util.List; diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/P2PBlobTransferManager.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/P2PBlobTransferManager.java similarity index 95% rename from internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/P2PBlobTransferManager.java rename to clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/P2PBlobTransferManager.java index edf75cd033..16432b533e 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/P2PBlobTransferManager.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/P2PBlobTransferManager.java @@ -1,4 +1,4 @@ -package com.linkedin.venice.blobtransfer; +package com.linkedin.davinci.blobtransfer; import com.linkedin.venice.exceptions.VeniceException; import java.util.concurrent.CompletionStage; diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/client/MetadataAggregator.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/client/MetadataAggregator.java new file mode 100644 index 0000000000..02fbf7080a --- /dev/null +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/client/MetadataAggregator.java @@ -0,0 +1,30 @@ +package com.linkedin.davinci.blobtransfer.client; + +import com.linkedin.davinci.blobtransfer.BlobTransferUtils; +import io.netty.handler.codec.http.HttpObjectAggregator; +import io.netty.handler.codec.http.HttpResponse; + + +/** + * MetadataAggregator is a custom HttpObjectAggregator that + * only aggregated HttpResponse messages for metadata. + */ +public class MetadataAggregator extends HttpObjectAggregator { + public MetadataAggregator(int maxContentLength) { + super(maxContentLength); + } + + @Override + public boolean acceptInboundMessage(Object msg) throws Exception { + if (msg instanceof HttpResponse) { + HttpResponse httpMessage = (HttpResponse) msg; + // only accept metadata messages to be aggregated + if (BlobTransferUtils.isMetadataMessage(httpMessage)) { + return super.acceptInboundMessage(msg); + } else { + return false; + } + } + return super.acceptInboundMessage(msg); + } +} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/client/NettyFileTransferClient.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/client/NettyFileTransferClient.java similarity index 76% rename from internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/client/NettyFileTransferClient.java rename to clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/client/NettyFileTransferClient.java index 4b88d3f635..5b1ec31fa6 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/client/NettyFileTransferClient.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/client/NettyFileTransferClient.java @@ -1,5 +1,6 @@ -package com.linkedin.venice.blobtransfer.client; +package com.linkedin.davinci.blobtransfer.client; +import com.linkedin.davinci.storage.StorageMetadataService; import io.netty.bootstrap.Bootstrap; import io.netty.channel.Channel; import io.netty.channel.ChannelInitializer; @@ -22,16 +23,20 @@ public class NettyFileTransferClient { private static final Logger LOGGER = LogManager.getLogger(NettyFileTransferClient.class); + private static final int MAX_METADATA_CONTENT_LENGTH = 1024 * 1024 * 100; EventLoopGroup workerGroup; Bootstrap clientBootstrap; private final String baseDir; private final int serverPort; + private StorageMetadataService storageMetadataService; // TODO 1: move tunable configs to a config class // TODO 2: consider either increasing worker threads or have a dedicated thread pool to handle requests. - public NettyFileTransferClient(int serverPort, String baseDir) { + public NettyFileTransferClient(int serverPort, String baseDir, StorageMetadataService storageMetadataService) { this.baseDir = baseDir; this.serverPort = serverPort; + this.storageMetadataService = storageMetadataService; + clientBootstrap = new Bootstrap(); workerGroup = new NioEventLoopGroup(); clientBootstrap.group(workerGroup); @@ -50,8 +55,13 @@ public CompletionStage get(String host, String storeName, int versi // Connects to the remote host try { Channel ch = clientBootstrap.connect(host, serverPort).sync().channel(); + // Request to get the blob file and metadata // Attach the file handler to the pipeline - ch.pipeline().addLast(new P2PFileTransferClientHandler(baseDir, inputStream, storeName, version, partition)); + // Attach the metadata handler to the pipeline + ch.pipeline() + .addLast(new MetadataAggregator(MAX_METADATA_CONTENT_LENGTH)) + .addLast(new P2PFileTransferClientHandler(baseDir, inputStream, storeName, version, partition)) + .addLast(new P2PMetadataTransferHandler(storageMetadataService, baseDir, storeName, version, partition)); // Send a GET request ch.writeAndFlush(prepareRequest(storeName, version, partition)); } catch (Exception e) { diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/client/P2PFileTransferClientHandler.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/client/P2PFileTransferClientHandler.java similarity index 90% rename from internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/client/P2PFileTransferClientHandler.java rename to clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/client/P2PFileTransferClientHandler.java index f69ce1355f..60a2779310 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/client/P2PFileTransferClientHandler.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/client/P2PFileTransferClientHandler.java @@ -1,9 +1,10 @@ -package com.linkedin.venice.blobtransfer.client; +package com.linkedin.davinci.blobtransfer.client; -import static com.linkedin.venice.blobtransfer.BlobTransferUtils.BLOB_TRANSFER_COMPLETED; -import static com.linkedin.venice.blobtransfer.BlobTransferUtils.BLOB_TRANSFER_STATUS; +import static com.linkedin.davinci.blobtransfer.BlobTransferUtils.BLOB_TRANSFER_COMPLETED; +import static com.linkedin.davinci.blobtransfer.BlobTransferUtils.BLOB_TRANSFER_STATUS; -import com.linkedin.venice.blobtransfer.BlobTransferPayload; +import com.linkedin.davinci.blobtransfer.BlobTransferPayload; +import com.linkedin.davinci.blobtransfer.BlobTransferUtils; import com.linkedin.venice.exceptions.VeniceException; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufInputStream; @@ -15,6 +16,7 @@ import io.netty.handler.codec.http.HttpObject; import io.netty.handler.codec.http.HttpResponse; import io.netty.handler.codec.http.HttpResponseStatus; +import io.netty.util.ReferenceCountUtil; import java.io.InputStream; import java.nio.channels.Channels; import java.nio.channels.FileChannel; @@ -63,6 +65,14 @@ protected void channelRead0(ChannelHandlerContext ctx, HttpObject msg) throws Ex if (!response.status().equals(HttpResponseStatus.OK)) { throw new VeniceException("Failed to fetch file from remote peer. Response: " + response.status()); } + // redirect the message to the next handler if it's a metadata transfer + boolean isMetadataMessage = BlobTransferUtils.isMetadataMessage(response); + if (isMetadataMessage) { + ReferenceCountUtil.retain(msg); + ctx.fireChannelRead(msg); + return; + } + // Already end of transfer. Close the connection and completes the future if (response.headers().get(BLOB_TRANSFER_STATUS) != null && response.headers().get(BLOB_TRANSFER_STATUS).equals(BLOB_TRANSFER_COMPLETED)) { @@ -72,11 +82,10 @@ protected void channelRead0(ChannelHandlerContext ctx, HttpObject msg) throws Ex // Parse the file name this.fileName = getFileNameFromHeader(response); - if (this.fileName == null) { throw new VeniceException("No file name specified in the response for " + payload.getFullResourceName()); } - LOGGER.info("Starting blob transfer for file: {}", fileName); + LOGGER.debug("Starting blob transfer for file: {}", fileName); this.fileContentLength = Long.parseLong(response.headers().get(HttpHeaderNames.CONTENT_LENGTH)); // Create the directory @@ -91,8 +100,6 @@ protected void channelRead0(ChannelHandlerContext ctx, HttpObject msg) throws Ex HttpContent content = (HttpContent) msg; ByteBuf byteBuf = content.content(); if (byteBuf.readableBytes() == 0) { - // hit EMPTY_LAST_CONTENT, it indicates the end of all file transfer. - // Skip it since it's not going to be used return; } // defensive check diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/client/P2PMetadataTransferHandler.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/client/P2PMetadataTransferHandler.java new file mode 100644 index 0000000000..32b6ca8318 --- /dev/null +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/client/P2PMetadataTransferHandler.java @@ -0,0 +1,117 @@ +package com.linkedin.davinci.blobtransfer.client; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.linkedin.davinci.blobtransfer.BlobTransferPartitionMetadata; +import com.linkedin.davinci.blobtransfer.BlobTransferPayload; +import com.linkedin.davinci.storage.StorageMetadataService; +import com.linkedin.venice.exceptions.VeniceException; +import com.linkedin.venice.kafka.protocol.state.PartitionState; +import com.linkedin.venice.kafka.protocol.state.StoreVersionState; +import com.linkedin.venice.offsets.OffsetRecord; +import com.linkedin.venice.serialization.avro.AvroProtocolDefinition; +import com.linkedin.venice.serialization.avro.InternalAvroSpecificSerializer; +import com.linkedin.venice.utils.ObjectMapperFactory; +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.handler.codec.http.FullHttpResponse; +import io.netty.handler.codec.http.HttpResponseStatus; +import java.io.IOException; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + + +/** + * The client-side Netty handler to process responses for P2P metadata transfer. + */ +public class P2PMetadataTransferHandler extends SimpleChannelInboundHandler { + private static final Logger LOGGER = LogManager.getLogger(P2PMetadataTransferHandler.class); + private static final InternalAvroSpecificSerializer storeVersionStateSerializer = + AvroProtocolDefinition.STORE_VERSION_STATE.getSerializer(); + private static final InternalAvroSpecificSerializer partitionStateSerializer = + AvroProtocolDefinition.PARTITION_STATE.getSerializer(); + + private final BlobTransferPayload payload; + private BlobTransferPartitionMetadata metadata; + private StorageMetadataService storageMetadataService; + + public P2PMetadataTransferHandler( + StorageMetadataService storageMetadataService, + String baseDir, + String storeName, + int version, + int partition) { + this.storageMetadataService = storageMetadataService; + this.payload = new BlobTransferPayload(baseDir, storeName, version, partition); + } + + @Override + protected void channelRead0(ChannelHandlerContext ctx, FullHttpResponse msg) throws Exception { + LOGGER.debug("Received metadata response from remote peer for topic {}", payload.getTopicName()); + + processMetadata(msg); + + LOGGER.debug( + "Successfully processed metadata response from remote peer for topic {} with metadata {}", + payload.getTopicName(), + metadata); + } + + private void processMetadata(FullHttpResponse msg) throws IOException { + if (!msg.status().equals(HttpResponseStatus.OK)) { + throw new VeniceException("Failed to fetch metadata from remote peer. Response: " + msg.status()); + } + + ByteBuf content = msg.content(); + byte[] metadataBytes = new byte[content.readableBytes()]; + ObjectMapper objectMapper = ObjectMapperFactory.getInstance(); + content.readBytes(metadataBytes); + BlobTransferPartitionMetadata transferredMetadata = + objectMapper.readValue(metadataBytes, BlobTransferPartitionMetadata.class); + if (transferredMetadata == null) { + throw new VeniceException("No transferPartitionMetadata received for topic " + payload.getTopicName()); + } + + metadata = transferredMetadata; + updateStorePartitionMetadata(storageMetadataService, metadata); + } + + /** + * Sync the offset record and store version state + * @param storageMetadataService storage metadata service + * @param transferredPartitionMetadata transferred partition metadata + */ + public void updateStorePartitionMetadata( + StorageMetadataService storageMetadataService, + BlobTransferPartitionMetadata transferredPartitionMetadata) { + LOGGER.debug("Start updating store partition metadata for topic {}. ", transferredPartitionMetadata.topicName); + // update the offset record in storage service + storageMetadataService.put( + transferredPartitionMetadata.topicName, + transferredPartitionMetadata.partitionId, + new OffsetRecord(transferredPartitionMetadata.offsetRecord.array(), partitionStateSerializer)); + // update the metadata SVS + updateStorageVersionState(storageMetadataService, transferredPartitionMetadata); + } + + private void updateStorageVersionState( + StorageMetadataService storageMetadataService, + BlobTransferPartitionMetadata transferPartitionMetadata) { + StoreVersionState storeVersionState = storeVersionStateSerializer + .deserialize(transferPartitionMetadata.topicName, transferPartitionMetadata.storeVersionState.array()); + storageMetadataService.computeStoreVersionState(transferPartitionMetadata.topicName, previousStoreVersionState -> { + if (previousStoreVersionState != null) { + previousStoreVersionState.topicSwitch = storeVersionState.topicSwitch; + return previousStoreVersionState; + } else { + return storeVersionState; + } + }); + } + + @VisibleForTesting + public BlobTransferPartitionMetadata getMetadata() { + return metadata; + } +} diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/server/BlobTransferNettyChannelInitializer.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/server/BlobTransferNettyChannelInitializer.java similarity index 74% rename from internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/server/BlobTransferNettyChannelInitializer.java rename to clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/server/BlobTransferNettyChannelInitializer.java index b5c575e6a3..01c951d920 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/server/BlobTransferNettyChannelInitializer.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/server/BlobTransferNettyChannelInitializer.java @@ -1,5 +1,6 @@ -package com.linkedin.venice.blobtransfer.server; +package com.linkedin.davinci.blobtransfer.server; +import com.linkedin.davinci.storage.StorageMetadataService; import io.netty.channel.ChannelInitializer; import io.netty.channel.ChannelPipeline; import io.netty.channel.socket.SocketChannel; @@ -11,9 +12,11 @@ public class BlobTransferNettyChannelInitializer extends ChannelInitializer { private final String baseDir; + private StorageMetadataService storageMetadataService; - public BlobTransferNettyChannelInitializer(String baseDir) { + public BlobTransferNettyChannelInitializer(String baseDir, StorageMetadataService storageMetadataService) { this.baseDir = baseDir; + this.storageMetadataService = storageMetadataService; } @Override @@ -29,6 +32,6 @@ protected void initChannel(SocketChannel ch) throws Exception { // for safe writing of chunks for responses .addLast("chunker", new ChunkedWriteHandler()) // for handling p2p file transfer - .addLast("p2pFileTransferHandler", new P2PFileTransferServerHandler(baseDir)); + .addLast("p2pFileTransferHandler", new P2PFileTransferServerHandler(baseDir, storageMetadataService)); } } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/server/P2PBlobTransferService.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/server/P2PBlobTransferService.java similarity index 91% rename from internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/server/P2PBlobTransferService.java rename to clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/server/P2PBlobTransferService.java index f1f61f083c..d7257579fe 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/server/P2PBlobTransferService.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/server/P2PBlobTransferService.java @@ -1,5 +1,6 @@ -package com.linkedin.venice.blobtransfer.server; +package com.linkedin.davinci.blobtransfer.server; +import com.linkedin.davinci.storage.StorageMetadataService; import com.linkedin.venice.service.AbstractVeniceService; import io.netty.bootstrap.ServerBootstrap; import io.netty.channel.ChannelFuture; @@ -30,7 +31,7 @@ public class P2PBlobTransferService extends AbstractVeniceService { // TODO 5: add compression support // TODO 6: consider either increasing worker threads or have a dedicated thread pool to handle requests. - public P2PBlobTransferService(int port, String baseDir) { + public P2PBlobTransferService(int port, String baseDir, StorageMetadataService storageMetadataService) { this.port = port; this.serverBootstrap = new ServerBootstrap(); @@ -47,7 +48,7 @@ public P2PBlobTransferService(int port, String baseDir) { serverBootstrap.group(bossGroup, workerGroup) .channel(socketChannelClass) - .childHandler(new BlobTransferNettyChannelInitializer(baseDir)) + .childHandler(new BlobTransferNettyChannelInitializer(baseDir, storageMetadataService)) .option(ChannelOption.SO_BACKLOG, 1000) .option(ChannelOption.SO_REUSEADDR, true) .childOption(ChannelOption.SO_KEEPALIVE, true) diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/server/P2PFileTransferServerHandler.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/server/P2PFileTransferServerHandler.java similarity index 67% rename from internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/server/P2PFileTransferServerHandler.java rename to clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/server/P2PFileTransferServerHandler.java index d09e33e77d..b89ddbe595 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/server/P2PFileTransferServerHandler.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/blobtransfer/server/P2PFileTransferServerHandler.java @@ -1,12 +1,25 @@ -package com.linkedin.venice.blobtransfer.server; +package com.linkedin.davinci.blobtransfer.server; -import static com.linkedin.venice.blobtransfer.BlobTransferUtils.BLOB_TRANSFER_COMPLETED; -import static com.linkedin.venice.blobtransfer.BlobTransferUtils.BLOB_TRANSFER_STATUS; +import static com.linkedin.davinci.blobtransfer.BlobTransferUtils.BLOB_TRANSFER_COMPLETED; +import static com.linkedin.davinci.blobtransfer.BlobTransferUtils.BLOB_TRANSFER_STATUS; +import static com.linkedin.davinci.blobtransfer.BlobTransferUtils.BLOB_TRANSFER_TYPE; +import static com.linkedin.davinci.blobtransfer.BlobTransferUtils.BlobTransferType; import static com.linkedin.venice.utils.NettyUtils.setupResponseAndFlush; +import static io.netty.handler.codec.http.HttpHeaderValues.APPLICATION_JSON; import static io.netty.handler.codec.http.HttpHeaderValues.APPLICATION_OCTET_STREAM; -import com.linkedin.venice.blobtransfer.BlobTransferPayload; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.linkedin.davinci.blobtransfer.BlobTransferPartitionMetadata; +import com.linkedin.davinci.blobtransfer.BlobTransferPayload; +import com.linkedin.davinci.storage.StorageMetadataService; +import com.linkedin.venice.kafka.protocol.state.StoreVersionState; +import com.linkedin.venice.offsets.OffsetRecord; import com.linkedin.venice.request.RequestHelper; +import com.linkedin.venice.serialization.avro.AvroProtocolDefinition; +import com.linkedin.venice.serialization.avro.InternalAvroSpecificSerializer; +import com.linkedin.venice.utils.ObjectMapperFactory; +import io.netty.buffer.Unpooled; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelHandler; import io.netty.channel.ChannelHandlerContext; @@ -15,6 +28,7 @@ import io.netty.handler.codec.http.DefaultFullHttpResponse; import io.netty.handler.codec.http.DefaultHttpResponse; import io.netty.handler.codec.http.FullHttpRequest; +import io.netty.handler.codec.http.FullHttpResponse; import io.netty.handler.codec.http.HttpChunkedInput; import io.netty.handler.codec.http.HttpHeaderNames; import io.netty.handler.codec.http.HttpMethod; @@ -30,6 +44,7 @@ import java.io.IOException; import java.io.RandomAccessFile; import java.net.URI; +import java.nio.ByteBuffer; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -41,11 +56,15 @@ @ChannelHandler.Sharable public class P2PFileTransferServerHandler extends SimpleChannelInboundHandler { private static final Logger LOGGER = LogManager.getLogger(P2PFileTransferServerHandler.class); + private static final InternalAvroSpecificSerializer storeVersionStateSerializer = + AvroProtocolDefinition.STORE_VERSION_STATE.getSerializer(); private boolean useZeroCopy = false; private final String baseDir; + private StorageMetadataService storageMetadataService; - public P2PFileTransferServerHandler(String baseDir) { + public P2PFileTransferServerHandler(String baseDir, StorageMetadataService storageMetadataService) { this.baseDir = baseDir; + this.storageMetadataService = storageMetadataService; } @Override @@ -113,6 +132,9 @@ protected void channelRead0(ChannelHandlerContext ctx, FullHttpRequest httpReque sendFile(file, ctx); } + // transfer metadata + sendMetadata(blobTransferRequest, ctx); + // end of transfer HttpResponse endOfTransfer = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); endOfTransfer.headers().set(BLOB_TRANSFER_STATUS, BLOB_TRANSFER_COMPLETED); @@ -159,6 +181,7 @@ private void sendFile(File file, ChannelHandlerContext ctx) throws IOException { response.headers().set(HttpHeaderNames.CONTENT_LENGTH, length); response.headers().set(HttpHeaderNames.CONTENT_TYPE, APPLICATION_OCTET_STREAM); response.headers().set(HttpHeaderNames.CONTENT_DISPOSITION, "attachment; filename=\"" + file.getName() + "\""); + response.headers().set(BLOB_TRANSFER_TYPE, BlobTransferType.FILE); ctx.write(response); @@ -187,9 +210,53 @@ private void sendFile(File file, ChannelHandlerContext ctx) throws IOException { }); } + public void sendMetadata(BlobTransferPayload blobTransferRequest, ChannelHandlerContext ctx) + throws JsonProcessingException { + // prepare metadata + BlobTransferPartitionMetadata metadata = null; + try { + StoreVersionState storeVersionState = + storageMetadataService.getStoreVersionState(blobTransferRequest.getTopicName()); + java.nio.ByteBuffer storeVersionStateByte = + ByteBuffer.wrap(storeVersionStateSerializer.serialize(blobTransferRequest.getTopicName(), storeVersionState)); + + OffsetRecord offsetRecord = + storageMetadataService.getLastOffset(blobTransferRequest.getTopicName(), blobTransferRequest.getPartition()); + java.nio.ByteBuffer offsetRecordByte = ByteBuffer.wrap(offsetRecord.toBytes()); + + metadata = new BlobTransferPartitionMetadata( + blobTransferRequest.getTopicName(), + blobTransferRequest.getPartition(), + offsetRecordByte, + storeVersionStateByte); + } catch (Exception e) { + byte[] errBody = ("Failed to get metadata for " + blobTransferRequest.getTopicName()).getBytes(); + setupResponseAndFlush(HttpResponseStatus.INTERNAL_SERVER_ERROR, errBody, false, ctx); + } + + ObjectMapper objectMapper = ObjectMapperFactory.getInstance(); + String jsonMetadata = objectMapper.writeValueAsString(metadata); + byte[] metadataBytes = jsonMetadata.getBytes(); + + // send metadata + FullHttpResponse metadataResponse = + new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK, Unpooled.wrappedBuffer(metadataBytes)); + metadataResponse.headers().set(HttpHeaderNames.CONTENT_LENGTH, metadataBytes.length); + metadataResponse.headers().set(HttpHeaderNames.CONTENT_TYPE, APPLICATION_JSON); + metadataResponse.headers().set(BLOB_TRANSFER_TYPE, BlobTransferType.METADATA); + + ctx.writeAndFlush(metadataResponse).addListener(future -> { + if (future.isSuccess()) { + LOGGER.debug("Metadata for {} sent successfully", blobTransferRequest.getTopicName()); + } else { + LOGGER.error("Failed to send metadata for {}", blobTransferRequest.getTopicName()); + } + }); + } + /** * Parse the URI to locate the blob - * @param request + * @param uri * @return */ private BlobTransferPayload parseBlobTransferPayload(URI uri) throws IllegalArgumentException { diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/helix/HelixParticipationService.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/helix/HelixParticipationService.java index 3f04f17dc1..a0bbd04a9e 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/helix/HelixParticipationService.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/helix/HelixParticipationService.java @@ -1,5 +1,6 @@ package com.linkedin.davinci.helix; +import com.linkedin.davinci.blobtransfer.BlobTransferManager; import com.linkedin.davinci.config.VeniceConfigLoader; import com.linkedin.davinci.config.VeniceServerConfig; import com.linkedin.davinci.config.VeniceStoreVersionConfig; @@ -13,7 +14,6 @@ import com.linkedin.davinci.stats.ingestion.heartbeat.HeartbeatMonitoringService; import com.linkedin.davinci.storage.StorageMetadataService; import com.linkedin.davinci.storage.StorageService; -import com.linkedin.venice.blobtransfer.BlobTransferManager; import com.linkedin.venice.common.VeniceSystemStoreType; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.helix.HelixAdapterSerializer; diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/ingestion/DefaultIngestionBackend.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/ingestion/DefaultIngestionBackend.java index 50b45a9a2c..7f57536048 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/ingestion/DefaultIngestionBackend.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/ingestion/DefaultIngestionBackend.java @@ -1,5 +1,6 @@ package com.linkedin.davinci.ingestion; +import com.linkedin.davinci.blobtransfer.BlobTransferManager; import com.linkedin.davinci.config.VeniceServerConfig; import com.linkedin.davinci.config.VeniceStoreVersionConfig; import com.linkedin.davinci.kafka.consumer.KafkaStoreIngestionService; @@ -7,7 +8,6 @@ import com.linkedin.davinci.storage.StorageMetadataService; import com.linkedin.davinci.storage.StorageService; import com.linkedin.davinci.store.AbstractStorageEngine; -import com.linkedin.venice.blobtransfer.BlobTransferManager; import com.linkedin.venice.exceptions.VenicePeersNotFoundException; import com.linkedin.venice.kafka.protocol.state.StoreVersionState; import com.linkedin.venice.meta.Store; diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/ingestion/IsolatedIngestionBackend.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/ingestion/IsolatedIngestionBackend.java index d5db77ccdf..f9716f6a52 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/ingestion/IsolatedIngestionBackend.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/ingestion/IsolatedIngestionBackend.java @@ -4,6 +4,7 @@ import static com.linkedin.venice.ingestion.protocol.enums.IngestionCommandType.START_CONSUMPTION; import static com.linkedin.venice.ingestion.protocol.enums.IngestionCommandType.STOP_CONSUMPTION; +import com.linkedin.davinci.blobtransfer.BlobTransferManager; import com.linkedin.davinci.config.VeniceConfigLoader; import com.linkedin.davinci.config.VeniceStoreVersionConfig; import com.linkedin.davinci.ingestion.main.MainIngestionMonitorService; @@ -15,7 +16,6 @@ import com.linkedin.davinci.notifier.VeniceNotifier; import com.linkedin.davinci.storage.StorageMetadataService; import com.linkedin.davinci.storage.StorageService; -import com.linkedin.venice.blobtransfer.BlobTransferManager; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.ingestion.protocol.enums.IngestionCommandType; import com.linkedin.venice.ingestion.protocol.enums.IngestionComponentType; diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/blobtransfer/TestNettyP2PBlobTransferManager.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/blobtransfer/TestNettyP2PBlobTransferManager.java similarity index 74% rename from internal/venice-common/src/test/java/com/linkedin/venice/blobtransfer/TestNettyP2PBlobTransferManager.java rename to clients/da-vinci-client/src/test/java/com/linkedin/davinci/blobtransfer/TestNettyP2PBlobTransferManager.java index 3b35878689..16032372eb 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/blobtransfer/TestNettyP2PBlobTransferManager.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/blobtransfer/TestNettyP2PBlobTransferManager.java @@ -1,13 +1,21 @@ -package com.linkedin.venice.blobtransfer; +package com.linkedin.davinci.blobtransfer; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; -import com.linkedin.venice.blobtransfer.client.NettyFileTransferClient; -import com.linkedin.venice.blobtransfer.server.P2PBlobTransferService; +import com.linkedin.davinci.blobtransfer.client.NettyFileTransferClient; +import com.linkedin.davinci.blobtransfer.server.P2PBlobTransferService; +import com.linkedin.davinci.storage.StorageMetadataService; +import com.linkedin.venice.blobtransfer.BlobFinder; +import com.linkedin.venice.blobtransfer.BlobPeersDiscoveryResponse; import com.linkedin.venice.exceptions.VenicePeersNotFoundException; +import com.linkedin.venice.kafka.protocol.state.PartitionState; +import com.linkedin.venice.kafka.protocol.state.StoreVersionState; +import com.linkedin.venice.offsets.OffsetRecord; +import com.linkedin.venice.serialization.avro.AvroProtocolDefinition; +import com.linkedin.venice.serialization.avro.InternalAvroSpecificSerializer; import com.linkedin.venice.store.rocksdb.RocksDBUtils; import com.linkedin.venice.utils.TestUtils; import java.io.IOException; @@ -23,6 +31,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -34,6 +43,7 @@ public class TestNettyP2PBlobTransferManager { P2PBlobTransferService server; NettyFileTransferClient client; NettyP2PBlobTransferManager manager; + StorageMetadataService storageMetadataService; Path tmpSnapshotDir; Path tmpPartitionDir; String TEST_STORE = "test_store"; @@ -48,9 +58,11 @@ public void setUp() throws Exception { tmpSnapshotDir = Files.createTempDirectory(TMP_SNAPSHOT_DIR); tmpPartitionDir = Files.createTempDirectory(TMP_PARTITION_DIR); // intentionally use different directories for snapshot and partition so that we can verify the file transfer - server = new P2PBlobTransferService(port, tmpSnapshotDir.toString()); - client = new NettyFileTransferClient(port, tmpPartitionDir.toString()); + storageMetadataService = mock(StorageMetadataService.class); + server = new P2PBlobTransferService(port, tmpSnapshotDir.toString(), storageMetadataService); + client = new NettyFileTransferClient(port, tmpPartitionDir.toString(), storageMetadataService); finder = mock(BlobFinder.class); + manager = new NettyP2PBlobTransferManager(server, client, finder); manager.start(); } @@ -111,6 +123,15 @@ public void testLocalFileTransfer() throws IOException, ExecutionException, Inte response.setDiscoveryResult(Collections.singletonList("localhost")); doReturn(response).when(finder).discoverBlobPeers(anyString(), anyInt(), anyInt()); + StoreVersionState storeVersionState = new StoreVersionState(); + Mockito.doReturn(storeVersionState).when(storageMetadataService).getStoreVersionState(Mockito.any()); + + InternalAvroSpecificSerializer partitionStateSerializer = + AvroProtocolDefinition.PARTITION_STATE.getSerializer(); + OffsetRecord expectOffsetRecord = new OffsetRecord(partitionStateSerializer); + expectOffsetRecord.setOffsetLag(1000L); + Mockito.doReturn(expectOffsetRecord).when(storageMetadataService).getLastOffset(Mockito.any(), Mockito.anyInt()); + // Prepare files in the snapshot directory Path snapshotDir = Paths.get( RocksDBUtils.composeSnapshotDir(tmpSnapshotDir.toString(), TEST_STORE + "_v" + TEST_VERSION, TEST_PARTITION)); @@ -155,5 +176,18 @@ public void testLocalFileTransfer() throws IOException, ExecutionException, Inte Assert.assertTrue(Arrays.equals(Files.readAllBytes(file1), Files.readAllBytes(destFile1))); Assert.assertTrue(Arrays.equals(Files.readAllBytes(file2), Files.readAllBytes(destFile2))); Assert.assertTrue(Arrays.equals(Files.readAllBytes(file3), Files.readAllBytes(destFile3))); + + // Verify the metadata is retrieved + Mockito.verify(storageMetadataService, Mockito.times(1)) + .getLastOffset(TEST_STORE + "_v" + TEST_VERSION, TEST_PARTITION); + Mockito.verify(storageMetadataService, Mockito.times(1)).getStoreVersionState(TEST_STORE + "_v" + TEST_VERSION); + + // Verify the record is updated + Mockito.verify(storageMetadataService, Mockito.times(1)) + .put(TEST_STORE + "_v" + TEST_VERSION, TEST_PARTITION, expectOffsetRecord); + + // Verify the store version state is updated + Mockito.verify(storageMetadataService, Mockito.times(1)) + .computeStoreVersionState(Mockito.anyString(), Mockito.any()); } } diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/blobtransfer/TestP2PFileTransferClientHandler.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/blobtransfer/TestP2PFileTransferClientHandler.java new file mode 100644 index 0000000000..a2cc424538 --- /dev/null +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/blobtransfer/TestP2PFileTransferClientHandler.java @@ -0,0 +1,378 @@ +package com.linkedin.davinci.blobtransfer; + +import static com.linkedin.davinci.blobtransfer.BlobTransferUtils.BLOB_TRANSFER_COMPLETED; +import static com.linkedin.davinci.blobtransfer.BlobTransferUtils.BLOB_TRANSFER_STATUS; +import static com.linkedin.davinci.blobtransfer.BlobTransferUtils.BLOB_TRANSFER_TYPE; +import static com.linkedin.davinci.blobtransfer.BlobTransferUtils.BlobTransferType; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.linkedin.davinci.blobtransfer.client.MetadataAggregator; +import com.linkedin.davinci.blobtransfer.client.P2PFileTransferClientHandler; +import com.linkedin.davinci.blobtransfer.client.P2PMetadataTransferHandler; +import com.linkedin.davinci.storage.StorageMetadataService; +import com.linkedin.venice.exceptions.VeniceException; +import com.linkedin.venice.kafka.protocol.state.PartitionState; +import com.linkedin.venice.offsets.OffsetRecord; +import com.linkedin.venice.serialization.avro.AvroProtocolDefinition; +import com.linkedin.venice.serialization.avro.InternalAvroSpecificSerializer; +import io.netty.buffer.Unpooled; +import io.netty.channel.embedded.EmbeddedChannel; +import io.netty.handler.codec.http.DefaultFullHttpResponse; +import io.netty.handler.codec.http.DefaultHttpContent; +import io.netty.handler.codec.http.DefaultHttpResponse; +import io.netty.handler.codec.http.DefaultLastHttpContent; +import io.netty.handler.codec.http.FullHttpResponse; +import io.netty.handler.codec.http.HttpContent; +import io.netty.handler.codec.http.HttpHeaderNames; +import io.netty.handler.codec.http.HttpResponseStatus; +import io.netty.handler.codec.http.HttpVersion; +import io.netty.handler.codec.http.LastHttpContent; +import io.netty.util.CharsetUtil; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Comparator; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + + +public class TestP2PFileTransferClientHandler { + EmbeddedChannel ch; + Path baseDir; + String TEST_STORE = "test_store"; + int TEST_VERSION = 1; + int TEST_PARTITION = 0; + CompletionStage inputStreamFuture; + StorageMetadataService storageMetadataService; + + P2PFileTransferClientHandler clientFileHandler; + P2PMetadataTransferHandler clientMetadataHandler; + + @BeforeMethod + public void setUp() throws IOException { + baseDir = Files.createTempDirectory("tmp"); + inputStreamFuture = new CompletableFuture<>(); + storageMetadataService = Mockito.mock(StorageMetadataService.class); + + clientFileHandler = Mockito.spy( + new P2PFileTransferClientHandler( + baseDir.toString(), + inputStreamFuture, + TEST_STORE, + TEST_VERSION, + TEST_PARTITION)); + + clientMetadataHandler = Mockito.spy( + new P2PMetadataTransferHandler( + storageMetadataService, + baseDir.toString(), + TEST_STORE, + TEST_VERSION, + TEST_PARTITION)); + + Mockito.doNothing().when(clientMetadataHandler).updateStorePartitionMetadata(Mockito.any(), Mockito.any()); + + ch = new EmbeddedChannel(new MetadataAggregator(1024 * 1024 * 100), clientFileHandler, clientMetadataHandler); + } + + @AfterMethod + public void teardown() throws IOException { + ch.close(); + Files.walk(baseDir).sorted(Comparator.reverseOrder()).forEach(path -> { + try { + Files.delete(path); + } catch (IOException e) { + e.printStackTrace(); + } + }); + } + + @Test + public void testFailToGetResponse() { + DefaultHttpResponse response = + new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.INTERNAL_SERVER_ERROR); + ch.writeInbound(response); + try { + inputStreamFuture.toCompletableFuture().get(1, TimeUnit.MINUTES); + Assert.fail("Expected exception not thrown"); + } catch (Exception e) { + Assert.assertTrue(e.getCause() instanceof VeniceException); + Assert.assertEquals( + e.getCause().getMessage(), + "Failed to fetch file from remote peer. Response: 500 Internal Server Error"); + } + } + + @Test + public void testInvalidResponseHeader() { + DefaultHttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + response.headers().add(BLOB_TRANSFER_TYPE, BlobTransferType.FILE); + ch.writeInbound(response); + try { + inputStreamFuture.toCompletableFuture().get(1, TimeUnit.MINUTES); + Assert.fail("Expected exception not thrown"); + } catch (Exception e) { + Assert.assertTrue(e.getCause() instanceof VeniceException); + Assert.assertEquals(e.getCause().getMessage(), "No file name specified in the response for test_store_v1-0"); + } + } + + @Test + public void testFileSizeMismatchTransfer() { + // response + DefaultHttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + response.headers().add("Content-Disposition", "filename=\"test_file.txt\""); + response.headers().add("Content-Length", "5"); + response.headers().add(BLOB_TRANSFER_TYPE, BlobTransferType.FILE); + // content 1 + // length 1 + HttpContent chunk1 = new DefaultLastHttpContent(Unpooled.copiedBuffer("0", CharsetUtil.UTF_8)); + + ch.writeInbound(response); + ch.writeInbound(chunk1); + try { + inputStreamFuture.toCompletableFuture().get(1, TimeUnit.MINUTES); + Assert.fail("Expected exception not thrown"); + } catch (Exception e) { + Assert.assertTrue(e.getCause() instanceof VeniceException); + Assert.assertEquals(e.getCause().getMessage(), "File size mismatch for test_file.txt. Expected: 5, Actual: 1"); + } + } + + // Technically, it shouldn't happen as the response and content are supposed to arrive in order but just in case + @Test + public void testOutOfOrderResponseTransfer() { + HttpContent chunk1 = new DefaultLastHttpContent(Unpooled.copiedBuffer("0", CharsetUtil.UTF_8)); + + ch.writeInbound(chunk1); + try { + inputStreamFuture.toCompletableFuture().get(1, TimeUnit.MINUTES); + Assert.fail("Expected exception not thrown"); + } catch (Exception e) { + Assert.assertTrue(e.getCause() instanceof VeniceException); + Assert.assertEquals(e.getCause().getMessage(), "No file opened to write for test_store_v1-0"); + } + } + + @Test + public void testSingleFileTransfer() throws ExecutionException, InterruptedException, IOException, TimeoutException { + // response + DefaultHttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + response.headers().add("Content-Disposition", "filename=\"test_file.txt\""); + response.headers().add("Content-Length", "5"); + response.headers().add(BLOB_TRANSFER_TYPE, BlobTransferType.FILE); + // content 1 + HttpContent chunk = new DefaultLastHttpContent(Unpooled.copiedBuffer("12345", CharsetUtil.UTF_8)); + + // End of a file transfer + HttpContent endOfFile = LastHttpContent.EMPTY_LAST_CONTENT; + + // End of all file transfer + DefaultHttpResponse endOfTransfer = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + endOfTransfer.headers().add(BLOB_TRANSFER_STATUS, BLOB_TRANSFER_COMPLETED); + + ch.writeInbound(response); + ch.writeInbound(chunk); + ch.writeInbound(endOfFile); + ch.writeInbound(endOfTransfer); + inputStreamFuture.toCompletableFuture().get(1, TimeUnit.MINUTES); + + // verify the content is written to the disk + BlobTransferPayload payload = new BlobTransferPayload(baseDir.toString(), TEST_STORE, TEST_VERSION, TEST_PARTITION); + Path dest = Paths.get(payload.getPartitionDir()); + Assert.assertTrue(Files.exists(dest)); + Assert.assertTrue(Files.isDirectory(dest)); + Path file1 = dest.resolve("test_file.txt"); + Assert.assertTrue(Files.exists(file1)); + Assert.assertEquals(Files.size(file1), 5); + } + + @Test + public void testMultipleFilesTransfer() + throws ExecutionException, InterruptedException, IOException, TimeoutException { + // response 1 + DefaultHttpResponse response1 = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + response1.headers().add("Content-Disposition", "filename=\"test_file1.txt\""); + response1.headers().add("Content-Length", "5"); + response1.headers().add(BLOB_TRANSFER_TYPE, BlobTransferType.FILE); + // response 2 + DefaultHttpResponse response2 = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + response2.headers().add("Content-Disposition", "filename=\"test_file2.txt\""); + response2.headers().add("Content-Length", "10"); + response2.headers().add(BLOB_TRANSFER_TYPE, BlobTransferType.FILE); + // content + HttpContent chunk1 = new DefaultLastHttpContent(Unpooled.copiedBuffer("12345", CharsetUtil.UTF_8)); + HttpContent chunk2 = new DefaultHttpContent(Unpooled.copiedBuffer("67890", CharsetUtil.UTF_8)); + HttpContent chunk3 = new DefaultLastHttpContent(Unpooled.copiedBuffer("13579", CharsetUtil.UTF_8)); + + // End of all file transfer + DefaultHttpResponse endOfTransfer = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + endOfTransfer.headers().add(BLOB_TRANSFER_STATUS, BLOB_TRANSFER_COMPLETED); + + ch.writeInbound(response1); + ch.writeInbound(chunk1); + ch.writeInbound(response2); + ch.writeInbound(chunk2); + ch.writeInbound(chunk3); + ch.writeInbound(endOfTransfer); + inputStreamFuture.toCompletableFuture().get(1, TimeUnit.MINUTES); + + // verify the content is written to the disk + BlobTransferPayload payload = new BlobTransferPayload(baseDir.toString(), TEST_STORE, TEST_VERSION, TEST_PARTITION); + Path dest = Paths.get(payload.getPartitionDir()); + Assert.assertTrue(Files.exists(dest)); + Assert.assertTrue(Files.isDirectory(dest)); + Path file1 = dest.resolve("test_file1.txt"); + Assert.assertTrue(Files.exists(file1)); + Assert.assertEquals(Files.size(file1), 5); + Path file2 = dest.resolve("test_file2.txt"); + Assert.assertTrue(Files.exists(file2)); + Assert.assertEquals(Files.size(file2), 10); + } + + @Test + public void testSingleMetadataTransfer() + throws JsonProcessingException, ExecutionException, InterruptedException, TimeoutException { + BlobTransferPartitionMetadata expectedMetadata = new BlobTransferPartitionMetadata(); + expectedMetadata.setTopicName(TEST_STORE + "_v" + TEST_VERSION); + expectedMetadata.setPartitionId(TEST_PARTITION); + InternalAvroSpecificSerializer partitionStateSerializer = + AvroProtocolDefinition.PARTITION_STATE.getSerializer(); + OffsetRecord offsetRecord = new OffsetRecord(partitionStateSerializer); + offsetRecord.setOffsetLag(1000L); + expectedMetadata.setOffsetRecord(ByteBuffer.wrap(offsetRecord.toBytes())); + + ObjectMapper objectMapper = new ObjectMapper(); + String metadataJson = objectMapper.writeValueAsString(expectedMetadata); + byte[] metadataBytes = metadataJson.getBytes(CharsetUtil.UTF_8); + + FullHttpResponse metadataResponse = new DefaultFullHttpResponse( + HttpVersion.HTTP_1_1, + HttpResponseStatus.OK, + Unpooled.copiedBuffer(metadataJson, CharsetUtil.UTF_8)); + metadataResponse.headers().set(HttpHeaderNames.CONTENT_LENGTH, metadataBytes.length); + metadataResponse.headers().set(HttpHeaderNames.CONTENT_TYPE, "application/json"); + metadataResponse.headers().set(BLOB_TRANSFER_TYPE, BlobTransferType.METADATA); + + DefaultHttpResponse endOfTransfer = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + endOfTransfer.headers().add(BLOB_TRANSFER_STATUS, BLOB_TRANSFER_COMPLETED); + + // Simulate inbound data for the metadata response + ch.writeInbound(metadataResponse); + ch.writeInbound(endOfTransfer); + + // Ensure the future is completed + inputStreamFuture.toCompletableFuture().get(1, TimeUnit.MINUTES); + + // Verify that the metadata was correctly parsed and handled + BlobTransferPartitionMetadata actualMetadata = clientMetadataHandler.getMetadata(); + Assert.assertNotNull(actualMetadata); + Assert.assertEquals(actualMetadata.getTopicName(), expectedMetadata.getTopicName()); + Assert.assertEquals(actualMetadata.getPartitionId(), expectedMetadata.getPartitionId()); + Assert.assertEquals(actualMetadata.getOffsetRecord(), expectedMetadata.getOffsetRecord()); + Assert.assertTrue(inputStreamFuture.toCompletableFuture().isDone()); + } + + @Test + public void testMultipleFilesAndOneMetadataTransfer() + throws ExecutionException, InterruptedException, IOException, TimeoutException { + // Transfer of two files + // File 1 response + DefaultHttpResponse response1 = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + response1.headers().add("Content-Disposition", "filename=\"test_file1.txt\""); + response1.headers().add("Content-Length", "5"); + response1.headers().add(BLOB_TRANSFER_TYPE, BlobTransferType.FILE); + + // File 2 response + DefaultHttpResponse response2 = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + response2.headers().add("Content-Disposition", "filename=\"test_file2.txt\""); + response2.headers().add("Content-Length", "10"); + response2.headers().add(BLOB_TRANSFER_TYPE, BlobTransferType.FILE); + + // File content chunks + HttpContent chunk1 = new DefaultLastHttpContent(Unpooled.copiedBuffer("12345", CharsetUtil.UTF_8)); + HttpContent chunk2 = new DefaultHttpContent(Unpooled.copiedBuffer("67890", CharsetUtil.UTF_8)); + HttpContent chunk3 = new DefaultLastHttpContent(Unpooled.copiedBuffer("13579", CharsetUtil.UTF_8)); + + // Simulate inbound data for file transfer + ch.writeInbound(response1); + ch.writeInbound(chunk1); + ch.writeInbound(response2); + ch.writeInbound(chunk2); + ch.writeInbound(chunk3); + + // Transfer metadata + // Prepare metadata + BlobTransferPartitionMetadata expectMetadata = new BlobTransferPartitionMetadata(); + expectMetadata.setTopicName(TEST_STORE + "_v" + TEST_VERSION); + expectMetadata.setPartitionId(TEST_PARTITION); + InternalAvroSpecificSerializer partitionStateSerializer = + AvroProtocolDefinition.PARTITION_STATE.getSerializer(); + OffsetRecord offsetRecord = new OffsetRecord(partitionStateSerializer); + offsetRecord.setOffsetLag(1000L); + expectMetadata.setOffsetRecord(ByteBuffer.wrap(offsetRecord.toBytes())); + + // Serialize metadata to JSON + ObjectMapper objectMapper = new ObjectMapper(); + String metadataJson = objectMapper.writeValueAsString(expectMetadata); + byte[] metadataBytes = metadataJson.getBytes(CharsetUtil.UTF_8); + // Metadata HTTP response + FullHttpResponse metadataResponse = new DefaultFullHttpResponse( + HttpVersion.HTTP_1_1, + HttpResponseStatus.OK, + Unpooled.copiedBuffer(metadataJson, CharsetUtil.UTF_8)); + metadataResponse.headers().set(HttpHeaderNames.CONTENT_LENGTH, metadataBytes.length); + metadataResponse.headers().set(HttpHeaderNames.CONTENT_TYPE, "application/json"); + metadataResponse.headers().set(BLOB_TRANSFER_TYPE, BlobTransferType.METADATA); + + // Simulate inbound data for the metadata transfer + ch.writeInbound(metadataResponse); + + // End of metadata transfer + DefaultHttpResponse endOfMetadataTransfer = + new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + endOfMetadataTransfer.headers().add(BLOB_TRANSFER_STATUS, BLOB_TRANSFER_COMPLETED); + ch.writeInbound(endOfMetadataTransfer); + + // Ensure the future is completed + inputStreamFuture.toCompletableFuture().get(1, TimeUnit.MINUTES); + + // Verify the files are written to disk + BlobTransferPayload payload = new BlobTransferPayload(baseDir.toString(), TEST_STORE, TEST_VERSION, TEST_PARTITION); + Path dest = Paths.get(payload.getPartitionDir()); + Assert.assertTrue(Files.exists(dest)); + Assert.assertTrue(Files.isDirectory(dest)); + + // Verify file 1 + Path file1 = dest.resolve("test_file1.txt"); + Assert.assertTrue(Files.exists(file1)); + Assert.assertEquals(Files.size(file1), 5); + + // Verify file 2 + Path file2 = dest.resolve("test_file2.txt"); + Assert.assertTrue(Files.exists(file2)); + Assert.assertEquals(Files.size(file2), 10); + + // Verify the metadata was correctly parsed and handled + BlobTransferPartitionMetadata actualMetadata = clientMetadataHandler.getMetadata(); + Assert.assertNotNull(actualMetadata); + Assert.assertEquals(actualMetadata.getTopicName(), expectMetadata.getTopicName()); + Assert.assertEquals(actualMetadata.getPartitionId(), expectMetadata.getPartitionId()); + Assert.assertEquals(actualMetadata.getOffsetRecord(), expectMetadata.getOffsetRecord()); + + // Ensure the future is completed + Assert.assertTrue(inputStreamFuture.toCompletableFuture().isDone()); + } +} diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/blobtransfer/server/TestP2PFileTransferServerHandler.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/blobtransfer/TestP2PFileTransferServerHandler.java similarity index 51% rename from internal/venice-common/src/test/java/com/linkedin/venice/blobtransfer/server/TestP2PFileTransferServerHandler.java rename to clients/da-vinci-client/src/test/java/com/linkedin/davinci/blobtransfer/TestP2PFileTransferServerHandler.java index e83559e96b..48622fe9a5 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/blobtransfer/server/TestP2PFileTransferServerHandler.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/blobtransfer/TestP2PFileTransferServerHandler.java @@ -1,9 +1,20 @@ -package com.linkedin.venice.blobtransfer.server; +package com.linkedin.davinci.blobtransfer; -import static com.linkedin.venice.blobtransfer.BlobTransferUtils.BLOB_TRANSFER_COMPLETED; -import static com.linkedin.venice.blobtransfer.BlobTransferUtils.BLOB_TRANSFER_STATUS; +import static com.linkedin.davinci.blobtransfer.BlobTransferUtils.BLOB_TRANSFER_COMPLETED; +import static com.linkedin.davinci.blobtransfer.BlobTransferUtils.BLOB_TRANSFER_STATUS; +import static com.linkedin.davinci.blobtransfer.BlobTransferUtils.BLOB_TRANSFER_TYPE; +import static com.linkedin.davinci.blobtransfer.BlobTransferUtils.BlobTransferType; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.linkedin.davinci.blobtransfer.server.P2PFileTransferServerHandler; +import com.linkedin.davinci.storage.StorageMetadataService; +import com.linkedin.venice.kafka.protocol.state.PartitionState; +import com.linkedin.venice.kafka.protocol.state.StoreVersionState; +import com.linkedin.venice.offsets.OffsetRecord; +import com.linkedin.venice.serialization.avro.AvroProtocolDefinition; +import com.linkedin.venice.serialization.avro.InternalAvroSpecificSerializer; import com.linkedin.venice.store.rocksdb.RocksDBUtils; +import io.netty.buffer.ByteBuf; import io.netty.channel.DefaultFileRegion; import io.netty.channel.embedded.EmbeddedChannel; import io.netty.handler.codec.http.DefaultFullHttpRequest; @@ -12,10 +23,12 @@ import io.netty.handler.codec.http.FullHttpResponse; import io.netty.handler.codec.http.HttpHeaderNames; import io.netty.handler.codec.http.HttpMethod; +import io.netty.handler.codec.http.HttpResponseStatus; import io.netty.handler.codec.http.HttpVersion; import io.netty.handler.codec.http.LastHttpContent; import io.netty.handler.timeout.IdleStateEvent; import java.io.IOException; +import java.nio.ByteBuffer; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; @@ -23,6 +36,7 @@ import java.util.Comparator; import java.util.HashSet; import java.util.Set; +import org.mockito.Mockito; import org.testng.Assert; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeMethod; @@ -32,11 +46,15 @@ public class TestP2PFileTransferServerHandler { EmbeddedChannel ch; Path baseDir; + StorageMetadataService storageMetadataService; + P2PFileTransferServerHandler serverHandler; @BeforeMethod public void setUp() throws IOException { baseDir = Files.createTempDirectory("tmp"); - ch = new EmbeddedChannel(new P2PFileTransferServerHandler(baseDir.toString())); + storageMetadataService = Mockito.mock(StorageMetadataService.class); + serverHandler = new P2PFileTransferServerHandler(baseDir.toString(), storageMetadataService); + ch = new EmbeddedChannel(serverHandler); } @AfterMethod @@ -96,7 +114,18 @@ public void testIdleChannelClose() { } @Test - public void testTransferSingleFile() throws IOException { + public void testTransferSingleFileAndSingleMetadata() throws IOException { + // prepare response from metadata service + StoreVersionState storeVersionState = new StoreVersionState(); + Mockito.doReturn(storeVersionState).when(storageMetadataService).getStoreVersionState(Mockito.any()); + + InternalAvroSpecificSerializer partitionStateSerializer = + AvroProtocolDefinition.PARTITION_STATE.getSerializer(); + OffsetRecord offsetRecord = new OffsetRecord(partitionStateSerializer); + offsetRecord.setOffsetLag(1000L); + Mockito.doReturn(offsetRecord).when(storageMetadataService).getLastOffset(Mockito.any(), Mockito.anyInt()); + + // prepare the file request Path snapshotDir = Paths.get(RocksDBUtils.composeSnapshotDir(baseDir.toString(), "myStore_v1", 10)); Files.createDirectories(snapshotDir); Path file1 = snapshotDir.resolve("file1"); @@ -104,6 +133,7 @@ public void testTransferSingleFile() throws IOException { FullHttpRequest request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "/myStore/1/10"); ch.writeInbound(request); + // start of file1 Object response = ch.readOutbound(); Assert.assertTrue(response instanceof DefaultHttpResponse); @@ -111,20 +141,42 @@ public void testTransferSingleFile() throws IOException { Assert.assertEquals( httpResponse.headers().get(HttpHeaderNames.CONTENT_DISPOSITION), "attachment; filename=\"file1\""); + Assert.assertEquals(httpResponse.headers().get(BLOB_TRANSFER_TYPE), BlobTransferType.FILE.toString()); + // send the content in one chunk response = ch.readOutbound(); Assert.assertTrue(response instanceof DefaultFileRegion); + // the last empty response for file1 response = ch.readOutbound(); Assert.assertTrue(response instanceof LastHttpContent); // end of file1 + + // start of metadata + response = ch.readOutbound(); + Assert.assertTrue(response instanceof DefaultHttpResponse); + DefaultHttpResponse metadataResponse = (DefaultHttpResponse) response; + Assert.assertEquals(metadataResponse.headers().get(BLOB_TRANSFER_TYPE), BlobTransferType.METADATA.toString()); + // end of metadata + + // start of STATUS response response = ch.readOutbound(); Assert.assertTrue(response instanceof DefaultHttpResponse); DefaultHttpResponse endOfTransfer = (DefaultHttpResponse) response; Assert.assertEquals(endOfTransfer.headers().get(BLOB_TRANSFER_STATUS), BLOB_TRANSFER_COMPLETED); - // end of all file + // end of STATUS response } @Test public void testTransferMultipleFiles() throws IOException { + // prepare response from metadata service + StoreVersionState storeVersionState = new StoreVersionState(); + Mockito.doReturn(storeVersionState).when(storageMetadataService).getStoreVersionState(Mockito.any()); + + InternalAvroSpecificSerializer partitionStateSerializer = + AvroProtocolDefinition.PARTITION_STATE.getSerializer(); + OffsetRecord offsetRecord = new OffsetRecord(partitionStateSerializer); + offsetRecord.setOffsetLag(1000L); + Mockito.doReturn(offsetRecord).when(storageMetadataService).getLastOffset(Mockito.any(), Mockito.anyInt()); + Path snapshotDir = Paths.get(RocksDBUtils.composeSnapshotDir(baseDir.toString(), "myStore_v1", 10)); Files.createDirectories(snapshotDir); Path file1 = snapshotDir.resolve("file1"); @@ -157,10 +209,68 @@ public void testTransferMultipleFiles() throws IOException { response = ch.readOutbound(); Assert.assertTrue(response instanceof LastHttpContent); // end of a file2 + + // start of metadata + response = ch.readOutbound(); + Assert.assertTrue(response instanceof FullHttpResponse); + FullHttpResponse metadataResponse = (FullHttpResponse) response; + Assert.assertEquals(metadataResponse.headers().get(BLOB_TRANSFER_TYPE), BlobTransferType.METADATA.toString()); + + ByteBuf content = metadataResponse.content(); + byte[] metadataBytes = new byte[content.readableBytes()]; + content.readBytes(metadataBytes); + ObjectMapper objectMapper = new ObjectMapper(); + BlobTransferPartitionMetadata metadata = objectMapper.readValue(metadataBytes, BlobTransferPartitionMetadata.class); + + Assert.assertEquals(metadata.getTopicName(), "myStore_v1"); + Assert.assertEquals(metadata.getPartitionId(), 10); + Assert.assertEquals(metadata.getOffsetRecord(), ByteBuffer.wrap(offsetRecord.toBytes())); + + InternalAvroSpecificSerializer storeVersionStateSerializer = + AvroProtocolDefinition.STORE_VERSION_STATE.getSerializer(); + java.nio.ByteBuffer storeVersionStateByte = + ByteBuffer.wrap(storeVersionStateSerializer.serialize(metadata.getTopicName(), storeVersionState)); + Assert.assertEquals(metadata.getStoreVersionState(), storeVersionStateByte); + // end of metadata + + // start of STATUS response response = ch.readOutbound(); Assert.assertTrue(response instanceof DefaultHttpResponse); DefaultHttpResponse endOfTransfer = (DefaultHttpResponse) response; Assert.assertEquals(endOfTransfer.headers().get(BLOB_TRANSFER_STATUS), BLOB_TRANSFER_COMPLETED); - // end of all file + // end of STATUS response + } + + @Test + public void testWhenMetadataCreateError() throws IOException { + // prepare the file request + Path snapshotDir = Paths.get(RocksDBUtils.composeSnapshotDir(baseDir.toString(), "myStore_v1", 10)); + Files.createDirectories(snapshotDir); + Path file1 = snapshotDir.resolve("file1"); + Files.write(file1.toAbsolutePath(), "hello".getBytes()); + FullHttpRequest request = new DefaultFullHttpRequest(HttpVersion.HTTP_1_1, HttpMethod.GET, "/myStore/1/10"); + + ch.writeInbound(request); + + // start of file1 + Object response = ch.readOutbound(); + Assert.assertTrue(response instanceof DefaultHttpResponse); + DefaultHttpResponse httpResponse = (DefaultHttpResponse) response; + Assert.assertEquals( + httpResponse.headers().get(HttpHeaderNames.CONTENT_DISPOSITION), + "attachment; filename=\"file1\""); + Assert.assertEquals(httpResponse.headers().get(BLOB_TRANSFER_TYPE), BlobTransferType.FILE.toString()); + // send the content in one chunk + response = ch.readOutbound(); + Assert.assertTrue(response instanceof DefaultFileRegion); + // the last empty response for file1 + response = ch.readOutbound(); + Assert.assertTrue(response instanceof LastHttpContent); + // end of file1 + + // metadata in server side has error + response = ch.readOutbound(); + Assert.assertTrue(response instanceof DefaultHttpResponse); + Assert.assertEquals(((DefaultHttpResponse) response).status(), HttpResponseStatus.INTERNAL_SERVER_ERROR); } } diff --git a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/ingestion/DefaultIngestionBackendTest.java b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/ingestion/DefaultIngestionBackendTest.java index b3b357b422..a5dcf48fd9 100644 --- a/clients/da-vinci-client/src/test/java/com/linkedin/davinci/ingestion/DefaultIngestionBackendTest.java +++ b/clients/da-vinci-client/src/test/java/com/linkedin/davinci/ingestion/DefaultIngestionBackendTest.java @@ -9,13 +9,13 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import com.linkedin.davinci.blobtransfer.BlobTransferManager; import com.linkedin.davinci.config.VeniceServerConfig; import com.linkedin.davinci.config.VeniceStoreVersionConfig; import com.linkedin.davinci.kafka.consumer.KafkaStoreIngestionService; import com.linkedin.davinci.storage.StorageMetadataService; import com.linkedin.davinci.storage.StorageService; import com.linkedin.davinci.store.AbstractStorageEngine; -import com.linkedin.venice.blobtransfer.BlobTransferManager; import com.linkedin.venice.exceptions.VenicePeersNotFoundException; import com.linkedin.venice.kafka.protocol.state.StoreVersionState; import com.linkedin.venice.meta.ReadOnlyStoreRepository; diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/BlobTransferUtils.java b/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/BlobTransferUtils.java deleted file mode 100644 index f11634488d..0000000000 --- a/internal/venice-common/src/main/java/com/linkedin/venice/blobtransfer/BlobTransferUtils.java +++ /dev/null @@ -1,6 +0,0 @@ -package com.linkedin.venice.blobtransfer; - -public class BlobTransferUtils { - public static final String BLOB_TRANSFER_STATUS = "X-Blob-Transfer-Status"; - public static final String BLOB_TRANSFER_COMPLETED = "Completed"; -} diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/blobtransfer/client/TestP2PFileTransferClientHandler.java b/internal/venice-common/src/test/java/com/linkedin/venice/blobtransfer/client/TestP2PFileTransferClientHandler.java deleted file mode 100644 index 7503b6443e..0000000000 --- a/internal/venice-common/src/test/java/com/linkedin/venice/blobtransfer/client/TestP2PFileTransferClientHandler.java +++ /dev/null @@ -1,206 +0,0 @@ -package com.linkedin.venice.blobtransfer.client; - -import static com.linkedin.venice.blobtransfer.BlobTransferUtils.BLOB_TRANSFER_COMPLETED; -import static com.linkedin.venice.blobtransfer.BlobTransferUtils.BLOB_TRANSFER_STATUS; - -import com.linkedin.venice.blobtransfer.BlobTransferPayload; -import com.linkedin.venice.exceptions.VeniceException; -import io.netty.buffer.Unpooled; -import io.netty.channel.embedded.EmbeddedChannel; -import io.netty.handler.codec.http.DefaultFullHttpResponse; -import io.netty.handler.codec.http.DefaultHttpContent; -import io.netty.handler.codec.http.DefaultHttpResponse; -import io.netty.handler.codec.http.DefaultLastHttpContent; -import io.netty.handler.codec.http.HttpContent; -import io.netty.handler.codec.http.HttpResponseStatus; -import io.netty.handler.codec.http.HttpVersion; -import io.netty.handler.codec.http.LastHttpContent; -import io.netty.util.CharsetUtil; -import java.io.IOException; -import java.io.InputStream; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.Comparator; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionStage; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import org.testng.Assert; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; - - -public class TestP2PFileTransferClientHandler { - EmbeddedChannel ch; - Path baseDir; - String TEST_STORE = "test_store"; - int TEST_VERSION = 1; - int TEST_PARTITION = 0; - CompletionStage inputStreamFuture; - - @BeforeMethod - public void setUp() throws IOException { - baseDir = Files.createTempDirectory("tmp"); - inputStreamFuture = new CompletableFuture<>(); - ch = new EmbeddedChannel( - new P2PFileTransferClientHandler( - baseDir.toString(), - inputStreamFuture, - TEST_STORE, - TEST_VERSION, - TEST_PARTITION)); - } - - @AfterMethod - public void teardown() throws IOException { - ch.close(); - Files.walk(baseDir).sorted(Comparator.reverseOrder()).forEach(path -> { - try { - Files.delete(path); - } catch (IOException e) { - e.printStackTrace(); - } - }); - } - - @Test - public void testFailToGetResponse() { - DefaultHttpResponse response = - new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.INTERNAL_SERVER_ERROR); - ch.writeInbound(response); - try { - inputStreamFuture.toCompletableFuture().get(1, TimeUnit.MINUTES); - Assert.fail("Expected exception not thrown"); - } catch (Exception e) { - Assert.assertTrue(e.getCause() instanceof VeniceException); - Assert.assertEquals( - e.getCause().getMessage(), - "Failed to fetch file from remote peer. Response: 500 Internal Server Error"); - } - } - - @Test - public void testInvalidResponseHeader() { - DefaultHttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); - ch.writeInbound(response); - try { - inputStreamFuture.toCompletableFuture().get(1, TimeUnit.MINUTES); - Assert.fail("Expected exception not thrown"); - } catch (Exception e) { - Assert.assertTrue(e.getCause() instanceof VeniceException); - Assert.assertEquals(e.getCause().getMessage(), "No file name specified in the response for test_store_v1-0"); - } - } - - @Test - public void testFileSizeMismatchTransfer() { - // response - DefaultHttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); - response.headers().add("Content-Disposition", "filename=\"test_file.txt\""); - response.headers().add("Content-Length", "5"); - // content 1 - // length 1 - HttpContent chunk1 = new DefaultLastHttpContent(Unpooled.copiedBuffer("0", CharsetUtil.UTF_8)); - - ch.writeInbound(response); - ch.writeInbound(chunk1); - try { - inputStreamFuture.toCompletableFuture().get(1, TimeUnit.MINUTES); - Assert.fail("Expected exception not thrown"); - } catch (Exception e) { - Assert.assertTrue(e.getCause() instanceof VeniceException); - Assert.assertEquals(e.getCause().getMessage(), "File size mismatch for test_file.txt. Expected: 5, Actual: 1"); - } - } - - // Technically, it shouldn't happen as the response and content are supposed to arrive in order but just in case - @Test - public void testOutOfOrderResponseTransfer() { - HttpContent chunk1 = new DefaultLastHttpContent(Unpooled.copiedBuffer("0", CharsetUtil.UTF_8)); - - ch.writeInbound(chunk1); - try { - inputStreamFuture.toCompletableFuture().get(1, TimeUnit.MINUTES); - Assert.fail("Expected exception not thrown"); - } catch (Exception e) { - Assert.assertTrue(e.getCause() instanceof VeniceException); - Assert.assertEquals(e.getCause().getMessage(), "No file opened to write for test_store_v1-0"); - } - } - - @Test - public void testSingleFileTransfer() throws ExecutionException, InterruptedException, IOException, TimeoutException { - // response - DefaultHttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); - response.headers().add("Content-Disposition", "filename=\"test_file.txt\""); - response.headers().add("Content-Length", "5"); - // content 1 - HttpContent chunk = new DefaultLastHttpContent(Unpooled.copiedBuffer("12345", CharsetUtil.UTF_8)); - - // End of a file transfer - HttpContent endOfFile = LastHttpContent.EMPTY_LAST_CONTENT; - - // End of all file transfer - DefaultHttpResponse endOfTransfer = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); - endOfTransfer.headers().add(BLOB_TRANSFER_STATUS, BLOB_TRANSFER_COMPLETED); - - ch.writeInbound(response); - ch.writeInbound(chunk); - ch.writeInbound(endOfFile); - ch.writeInbound(endOfTransfer); - inputStreamFuture.toCompletableFuture().get(1, TimeUnit.MINUTES); - - // verify the content is written to the disk - BlobTransferPayload payload = new BlobTransferPayload(baseDir.toString(), TEST_STORE, TEST_VERSION, TEST_PARTITION); - Path dest = Paths.get(payload.getPartitionDir()); - Assert.assertTrue(Files.exists(dest)); - Assert.assertTrue(Files.isDirectory(dest)); - Path file1 = dest.resolve("test_file.txt"); - Assert.assertTrue(Files.exists(file1)); - Assert.assertEquals(Files.size(file1), 5); - } - - @Test - public void testMultipleFilesTransfer() - throws ExecutionException, InterruptedException, IOException, TimeoutException { - // response 1 - DefaultHttpResponse response1 = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); - response1.headers().add("Content-Disposition", "filename=\"test_file1.txt\""); - response1.headers().add("Content-Length", "5"); - // response 2 - DefaultHttpResponse response2 = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); - response2.headers().add("Content-Disposition", "filename=\"test_file2.txt\""); - response2.headers().add("Content-Length", "10"); - // content - HttpContent chunk1 = new DefaultLastHttpContent(Unpooled.copiedBuffer("12345", CharsetUtil.UTF_8)); - HttpContent chunk2 = new DefaultHttpContent(Unpooled.copiedBuffer("67890", CharsetUtil.UTF_8)); - HttpContent chunk3 = new DefaultLastHttpContent(Unpooled.copiedBuffer("13579", CharsetUtil.UTF_8)); - - // End of all file transfer - DefaultHttpResponse endOfTransfer = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); - endOfTransfer.headers().add(BLOB_TRANSFER_STATUS, BLOB_TRANSFER_COMPLETED); - - ch.writeInbound(response1); - ch.writeInbound(chunk1); - ch.writeInbound(response2); - ch.writeInbound(chunk2); - ch.writeInbound(chunk3); - ch.writeInbound(endOfTransfer); - inputStreamFuture.toCompletableFuture().get(1, TimeUnit.MINUTES); - - // verify the content is written to the disk - BlobTransferPayload payload = new BlobTransferPayload(baseDir.toString(), TEST_STORE, TEST_VERSION, TEST_PARTITION); - Path dest = Paths.get(payload.getPartitionDir()); - Assert.assertTrue(Files.exists(dest)); - Assert.assertTrue(Files.isDirectory(dest)); - Path file1 = dest.resolve("test_file1.txt"); - Assert.assertTrue(Files.exists(file1)); - Assert.assertEquals(Files.size(file1), 5); - Path file2 = dest.resolve("test_file2.txt"); - Assert.assertTrue(Files.exists(file2)); - Assert.assertEquals(Files.size(file2), 10); - } -} diff --git a/services/venice-server/src/main/java/com/linkedin/venice/server/VeniceServer.java b/services/venice-server/src/main/java/com/linkedin/venice/server/VeniceServer.java index 206e5a2465..0314e0f194 100644 --- a/services/venice-server/src/main/java/com/linkedin/venice/server/VeniceServer.java +++ b/services/venice-server/src/main/java/com/linkedin/venice/server/VeniceServer.java @@ -1,6 +1,8 @@ package com.linkedin.venice.server; import com.linkedin.avro.fastserde.FastDeserializerGeneratorAccessor; +import com.linkedin.davinci.blobtransfer.BlobTransferManager; +import com.linkedin.davinci.blobtransfer.BlobTransferUtil; import com.linkedin.davinci.compression.StorageEngineBackedCompressorFactory; import com.linkedin.davinci.config.VeniceClusterConfig; import com.linkedin.davinci.config.VeniceConfigLoader; @@ -21,8 +23,6 @@ import com.linkedin.davinci.storage.StorageService; import com.linkedin.venice.acl.DynamicAccessController; import com.linkedin.venice.acl.StaticAccessController; -import com.linkedin.venice.blobtransfer.BlobTransferManager; -import com.linkedin.venice.blobtransfer.BlobTransferUtil; import com.linkedin.venice.cleaner.BackupVersionOptimizationService; import com.linkedin.venice.cleaner.LeakedResourceCleaner; import com.linkedin.venice.cleaner.ResourceReadUsageTracker; @@ -452,7 +452,8 @@ private List createServices() { serverConfig.getDvcP2pBlobTransferServerPort(), serverConfig.getDvcP2pBlobTransferClientPort(), serverConfig.getRocksDBPath(), - customizedViewFuture); + customizedViewFuture, + storageMetadataService); } else { blobTransferManager = null; } From 93d86d8a377c474e03a9e815b842cf127352dfcc Mon Sep 17 00:00:00 2001 From: Gaojie Liu Date: Thu, 26 Sep 2024 10:23:46 -0700 Subject: [PATCH 13/16] [server] Added metrics to track batch processing perf (#1195) * [server] Added metrics to track batch processing perf This PR added the following metrics: Host/Store/store-version level: batch_processing_request batch_processing_request_size batch_processing_request_records batch_processing_request_latency batch_processing_request_error Host/Store level: leader_produce_latency * Removed debug code * Fixed diff coverage issue --- .../ActiveActiveStoreIngestionTask.java | 5 +- .../consumer/IngestionBatchProcessor.java | 36 ++++++++++- .../LeaderFollowerStoreIngestionTask.java | 7 ++- .../stats/AggVersionedIngestionStats.java | 15 +++++ .../stats/HostLevelIngestionStats.java | 62 +++++++++++++++++++ .../davinci/stats/IngestionStats.java | 52 ++++++++++++++++ .../davinci/stats/IngestionStatsReporter.java | 43 +++++++++++++ .../ActiveActiveStoreIngestionTaskTest.java | 5 +- .../consumer/IngestionBatchProcessorTest.java | 57 ++++++++++++++++- .../venice/stats/AbstractVeniceStats.java | 13 ++++ .../venice/stats/AbstractVeniceStatsTest.java | 17 +++++ 11 files changed, 303 insertions(+), 9 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java index c317c065f2..38c7d6bbb9 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/ActiveActiveStoreIngestionTask.java @@ -164,7 +164,9 @@ public ActiveActiveStoreIngestionTask( null, this::processActiveActiveMessage, isWriteComputationEnabled, - isActiveActiveReplicationEnabled()); + isActiveActiveReplicationEnabled(), + aggVersionedIngestionStats, + getHostLevelIngestionStats()); }); } @@ -608,7 +610,6 @@ protected void processMessageAndMaybeProduceToKafka( } PubSubMessage consumerRecord = consumerRecordWrapper.getMessage(); KafkaKey kafkaKey = consumerRecord.getKey(); - // KafkaMessageEnvelope kafkaValue = consumerRecord.getValue(); byte[] keyBytes = kafkaKey.getKey(); final MergeConflictResultWrapper mergeConflictResultWrapper; if (consumerRecordWrapper.getProcessedResult() != null 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 index 3e1a8202fd..299de9c0f0 100644 --- 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 @@ -1,10 +1,14 @@ package com.linkedin.davinci.kafka.consumer; +import com.linkedin.davinci.stats.AggVersionedIngestionStats; +import com.linkedin.davinci.stats.HostLevelIngestionStats; 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.meta.Version; import com.linkedin.venice.pubsub.api.PubSubMessage; +import com.linkedin.venice.utils.LatencyUtils; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -35,11 +39,15 @@ PubSubMessageProcessedResult apply( } private final String storeVersionName; + private final String storeName; + private final int version; private final ExecutorService batchProcessingThreadPool; private final KeyLevelLocksManager lockManager; private final boolean isWriteComputationEnabled; private final boolean isActiveActiveReplicationEnabled; private final ProcessingFunction processingFunction; + private final AggVersionedIngestionStats aggVersionedIngestionStats; + private final HostLevelIngestionStats hostLevelIngestionStats; public IngestionBatchProcessor( String storeVersionName, @@ -47,13 +55,20 @@ public IngestionBatchProcessor( KeyLevelLocksManager lockManager, ProcessingFunction processingFunction, boolean isWriteComputationEnabled, - boolean isActiveActiveReplicationEnabled) { + boolean isActiveActiveReplicationEnabled, + AggVersionedIngestionStats aggVersionedIngestionStats, + HostLevelIngestionStats hostLevelIngestionStats) { this.storeVersionName = storeVersionName; this.batchProcessingThreadPool = batchProcessingThreadPool; this.lockManager = lockManager; this.processingFunction = processingFunction; this.isWriteComputationEnabled = isWriteComputationEnabled; this.isActiveActiveReplicationEnabled = isActiveActiveReplicationEnabled; + this.aggVersionedIngestionStats = aggVersionedIngestionStats; + this.hostLevelIngestionStats = hostLevelIngestionStats; + + this.storeName = Version.parseStoreFromKafkaTopicName(storeVersionName); + this.version = Version.parseVersionFromKafkaTopicName(storeVersionName); } /** @@ -104,6 +119,7 @@ public List>> keyGroupMap = new HashMap<>(records.size()); - resultList.forEach(r -> { + + for (PubSubMessageProcessedResultWrapper r: resultList) { PubSubMessage message = r.getMessage(); if (!message.getKey().isControlMessage()) { ByteArrayKey byteArrayKey = ByteArrayKey.wrap(message.getKey().getKey()); keyGroupMap.computeIfAbsent(byteArrayKey, (ignored) -> new ArrayList<>()).add(r); + totalNumOfRecords++; } - }); + } + aggVersionedIngestionStats + .recordBatchProcessingRequest(storeName, version, totalNumOfRecords, System.currentTimeMillis()); + hostLevelIngestionStats.recordBatchProcessingRequest(totalNumOfRecords); + List> futureList = new ArrayList<>(keyGroupMap.size()); keyGroupMap.forEach((ignored, recordsWithTheSameKey) -> { futureList.add(CompletableFuture.runAsync(() -> { @@ -153,7 +177,13 @@ public List stat.recordBatchProcessingRequest(size, timestamp)); + } + + public void recordBatchProcessingRequestError(String storeName, int version) { + recordVersionedAndTotalStat(storeName, version, stat -> stat.recordBatchProcessingRequestError()); + } + + public void recordBatchProcessingLatency(String storeName, int version, double latency, long timestamp) { + recordVersionedAndTotalStat( + storeName, + version, + stat -> stat.recordBatchProcessingRequestLatency(latency, timestamp)); + } } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/stats/HostLevelIngestionStats.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/stats/HostLevelIngestionStats.java index e3fb9c3dea..661ed0a967 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/stats/HostLevelIngestionStats.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/stats/HostLevelIngestionStats.java @@ -1,5 +1,11 @@ package com.linkedin.davinci.stats; +import static com.linkedin.davinci.stats.IngestionStats.BATCH_PROCESSING_REQUEST; +import static com.linkedin.davinci.stats.IngestionStats.BATCH_PROCESSING_REQUEST_ERROR; +import static com.linkedin.davinci.stats.IngestionStats.BATCH_PROCESSING_REQUEST_LATENCY; +import static com.linkedin.davinci.stats.IngestionStats.BATCH_PROCESSING_REQUEST_RECORDS; +import static com.linkedin.davinci.stats.IngestionStats.BATCH_PROCESSING_REQUEST_SIZE; + import com.linkedin.davinci.config.VeniceServerConfig; import com.linkedin.davinci.kafka.consumer.PartitionConsumptionState; import com.linkedin.davinci.kafka.consumer.StoreIngestionTask; @@ -138,6 +144,13 @@ public class HostLevelIngestionStats extends AbstractVeniceStats { */ private final LongAdderRateGauge totalTombstoneCreationDCRRate; + private final Sensor leaderProduceLatencySensor; + private final LongAdderRateGauge batchProcessingRequestSensor; + private final Sensor batchProcessingRequestSizeSensor; + private final LongAdderRateGauge batchProcessingRequestRecordsSensor; + private final Sensor batchProcessingRequestLatencySensor; + private final LongAdderRateGauge batchProcessingRequestErrorSensor; + /** * @param totalStats the total stats singleton instance, or null if we are constructing the total stats */ @@ -435,6 +448,37 @@ public HostLevelIngestionStats( totalStats, () -> totalStats.leaderIngestionActiveActiveDeleteLatencySensor, avgAndMax()); + + this.leaderProduceLatencySensor = registerPerStoreAndTotalSensor( + "leader_produce_latency", + totalStats, + () -> totalStats.leaderProduceLatencySensor, + avgAndMax()); + this.batchProcessingRequestSensor = registerOnlyTotalRate( + BATCH_PROCESSING_REQUEST, + totalStats, + () -> totalStats.batchProcessingRequestSensor, + time); + this.batchProcessingRequestErrorSensor = registerOnlyTotalRate( + BATCH_PROCESSING_REQUEST_ERROR, + totalStats, + () -> totalStats.batchProcessingRequestErrorSensor, + time); + this.batchProcessingRequestRecordsSensor = registerOnlyTotalRate( + BATCH_PROCESSING_REQUEST_RECORDS, + totalStats, + () -> totalStats.batchProcessingRequestRecordsSensor, + time); + this.batchProcessingRequestSizeSensor = registerOnlyTotalSensor( + BATCH_PROCESSING_REQUEST_SIZE, + totalStats, + () -> totalStats.batchProcessingRequestSizeSensor, + avgAndMax()); + this.batchProcessingRequestLatencySensor = registerOnlyTotalSensor( + BATCH_PROCESSING_REQUEST_LATENCY, + totalStats, + () -> totalStats.batchProcessingRequestLatencySensor, + avgAndMax()); } /** Record a host-level byte consumption rate across all store versions */ @@ -614,4 +658,22 @@ public void recordTimestampRegressionDCRError() { public void recordOffsetRegressionDCRError() { totalOffsetRegressionDCRErrorRate.record(); } + + public void recordLeaderProduceLatency(double latency) { + leaderProduceLatencySensor.record(latency); + } + + public void recordBatchProcessingRequest(int size) { + batchProcessingRequestSensor.record(); + batchProcessingRequestRecordsSensor.record(size); + batchProcessingRequestSizeSensor.record(size); + } + + public void recordBatchProcessingRequestError() { + batchProcessingRequestErrorSensor.record(); + } + + public void recordBatchProcessingRequestLatency(double latency) { + batchProcessingRequestLatencySensor.record(latency); + } } diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/stats/IngestionStats.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/stats/IngestionStats.java index 7816f47010..281ee815a8 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/stats/IngestionStats.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/stats/IngestionStats.java @@ -64,6 +64,11 @@ public class IngestionStats { public static final String PRODUCER_CALLBACK_LATENCY = "producer_callback_latency"; public static final String LEADER_PREPROCESSING_LATENCY = "leader_preprocessing_latency"; public static final String INTERNAL_PREPROCESSING_LATENCY = "internal_preprocessing_latency"; + public static final String BATCH_PROCESSING_REQUEST = "batch_processing_request"; + public static final String BATCH_PROCESSING_REQUEST_SIZE = "batch_processing_request_size"; + public static final String BATCH_PROCESSING_REQUEST_RECORDS = "batch_processing_request_records"; + public static final String BATCH_PROCESSING_REQUEST_LATENCY = "batch_processing_request_latency"; + public static final String BATCH_PROCESSING_REQUEST_ERROR = "batch_processing_request_error"; private static final MetricConfig METRIC_CONFIG = new MetricConfig(); private StoreIngestionTask ingestionTask; @@ -117,6 +122,11 @@ public class IngestionStats { private Count transformerErrorCount = new Count(); private Sensor transformerErrorSensor; + private final LongAdderRateGauge batchProcessingRequestSensor = new LongAdderRateGauge(); + private final WritePathLatencySensor batchProcessingRequestSizeSensor; + private final LongAdderRateGauge batchProcessingRequestRecordsSensor = new LongAdderRateGauge(); + private final WritePathLatencySensor batchProcessingRequestLatencySensor; + private final LongAdderRateGauge batchProcessingRequestErrorSensor = new LongAdderRateGauge(); public IngestionStats(VeniceServerConfig serverConfig) { @@ -207,6 +217,14 @@ public IngestionStats(VeniceServerConfig serverConfig) { registerSensor(localMetricRepository, OFFSET_REGRESSION_DCR_ERROR, offsetRegressionDCRErrorSensor); registerSensor(localMetricRepository, TOMBSTONE_CREATION_DCR, tombstoneCreationDCRSensor); registerSensor(localMetricRepository, IDLE_TIME, idleTimeSensor); + + registerSensor(localMetricRepository, BATCH_PROCESSING_REQUEST, batchProcessingRequestSensor); + registerSensor(localMetricRepository, BATCH_PROCESSING_REQUEST_RECORDS, batchProcessingRequestRecordsSensor); + registerSensor(localMetricRepository, BATCH_PROCESSING_REQUEST_ERROR, batchProcessingRequestErrorSensor); + batchProcessingRequestSizeSensor = + new WritePathLatencySensor(localMetricRepository, METRIC_CONFIG, BATCH_PROCESSING_REQUEST_SIZE); + batchProcessingRequestLatencySensor = + new WritePathLatencySensor(localMetricRepository, METRIC_CONFIG, BATCH_PROCESSING_REQUEST_LATENCY); } private void registerSensor(MetricsRepository localMetricRepository, String sensorName, LongAdderRateGauge gauge) { @@ -665,6 +683,40 @@ public WritePathLatencySensor getLeaderProducerCompletionLatencySensor() { return leaderProducerCompletionLatencySensor; } + public void recordBatchProcessingRequest(int size, long currentTimeMs) { + batchProcessingRequestSensor.record(); + batchProcessingRequestRecordsSensor.record(size); + batchProcessingRequestSizeSensor.record(size, currentTimeMs); + } + + public double getBatchProcessingRequest() { + return batchProcessingRequestSensor.getRate(); + } + + public double getBatchProcessingRequestRecords() { + return batchProcessingRequestRecordsSensor.getRate(); + } + + public void recordBatchProcessingRequestError() { + batchProcessingRequestErrorSensor.record(); + } + + public double getBatchProcessingRequestError() { + return batchProcessingRequestErrorSensor.getRate(); + } + + public WritePathLatencySensor getBatchProcessingRequestSizeSensor() { + return batchProcessingRequestSizeSensor; + } + + public void recordBatchProcessingRequestLatency(double latency, long currentTimeMs) { + batchProcessingRequestLatencySensor.record(latency, currentTimeMs); + } + + public WritePathLatencySensor getBatchProcessingRequestLatencySensor() { + return batchProcessingRequestLatencySensor; + } + public static double unAvailableToZero(double value) { /* When data is unavailable, return 0 instead of NaN or Infinity. Some metrics are initialized to -INF. This can cause problems when metrics are aggregated. Use only when zero makes semantic sense. diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/stats/IngestionStatsReporter.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/stats/IngestionStatsReporter.java index 9630437ed1..e88f39e5c9 100644 --- a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/stats/IngestionStatsReporter.java +++ b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/stats/IngestionStatsReporter.java @@ -2,6 +2,11 @@ import static com.linkedin.davinci.stats.IngestionStats.BATCH_FOLLOWER_OFFSET_LAG; import static com.linkedin.davinci.stats.IngestionStats.BATCH_LEADER_OFFSET_LAG; +import static com.linkedin.davinci.stats.IngestionStats.BATCH_PROCESSING_REQUEST; +import static com.linkedin.davinci.stats.IngestionStats.BATCH_PROCESSING_REQUEST_ERROR; +import static com.linkedin.davinci.stats.IngestionStats.BATCH_PROCESSING_REQUEST_LATENCY; +import static com.linkedin.davinci.stats.IngestionStats.BATCH_PROCESSING_REQUEST_RECORDS; +import static com.linkedin.davinci.stats.IngestionStats.BATCH_PROCESSING_REQUEST_SIZE; import static com.linkedin.davinci.stats.IngestionStats.BATCH_REPLICATION_LAG; import static com.linkedin.davinci.stats.IngestionStats.BYTES_CONSUMED_METRIC_NAME; import static com.linkedin.davinci.stats.IngestionStats.CONSUMED_RECORD_END_TO_END_PROCESSING_LATENCY; @@ -226,6 +231,44 @@ protected void registerStats() { "local_broker_to_follower_consumer", IngestionStats::getLocalBrokerFollowerConsumerLatencySensor); registerLatencySensor("leader_producer_completion", IngestionStats::getLeaderProducerCompletionLatencySensor); + + registerSensor( + new IngestionStatsGauge(this, () -> getStats().getBatchProcessingRequest(), 0, BATCH_PROCESSING_REQUEST)); + registerSensor( + new IngestionStatsGauge( + this, + () -> getStats().getBatchProcessingRequestError(), + BATCH_PROCESSING_REQUEST_ERROR)); + registerSensor( + new IngestionStatsGauge( + this, + () -> getStats().getBatchProcessingRequestRecords(), + 0, + BATCH_PROCESSING_REQUEST_RECORDS)); + registerSensor( + new IngestionStatsGauge( + this, + () -> getStats().getBatchProcessingRequestSizeSensor().getAvg(), + 0, + BATCH_PROCESSING_REQUEST_SIZE + "_avg")); + registerSensor( + new IngestionStatsGauge( + this, + () -> getStats().getBatchProcessingRequestSizeSensor().getMax(), + 0, + BATCH_PROCESSING_REQUEST_SIZE + "_max")); + registerSensor( + new IngestionStatsGauge( + this, + () -> getStats().getBatchProcessingRequestLatencySensor().getAvg(), + 0, + BATCH_PROCESSING_REQUEST_LATENCY + "_avg")); + registerSensor( + new IngestionStatsGauge( + this, + () -> getStats().getBatchProcessingRequestLatencySensor().getMax(), + 0, + BATCH_PROCESSING_REQUEST_LATENCY + "_max")); } } 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 afedb84fe1..86c4a32b77 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 @@ -5,6 +5,7 @@ import static com.linkedin.venice.ConfigKeys.ZOOKEEPER_ADDRESS; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyDouble; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; @@ -315,8 +316,9 @@ public void testLeaderCanSendValueChunksIntoDrainer() throws InterruptedExceptio long beforeProcessingRecordTimestamp = 0; boolean resultReuseInput = true; + HostLevelIngestionStats mockHostLevelIngestionStats = mock(HostLevelIngestionStats.class); ActiveActiveStoreIngestionTask ingestionTask = mock(ActiveActiveStoreIngestionTask.class); - when(ingestionTask.getHostLevelIngestionStats()).thenReturn(mock(HostLevelIngestionStats.class)); + when(ingestionTask.getHostLevelIngestionStats()).thenReturn(mockHostLevelIngestionStats); when(ingestionTask.getVersionIngestionStats()).thenReturn(mock(AggVersionedIngestionStats.class)); when(ingestionTask.getVersionedDIVStats()).thenReturn(mock(AggVersionedDIVStats.class)); when(ingestionTask.getKafkaVersionTopic()).thenReturn(testTopic); @@ -452,6 +454,7 @@ public void testLeaderCanSendValueChunksIntoDrainer() throws InterruptedExceptio Assert.assertEquals( leaderProducedRecordContextArgumentCaptor.getAllValues().get(3).getKeyBytes(), kafkaKeyArgumentCaptor.getAllValues().get(4).getKey()); + verify(mockHostLevelIngestionStats).recordLeaderProduceLatency(anyDouble()); } @Test 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 index 9225980610..768e3ae364 100644 --- 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 @@ -1,15 +1,22 @@ package com.linkedin.davinci.kafka.consumer; +import static org.mockito.ArgumentMatchers.anyDouble; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; 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 static org.testng.Assert.expectThrows; import com.linkedin.alpini.base.concurrency.ExecutorService; import com.linkedin.alpini.base.concurrency.Executors; +import com.linkedin.davinci.stats.AggVersionedIngestionStats; +import com.linkedin.davinci.stats.HostLevelIngestionStats; import com.linkedin.davinci.utils.ByteArrayKey; +import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.kafka.protocol.KafkaMessageEnvelope; import com.linkedin.venice.kafka.protocol.Put; import com.linkedin.venice.kafka.protocol.enums.MessageType; @@ -106,7 +113,9 @@ public void lockKeysTest() { mockKeyLevelLocksManager, (ignored1, ignored2, ignored3, ignored4, ignored5, ignored6, ignored7) -> null, true, - true); + true, + mock(AggVersionedIngestionStats.class), + mock(HostLevelIngestionStats.class)); List locks = batchProcessor.lockKeys(Arrays.asList(rtMessage1, rtMessage2)); verify(mockKeyLevelLocksManager).acquireLockByKey(ByteArrayKey.wrap(key1)); verify(mockKeyLevelLocksManager).acquireLockByKey(ByteArrayKey.wrap(key2)); @@ -146,6 +155,9 @@ public void processTest() { 101, 100); + AggVersionedIngestionStats mockAggVersionedIngestionStats = mock(AggVersionedIngestionStats.class); + HostLevelIngestionStats mockHostLevelIngestionStats = mock(HostLevelIngestionStats.class); + IngestionBatchProcessor batchProcessor = new IngestionBatchProcessor( "store_v1", Executors.newFixedThreadPool(1, new DaemonThreadFactory("test")), @@ -165,7 +177,9 @@ public void processTest() { return null; }, true, - true); + true, + mockAggVersionedIngestionStats, + mockHostLevelIngestionStats); List> result = batchProcessor.process( Arrays.asList(rtMessage1, rtMessage2), @@ -185,6 +199,45 @@ public void processTest() { assertEquals( resultForKey2.getProcessedResult().getWriteComputeResultWrapper().getNewPut().putValue.array(), "value2".getBytes()); + verify(mockAggVersionedIngestionStats).recordBatchProcessingRequest(eq("store"), eq(1), eq(2), anyLong()); + verify(mockAggVersionedIngestionStats).recordBatchProcessingLatency(eq("store"), eq(1), anyDouble(), anyLong()); + verify(mockHostLevelIngestionStats).recordBatchProcessingRequest(2); + verify(mockHostLevelIngestionStats).recordBatchProcessingRequestLatency(anyDouble()); + + // Error path + 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())) { + throw new VeniceException("Fake"); + } + return null; + }, + true, + true, + mockAggVersionedIngestionStats, + mockHostLevelIngestionStats); + final IngestionBatchProcessor finalBatchProcessor = batchProcessor; + VeniceException exception = expectThrows( + VeniceException.class, + () -> finalBatchProcessor.process( + Arrays.asList(rtMessage1, rtMessage2), + mock(PartitionConsumptionState.class), + 1, + "test_kafka", + 1, + 1, + 1)); + assertTrue(exception.getMessage().contains("Failed to execute the batch processing")); + verify(mockAggVersionedIngestionStats).recordBatchProcessingRequestError("store", 1); + verify(mockHostLevelIngestionStats).recordBatchProcessingRequestError(); } } diff --git a/internal/venice-client-common/src/main/java/com/linkedin/venice/stats/AbstractVeniceStats.java b/internal/venice-client-common/src/main/java/com/linkedin/venice/stats/AbstractVeniceStats.java index 1aad9bc3b8..9bbf2dee89 100644 --- a/internal/venice-client-common/src/main/java/com/linkedin/venice/stats/AbstractVeniceStats.java +++ b/internal/venice-client-common/src/main/java/com/linkedin/venice/stats/AbstractVeniceStats.java @@ -201,6 +201,19 @@ protected Sensor registerPerStoreAndTotalSensor( return registerSensor(sensorName, parent, stats); } + protected Sensor registerOnlyTotalSensor( + String sensorName, + AbstractVeniceStats totalStats, + Supplier totalSensor, + MeasurableStat... stats) { + + if (totalStats == null) { + return registerSensor(sensorName, stats); + } else { + return totalSensor.get(); + } + } + /** * Only register sensor for total stats. If not provided, create a new one. * @param sensorName diff --git a/internal/venice-client-common/src/test/java/com/linkedin/venice/stats/AbstractVeniceStatsTest.java b/internal/venice-client-common/src/test/java/com/linkedin/venice/stats/AbstractVeniceStatsTest.java index 3627a1c5c7..9531662e10 100644 --- a/internal/venice-client-common/src/test/java/com/linkedin/venice/stats/AbstractVeniceStatsTest.java +++ b/internal/venice-client-common/src/test/java/com/linkedin/venice/stats/AbstractVeniceStatsTest.java @@ -6,6 +6,7 @@ import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertTrue; import com.linkedin.venice.client.stats.BasicClientStats; @@ -228,4 +229,20 @@ public void testRegisterOnlyTotalRate() { sensor = stats.registerOnlyTotalRate("testSensor", null, () -> parentCount, SystemTime.INSTANCE); Assert.assertNotEquals(sensor, parentCount); } + + @Test + public void testRegisterOnlyTotalSensor() { + MetricsRepository metricsRepository = new MetricsRepository(); + + AbstractVeniceStats stats = new AbstractVeniceStats(metricsRepository, "testStore"); + AbstractVeniceStats totalStats = new AbstractVeniceStats(metricsRepository, "total"); + Sensor totalSensor = totalStats.registerSensor("testSensor", new OccurrenceRate()); + // 1) total stats is not null so use ths supplier + Sensor sensor = stats.registerOnlyTotalSensor("testSensor", totalStats, () -> totalSensor, new OccurrenceRate()); + assertEquals(sensor, totalSensor); + + // 2) total stats is null, so created a new one + Sensor newTotalSensor = stats.registerOnlyTotalSensor("testSensor", null, () -> totalSensor, new OccurrenceRate()); + assertNotEquals(newTotalSensor, totalSensor); + } } From c772dd39560bf2d26fea5bdd86d191b083567cb0 Mon Sep 17 00:00:00 2001 From: Sourav Maji Date: Thu, 26 Sep 2024 11:48:45 -0700 Subject: [PATCH 14/16] Revert "[common][server][dvc-client] Make a true non-blocking end-offset fetcher (#1201) Revert "[common][server][dvc-client] Make a true non-blocking end-offset fetcher (#1170)" This reverts commit 262101c6968abbd47b5d2ea28e4eb9e0c4784778. Co-authored-by: Sourav Maji --- .../kafka/consumer/StoreIngestionTask.java | 14 +------------- .../consumer/StoreIngestionTaskTest.java | 12 ------------ .../venice/pubsub/manager/TopicManager.java | 5 ----- .../pubsub/manager/TopicMetadataFetcher.java | 19 ------------------- .../manager/TopicMetadataFetcherTest.java | 3 --- 5 files changed, 1 insertion(+), 52 deletions(-) diff --git a/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java b/clients/da-vinci-client/src/main/java/com/linkedin/davinci/kafka/consumer/StoreIngestionTask.java index 75a8d6c454..bf3e35fb22 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 @@ -102,7 +102,6 @@ import com.linkedin.venice.utils.LatencyUtils; import com.linkedin.venice.utils.PartitionUtils; import com.linkedin.venice.utils.RedundantExceptionFilter; -import com.linkedin.venice.utils.RetryUtils; import com.linkedin.venice.utils.SparseConcurrentList; import com.linkedin.venice.utils.Time; import com.linkedin.venice.utils.Timer; @@ -2220,18 +2219,7 @@ protected long getTopicPartitionEndOffSet(String kafkaUrl, PubSubTopic pubSubTop if (offsetFromConsumer >= 0) { return offsetFromConsumer; } - try { - return RetryUtils.executeWithMaxAttempt(() -> { - long offset = getTopicManager(kafkaUrl).getLatestOffsetCachedNonBlocking(pubSubTopic, partition); - if (offset == -1) { - throw new VeniceException("Found latest offset -1"); - } - return offset; - }, 5, Duration.ofSeconds(1), Collections.singletonList(VeniceException.class)); - } catch (Exception e) { - LOGGER.error("Could not find latest offset for {} even after 5 retries", pubSubTopic.getName()); - return -1; - } + return getTopicManager(kafkaUrl).getLatestOffsetCached(pubSubTopic, partition); } protected long getPartitionOffsetLagBasedOnMetrics(String kafkaSourceAddress, PubSubTopic topic, int partition) { 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 d02e59bd9c..a5b5969a80 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 @@ -3017,9 +3017,6 @@ public void testIsReadyToServe(NodeType nodeType, AAConfig aaConfig, DataReplica doReturn(5L).when(mockTopicManager).getLatestOffsetCached(any(), anyInt()); doReturn(150L).when(mockTopicManagerRemoteKafka).getLatestOffsetCached(any(), anyInt()); doReturn(150L).when(aggKafkaConsumerService).getLatestOffsetBasedOnMetrics(anyString(), any(), any()); - long endOffset = - storeIngestionTaskUnderTest.getTopicPartitionEndOffSet(localKafkaConsumerService.kafkaUrl, pubSubTopic, 1); - assertEquals(endOffset, 150L); if (nodeType == NodeType.LEADER) { // case 6a: leader replica => partition is not ready to serve doReturn(LeaderFollowerStateType.LEADER).when(mockPcsBufferReplayStartedRemoteLagging).getLeaderFollowerState(); @@ -3179,15 +3176,6 @@ public void testActiveActiveStoreIsReadyToServe(HybridConfig hybridConfig, NodeT } else { assertTrue(storeIngestionTaskUnderTest.isReadyToServe(mockPcsMultipleSourceKafkaServers)); } - doReturn(10L).when(aggKafkaConsumerService).getLatestOffsetBasedOnMetrics(anyString(), any(), any()); - long endOffset = - storeIngestionTaskUnderTest.getTopicPartitionEndOffSet(localKafkaConsumerService.kafkaUrl, pubSubTopic, 0); - assertEquals(endOffset, 10L); - doReturn(-1L).when(aggKafkaConsumerService).getLatestOffsetBasedOnMetrics(anyString(), any(), any()); - endOffset = - storeIngestionTaskUnderTest.getTopicPartitionEndOffSet(localKafkaConsumerService.kafkaUrl, pubSubTopic, 0); - assertEquals(endOffset, 0L); - } @DataProvider diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java index c91d3c452e..1c3c0ed915 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicManager.java @@ -731,11 +731,6 @@ public long getLatestOffsetCached(PubSubTopic pubSubTopic, int partitionId) { return topicMetadataFetcher.getLatestOffsetCached(new PubSubTopicPartitionImpl(pubSubTopic, partitionId)); } - public long getLatestOffsetCachedNonBlocking(PubSubTopic pubSubTopic, int partitionId) { - return topicMetadataFetcher - .getLatestOffsetCachedNonBlocking(new PubSubTopicPartitionImpl(pubSubTopic, partitionId)); - } - public long getProducerTimestampOfLastDataMessageWithRetries(PubSubTopicPartition pubSubTopicPartition, int retries) { return topicMetadataFetcher.getProducerTimestampOfLastDataMessageWithRetries(pubSubTopicPartition, retries); } diff --git a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java index 4a4e7ac85e..9c8e80f4ac 100644 --- a/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java +++ b/internal/venice-common/src/main/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcher.java @@ -379,25 +379,6 @@ CompletableFuture getLatestOffsetWithRetriesAsync(PubSubTopicPartition pub .supplyAsync(() -> getLatestOffsetWithRetries(pubSubTopicPartition, retries), threadPoolExecutor); } - long getLatestOffsetCachedNonBlocking(PubSubTopicPartition pubSubTopicPartition) { - ValueAndExpiryTime cachedValue; - cachedValue = latestOffsetCache.get(pubSubTopicPartition); - updateCacheAsync( - pubSubTopicPartition, - cachedValue, - latestOffsetCache, - () -> getLatestOffsetWithRetriesAsync( - pubSubTopicPartition, - DEFAULT_MAX_RETRIES_FOR_POPULATING_TMD_CACHE_ENTRY)); - if (cachedValue == null) { - cachedValue = latestOffsetCache.get(pubSubTopicPartition); - if (cachedValue == null) { - return -1; - } - } - return cachedValue.getValue(); - } - long getLatestOffsetCached(PubSubTopicPartition pubSubTopicPartition) { ValueAndExpiryTime cachedValue; try { diff --git a/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcherTest.java b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcherTest.java index bca7d15629..214be223c5 100644 --- a/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcherTest.java +++ b/internal/venice-common/src/test/java/com/linkedin/venice/pubsub/manager/TopicMetadataFetcherTest.java @@ -260,9 +260,6 @@ public void testGetTopicLatestOffsets() { assertEquals(res.size(), offsetsMap.size()); assertEquals(res.get(0), 111L); assertEquals(res.get(1), 222L); - assertEquals( - topicMetadataFetcher.getLatestOffsetCachedNonBlocking(new PubSubTopicPartitionImpl(pubSubTopic, 0)), - -1); verify(consumerMock, times(3)).partitionsFor(pubSubTopic); verify(consumerMock, times(1)).endOffsets(eq(offsetsMap.keySet()), any(Duration.class)); From 04aeaebb1fa56b5f392336074120ac9eb72acff1 Mon Sep 17 00:00:00 2001 From: Jialin Liu Date: Thu, 26 Sep 2024 13:59:43 -0700 Subject: [PATCH 15/16] [controller] Superset schema generation should reflect default value updates in new value schema (#1184) When user updates a field default value, we should reflect it in the changes in superset schema, which is used in read-compute / partial update store. This should apply to all the value type, including union schema. --- .../venice/utils/AvroSchemaUtils.java | 5 +- .../venice/utils/AvroSupersetSchemaUtils.java | 67 +++++++++++-------- .../schema/TestAvroSupersetSchemaUtils.java | 45 ++++++++----- .../src/test/resources/UnionV1.avsc | 31 +++++++++ .../src/test/resources/UnionV2.avsc | 26 +++++++ .../VeniceParentHelixAdminTest.java | 43 ++++++++++++ .../venice/endToEnd/TestEmptyPush.java | 3 +- .../linkedin/venice/utils/TestWriteUtils.java | 7 ++ .../main/resources/valueSchema/UnionV1.avsc | 15 +++++ .../main/resources/valueSchema/UnionV2.avsc | 15 +++++ .../main/resources/valueSchema/UnionV3.avsc | 20 ++++++ .../controller/VeniceParentHelixAdmin.java | 2 - .../DefaultSupersetSchemaGenerator.java | 2 +- ...SupersetSchemaGeneratorWithCustomProp.java | 2 +- .../systemstore/SystemStoreRepairTask.java | 1 - 15 files changed, 232 insertions(+), 52 deletions(-) create mode 100644 internal/venice-client-common/src/test/resources/UnionV1.avsc create mode 100644 internal/venice-client-common/src/test/resources/UnionV2.avsc create mode 100644 internal/venice-test-common/src/main/resources/valueSchema/UnionV1.avsc create mode 100644 internal/venice-test-common/src/main/resources/valueSchema/UnionV2.avsc create mode 100644 internal/venice-test-common/src/main/resources/valueSchema/UnionV3.avsc diff --git a/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/AvroSchemaUtils.java b/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/AvroSchemaUtils.java index 7d3340ed27..fe96a6cb5b 100644 --- a/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/AvroSchemaUtils.java +++ b/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/AvroSchemaUtils.java @@ -262,10 +262,9 @@ public static SchemaEntry generateSupersetSchemaFromAllValueSchemas(Collection combinedSchema = new ArrayList<>(); - Map s2Schema = s2.getTypes().stream().collect(Collectors.toMap(s -> s.getName(), s -> s)); - for (Schema subSchemaInS1: s1.getTypes()) { - final String fieldName = subSchemaInS1.getName(); - final Schema subSchemaWithSameNameInS2 = s2Schema.get(fieldName); - if (subSchemaWithSameNameInS2 == null) { - combinedSchema.add(subSchemaInS1); + Map existingSchemaTypeMap = + existingSchema.getTypes().stream().collect(Collectors.toMap(Schema::getName, s -> s)); + for (Schema subSchemaInNewSchema: newSchema.getTypes()) { + final String fieldName = subSchemaInNewSchema.getName(); + final Schema subSchemaInExistingSchema = existingSchemaTypeMap.get(fieldName); + if (subSchemaInExistingSchema == null) { + combinedSchema.add(subSchemaInNewSchema); } else { - combinedSchema.add(generateSuperSetSchema(subSchemaInS1, subSchemaWithSameNameInS2)); - s2Schema.remove(fieldName); + combinedSchema.add(generateSupersetSchema(subSchemaInExistingSchema, subSchemaInNewSchema)); + existingSchemaTypeMap.remove(fieldName); } } - s2Schema.forEach((k, v) -> combinedSchema.add(v)); - + existingSchemaTypeMap.forEach((k, v) -> combinedSchema.add(v)); return Schema.createUnion(combinedSchema); } @@ -135,25 +140,31 @@ private static FieldBuilder deepCopySchemaField(Schema.Field field) { return fieldBuilder; } - private static List mergeFieldSchemas(Schema s1, Schema s2) { + /** + * Merge field schema from two schema object. The rule is: If a field exist in both new schema and old schema, we should + * generate the superset schema of these two versions of the same field, with new schema's information taking higher + * priority. + * @param newSchema new schema + * @param existingSchema old schema + * @return merged schema field + */ + private static List mergeFieldSchemas(Schema existingSchema, Schema newSchema) { List fields = new ArrayList<>(); - for (Schema.Field f1: s1.getFields()) { - Schema.Field f2 = s2.getField(f1.name()); + for (Schema.Field fieldInNewSchema: newSchema.getFields()) { + Schema.Field fieldInExistingSchema = existingSchema.getField(fieldInNewSchema.name()); - FieldBuilder fieldBuilder = deepCopySchemaField(f1); - if (f2 != null) { - fieldBuilder.setSchema(generateSuperSetSchema(f1.schema(), f2.schema())) - .setDoc(f1.doc() != null ? f1.doc() : f2.doc()); - // merge props from f2 - copyFieldProperties(fieldBuilder, f2); + FieldBuilder fieldBuilder = deepCopySchemaField(fieldInNewSchema); + if (fieldInExistingSchema != null) { + fieldBuilder.setSchema(generateSupersetSchema(fieldInExistingSchema.schema(), fieldInNewSchema.schema())) + .setDoc(fieldInNewSchema.doc() != null ? fieldInNewSchema.doc() : fieldInExistingSchema.doc()); } fields.add(fieldBuilder.build()); } - for (Schema.Field f2: s2.getFields()) { - if (s1.getField(f2.name()) == null) { - fields.add(deepCopySchemaField(f2).build()); + for (Schema.Field fieldInExistingSchema: existingSchema.getFields()) { + if (newSchema.getField(fieldInExistingSchema.name()) == null) { + fields.add(deepCopySchemaField(fieldInExistingSchema).build()); } } return fields; diff --git a/internal/venice-client-common/src/test/java/com/linkedin/venice/schema/TestAvroSupersetSchemaUtils.java b/internal/venice-client-common/src/test/java/com/linkedin/venice/schema/TestAvroSupersetSchemaUtils.java index ed8766d851..cdcc39d553 100644 --- a/internal/venice-client-common/src/test/java/com/linkedin/venice/schema/TestAvroSupersetSchemaUtils.java +++ b/internal/venice-client-common/src/test/java/com/linkedin/venice/schema/TestAvroSupersetSchemaUtils.java @@ -6,6 +6,7 @@ import static com.linkedin.venice.utils.TestWriteUtils.NAME_RECORD_V4_SCHEMA; import static com.linkedin.venice.utils.TestWriteUtils.NAME_RECORD_V5_SCHEMA; import static com.linkedin.venice.utils.TestWriteUtils.NAME_RECORD_V6_SCHEMA; +import static com.linkedin.venice.utils.TestWriteUtils.loadFileAsString; import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; import com.linkedin.venice.controllerapi.MultiSchemaResponse; @@ -36,7 +37,7 @@ public void testGenerateSupersetSchemaFromValueSchemasWithTwoSchemas() { AvroSchemaUtils.generateSupersetSchemaFromAllValueSchemas(Arrays.asList(schemaEntry1, schemaEntry2)); final Schema expectedSupersetSchema = - AvroSupersetSchemaUtils.generateSuperSetSchema(schemaEntry1.getSchema(), schemaEntry2.getSchema()); + AvroSupersetSchemaUtils.generateSupersetSchema(schemaEntry1.getSchema(), schemaEntry2.getSchema()); Assert.assertTrue( AvroSchemaUtils.compareSchemaIgnoreFieldOrder(expectedSupersetSchema, supersetSchemaEntry.getSchema())); Assert.assertEquals(supersetSchemaEntry.getId(), 2); @@ -142,7 +143,7 @@ public void testSupersetSchemaDefaultCompatibility() { Schema newValueSchema = AvroSchemaParseUtils.parseSchemaFromJSONStrictValidation(valueSchemaStr1); Schema existingValueSchema = AvroSchemaParseUtils.parseSchemaFromJSONStrictValidation(valueSchemaStr2); - Schema newSuperSetSchema = AvroSupersetSchemaUtils.generateSuperSetSchema(existingValueSchema, newValueSchema); + Schema newSuperSetSchema = AvroSupersetSchemaUtils.generateSupersetSchema(existingValueSchema, newValueSchema); Assert.assertTrue( new SchemaEntry(1, valueSchemaStr2) .isNewSchemaCompatible(new SchemaEntry(2, newSuperSetSchema), DirectionalSchemaCompatibilityType.FULL)); @@ -161,7 +162,7 @@ public void testStringVsAvroString() { Assert.assertNotEquals(s1, s2); Assert.assertTrue(AvroSchemaUtils.compareSchemaIgnoreFieldOrder(s1, s2)); - Schema s3 = AvroSupersetSchemaUtils.generateSuperSetSchema(s2, s1); + Schema s3 = AvroSupersetSchemaUtils.generateSupersetSchema(s2, s1); Assert.assertNotNull(s3); Assert.assertNotNull( AvroCompatibilityHelper.getSchemaPropAsJsonString(s3.getField("name").schema(), "avro.java.string")); @@ -177,7 +178,7 @@ public void testWithDifferentDocField() { Schema s2 = AvroSchemaParseUtils.parseSchemaFromJSONStrictValidation(schemaStr2); Assert.assertTrue(AvroSchemaUtils.compareSchemaIgnoreFieldOrder(s1, s2)); - Schema s3 = AvroSupersetSchemaUtils.generateSuperSetSchema(s1, s2); + Schema s3 = AvroSupersetSchemaUtils.generateSupersetSchema(s1, s2); Assert.assertNotNull(s3); } @@ -191,7 +192,7 @@ public void testSchemaMerge() { Schema s1 = AvroSchemaParseUtils.parseSchemaFromJSONStrictValidation(schemaStr1); Schema s2 = AvroSchemaParseUtils.parseSchemaFromJSONStrictValidation(schemaStr2); Assert.assertFalse(AvroSchemaUtils.compareSchemaIgnoreFieldOrder(s1, s2)); - Schema s3 = AvroSupersetSchemaUtils.generateSuperSetSchema(s1, s2); + Schema s3 = AvroSupersetSchemaUtils.generateSupersetSchema(s1, s2); Assert.assertNotNull(s3); } @@ -206,7 +207,7 @@ public void testSchemaMergeFields() { Schema s2 = AvroSchemaParseUtils.parseSchemaFromJSONStrictValidation(schemaStr2); Assert.assertFalse(AvroSchemaUtils.compareSchemaIgnoreFieldOrder(s1, s2)); - Schema s3 = AvroSupersetSchemaUtils.generateSuperSetSchema(s1, s2); + Schema s3 = AvroSupersetSchemaUtils.generateSupersetSchema(s1, s2); Assert.assertNotNull(s3.getField("id1")); Assert.assertNotNull(s3.getField("id2")); } @@ -222,7 +223,7 @@ public void testSchemaMergeFieldsBadDefaults() { Schema s2 = AvroSchemaParseUtils.parseSchemaFromJSONStrictValidation(schemaStr2); Assert.assertFalse(AvroSchemaUtils.compareSchemaIgnoreFieldOrder(s1, s2)); - Schema s3 = AvroSupersetSchemaUtils.generateSuperSetSchema(s1, s2); + Schema s3 = AvroSupersetSchemaUtils.generateSupersetSchema(s1, s2); Assert.assertNotNull(s3.getField("id1")); Assert.assertNotNull(s3.getField("id2")); } @@ -237,7 +238,7 @@ public void testWithIncompatibleSchema() { Schema s2 = AvroSchemaParseUtils.parseSchemaFromJSONStrictValidation(schemaStr2); Assert.assertFalse(AvroSchemaUtils.compareSchemaIgnoreFieldOrder(s1, s2)); - AvroSupersetSchemaUtils.generateSuperSetSchema(s1, s2); + AvroSupersetSchemaUtils.generateSupersetSchema(s1, s2); } @Test @@ -251,11 +252,26 @@ public void testSchemaMergeUnion() { Schema s2 = AvroSchemaParseUtils.parseSchemaFromJSONStrictValidation(schemaStr2); Assert.assertFalse(AvroSchemaUtils.compareSchemaIgnoreFieldOrder(s1, s2)); - Schema s3 = AvroSupersetSchemaUtils.generateSuperSetSchema(s1, s2); + Schema s3 = AvroSupersetSchemaUtils.generateSupersetSchema(s1, s2); Assert.assertNotNull(s3.getField("company")); Assert.assertNotNull(s3.getField("organization")); } + @Test + public void testSchemaMergeUnionWithComplexItemType() { + Schema s1 = AvroSchemaParseUtils.parseSchemaFromJSONStrictValidation(loadFileAsString("UnionV1.avsc")); + Schema s2 = AvroSchemaParseUtils.parseSchemaFromJSONStrictValidation(loadFileAsString("UnionV2.avsc")); + Assert.assertFalse(AvroSchemaUtils.compareSchemaIgnoreFieldOrder(s1, s2)); + Schema s3 = AvroSupersetSchemaUtils.generateSupersetSchema(s1, s2); + Assert.assertNotNull(s3.getField("age")); + Assert.assertNotNull(s3.getField("field")); + Schema.Field subFieldInS2 = s2.getField("field"); + Schema.Field subFieldInS3 = s3.getField("field"); + Schema unionSubFieldInS2 = subFieldInS2.schema().getTypes().get(1); + Schema unionSubFieldInS3 = subFieldInS3.schema().getTypes().get(1); + Assert.assertEquals(unionSubFieldInS3, unionSubFieldInS2); + } + @Test public void testWithNewFieldArrayRecord() { String recordSchemaStr1 = "{\n" + " \"type\" : \"record\",\n" + " \"name\" : \"testRecord\",\n" @@ -280,7 +296,7 @@ public void testWithNewFieldArrayRecord() { Schema s2 = AvroSchemaParseUtils.parseSchemaFromJSONStrictValidation(recordSchemaStr2); Assert.assertFalse(AvroSchemaUtils.compareSchemaIgnoreFieldOrder(s1, s2)); - Schema s3 = AvroSupersetSchemaUtils.generateSuperSetSchema(s1, s2); + Schema s3 = AvroSupersetSchemaUtils.generateSupersetSchema(s1, s2); Assert.assertNotNull(s3); } @@ -309,7 +325,7 @@ public void tesMergeWithDefaultValueUpdate() { Schema s2 = AvroSchemaParseUtils.parseSchemaFromJSONStrictValidation(schemaStr2); Assert.assertTrue(AvroSchemaUtils.compareSchemaIgnoreFieldOrder(s1, s2)); - Schema s3 = AvroSupersetSchemaUtils.generateSuperSetSchema(s1, s2); + Schema s3 = AvroSupersetSchemaUtils.generateSupersetSchema(s1, s2); Assert.assertNotNull(AvroSchemaUtils.getFieldDefault(s3.getField("salary"))); } @@ -333,7 +349,7 @@ public void testWithEnumEvolution() { Schema s2 = AvroSchemaParseUtils.parseSchemaFromJSONStrictValidation(schemaStr2); Assert.assertFalse(AvroSchemaUtils.compareSchemaIgnoreFieldOrder(s1, s2)); - AvroSupersetSchemaUtils.generateSuperSetSchema(s1, s2); + AvroSupersetSchemaUtils.generateSupersetSchema(s1, s2); } @Test @@ -425,8 +441,7 @@ public void testSupersetSchemaContainsMergeFieldProps() { Schema schema1 = AvroSchemaParseUtils.parseSchemaFromJSONStrictValidation(valueSchemaStr1); Schema schema2 = AvroSchemaParseUtils.parseSchemaFromJSONStrictValidation(valueSchemaStr2); - - Schema supersetSchema = AvroSupersetSchemaUtils.generateSuperSetSchema(schema1, schema2); + Schema supersetSchema = AvroSupersetSchemaUtils.generateSupersetSchema(schema1, schema2); Schema.Field intField = supersetSchema.getField("int_field"); Schema.Field stringField = supersetSchema.getField("string_field"); @@ -496,7 +511,7 @@ public void testValidateSubsetSchema() { Assert.assertNotEquals(NAME_RECORD_V5_SCHEMA, NAME_RECORD_V6_SCHEMA); // Test validation skip comparing props when checking for subset schema. Schema supersetSchemaForV5AndV4 = - AvroSupersetSchemaUtils.generateSuperSetSchema(NAME_RECORD_V5_SCHEMA, NAME_RECORD_V4_SCHEMA); + AvroSupersetSchemaUtils.generateSupersetSchema(NAME_RECORD_V5_SCHEMA, NAME_RECORD_V4_SCHEMA); Assert.assertTrue( AvroSupersetSchemaUtils.validateSubsetValueSchema(NAME_RECORD_V5_SCHEMA, supersetSchemaForV5AndV4.toString())); Assert.assertTrue( diff --git a/internal/venice-client-common/src/test/resources/UnionV1.avsc b/internal/venice-client-common/src/test/resources/UnionV1.avsc new file mode 100644 index 0000000000..81cd2b7006 --- /dev/null +++ b/internal/venice-client-common/src/test/resources/UnionV1.avsc @@ -0,0 +1,31 @@ +{ + "type" : "record", + "name" : "User", + "namespace" : "example.avro", + "fields" : [ + { + "name": "field", + "type": [ + "int", + { + "type": "record", + "name": "subField", + "fields": [ + { + "name": "name", + "type": "string", + "doc": "doc v1", + "default": "v1" + } + ] + } + ], + "default": 10 + }, + { + "name": "age", + "type": "int", + "default": 10 + } + ] +} diff --git a/internal/venice-client-common/src/test/resources/UnionV2.avsc b/internal/venice-client-common/src/test/resources/UnionV2.avsc new file mode 100644 index 0000000000..05288cf0d4 --- /dev/null +++ b/internal/venice-client-common/src/test/resources/UnionV2.avsc @@ -0,0 +1,26 @@ +{ + "type" : "record", + "name" : "User", + "namespace" : "example.avro", + "fields" : [ + { + "name": "field", + "type": [ + "int", + { + "type": "record", + "name": "subField", + "fields": [ + { + "name": "name", + "type": "string", + "doc": "doc v2", + "default": "v2" + } + ] + } + ], + "default": 20 + } + ] +} diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/VeniceParentHelixAdminTest.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/VeniceParentHelixAdminTest.java index 7ca1f0a7d6..af14b0eb21 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/VeniceParentHelixAdminTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/VeniceParentHelixAdminTest.java @@ -598,6 +598,7 @@ public void testStoreMetaDataUpdateFromParentToChildController( testWriteComputeSchemaAutoGeneration(parentControllerClient); testWriteComputeSchemaEnable(parentControllerClient); testWriteComputeSchemaAutoGenerationFailure(parentControllerClient); + testSupersetSchemaGenerationWithUpdateDefaultValue(parentControllerClient); testUpdateConfigs(parentControllerClient, childControllerClient); } } @@ -1132,6 +1133,48 @@ private void testWriteComputeSchemaEnable(ControllerClient parentControllerClien Assert.assertEquals(registeredWriteComputeSchema.size(), 1); } + private void testSupersetSchemaGenerationWithUpdateDefaultValue(ControllerClient parentControllerClient) { + String storeName = Utils.getUniqueString("test_store"); + String owner = "test_owner"; + String keySchemaStr = "\"long\""; + + // Step 1. Create a store with missing default fields schema + parentControllerClient + .createNewStore(storeName, owner, keySchemaStr, TestWriteUtils.UNION_RECORD_V1_SCHEMA.toString()); + MultiSchemaResponse valueAndWriteComputeSchemaResponse = + parentControllerClient.getAllValueAndDerivedSchema(storeName); + MultiSchemaResponse.Schema[] registeredSchemas = valueAndWriteComputeSchemaResponse.getSchemas(); + Assert.assertEquals(registeredSchemas.length, 1); + MultiSchemaResponse.Schema registeredSchema = registeredSchemas[0]; + Assert.assertFalse(registeredSchema.isDerivedSchema()); // No write compute schema yet. + + // Step 2. Update this store to enable write compute. + UpdateStoreQueryParams updateStoreQueryParams = new UpdateStoreQueryParams(); + updateStoreQueryParams.setWriteComputationEnabled(true); + parentControllerClient.updateStore(storeName, updateStoreQueryParams); + + // Could not enable write compute bad schema did not have defaults + StoreInfo store = parentControllerClient.getStore(storeName).getStore(); + Assert.assertTrue(store.isWriteComputationEnabled()); + Assert.assertEquals(store.getLatestSuperSetValueSchemaId(), 1); + + // Step 3. Add a valid latest value schema for write-compute + parentControllerClient.addValueSchema(storeName, TestWriteUtils.UNION_RECORD_V2_SCHEMA.toString()); + TestUtils.waitForNonDeterministicAssertion( + 30, + TimeUnit.SECONDS, + () -> Assert + .assertEquals(parentControllerClient.getStore(storeName).getStore().getLatestSuperSetValueSchemaId(), 2)); + + parentControllerClient.addValueSchema(storeName, TestWriteUtils.UNION_RECORD_V3_SCHEMA.toString()); + TestUtils.waitForNonDeterministicAssertion( + 30, + TimeUnit.SECONDS, + () -> Assert + .assertEquals(parentControllerClient.getStore(storeName).getStore().getLatestSuperSetValueSchemaId(), 3)); + + } + private List getWriteComputeSchemaStrs(MultiSchemaResponse.Schema[] registeredSchemas) { List writeComputeSchemaStrs = new ArrayList<>(); for (MultiSchemaResponse.Schema schema: registeredSchemas) { diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestEmptyPush.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestEmptyPush.java index 3a5db98724..15ae067ee4 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestEmptyPush.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/TestEmptyPush.java @@ -136,7 +136,8 @@ public void testEmptyPushByChangingCompressionStrategyForHybridStore() throws IO PubSubTopic storeRealTimeTopic = venice.getPubSubTopicRepository().getTopic(Version.composeRealTimeTopic(storeName)); assertTrue(topicManager.containsTopicAndAllPartitionsAreOnline(storeRealTimeTopic)); - + // One time refresh of router metadata. + venice.refreshAllRouterMetaData(); // Start writing some real-time records SystemProducer veniceProducer = IntegrationTestPushUtils.getSamzaProducer(venice, storeName, Version.PushType.STREAM); diff --git a/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/TestWriteUtils.java b/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/TestWriteUtils.java index 16ef0b5a48..62280a1dae 100644 --- a/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/TestWriteUtils.java +++ b/internal/venice-test-common/src/main/java/com/linkedin/venice/utils/TestWriteUtils.java @@ -100,6 +100,13 @@ public class TestWriteUtils { public static final Schema NAME_RECORD_V6_SCHEMA = AvroCompatibilityHelper.parse(loadSchemaFileFromResource("valueSchema/NameV6.avsc")); + public static final Schema UNION_RECORD_V1_SCHEMA = + AvroCompatibilityHelper.parse(loadSchemaFileFromResource("valueSchema/UnionV1.avsc")); + public static final Schema UNION_RECORD_V2_SCHEMA = + AvroCompatibilityHelper.parse(loadSchemaFileFromResource("valueSchema/UnionV2.avsc")); + public static final Schema UNION_RECORD_V3_SCHEMA = + AvroCompatibilityHelper.parse(loadSchemaFileFromResource("valueSchema/UnionV3.avsc")); + // ETL Schema public static final Schema ETL_KEY_SCHEMA = AvroCompatibilityHelper.parse(loadSchemaFileFromResource("etl/Key.avsc")); public static final Schema ETL_VALUE_SCHEMA = diff --git a/internal/venice-test-common/src/main/resources/valueSchema/UnionV1.avsc b/internal/venice-test-common/src/main/resources/valueSchema/UnionV1.avsc new file mode 100644 index 0000000000..1dbc85a391 --- /dev/null +++ b/internal/venice-test-common/src/main/resources/valueSchema/UnionV1.avsc @@ -0,0 +1,15 @@ +{ + "type" : "record", + "name" : "User", + "namespace" : "example.avro", + "fields" : [ + { + "name": "count", + "type": [ + "int", + "null" + ], + "default": 0 + } + ] +} diff --git a/internal/venice-test-common/src/main/resources/valueSchema/UnionV2.avsc b/internal/venice-test-common/src/main/resources/valueSchema/UnionV2.avsc new file mode 100644 index 0000000000..e1b349247e --- /dev/null +++ b/internal/venice-test-common/src/main/resources/valueSchema/UnionV2.avsc @@ -0,0 +1,15 @@ +{ + "type" : "record", + "name" : "User", + "namespace" : "example.avro", + "fields" : [ + { + "name": "count", + "type": [ + "null", + "int" + ], + "default": null + } + ] +} diff --git a/internal/venice-test-common/src/main/resources/valueSchema/UnionV3.avsc b/internal/venice-test-common/src/main/resources/valueSchema/UnionV3.avsc new file mode 100644 index 0000000000..0912be4c31 --- /dev/null +++ b/internal/venice-test-common/src/main/resources/valueSchema/UnionV3.avsc @@ -0,0 +1,20 @@ +{ + "type" : "record", + "name" : "User", + "namespace" : "example.avro", + "fields" : [ + { + "name": "count", + "type": [ + "null", + "int" + ], + "default": null + }, + { + "name": "dummyField", + "type" : "string", + "default" : "" + } + ] +} diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceParentHelixAdmin.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceParentHelixAdmin.java index aaf3fa1ed8..32331b5dea 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceParentHelixAdmin.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceParentHelixAdmin.java @@ -3097,7 +3097,6 @@ public SchemaEntry addValueSchema( SupersetSchemaGenerator supersetSchemaGenerator = getSupersetSchemaGenerator(clusterName); Schema newSuperSetSchema = supersetSchemaGenerator.generateSupersetSchema(existingValueSchema, newValueSchema); String newSuperSetSchemaStr = newSuperSetSchema.toString(); - if (supersetSchemaGenerator.compareSchema(newSuperSetSchema, newValueSchema)) { doUpdateSupersetSchemaID = true; @@ -3143,7 +3142,6 @@ public SchemaEntry addValueSchema( } else { doUpdateSupersetSchemaID = false; } - SchemaEntry addedSchemaEntry = addValueSchemaEntry(clusterName, storeName, newValueSchemaStr, schemaId, doUpdateSupersetSchemaID); diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/supersetschema/DefaultSupersetSchemaGenerator.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/supersetschema/DefaultSupersetSchemaGenerator.java index 0696a37fc9..0a5557aaf3 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/supersetschema/DefaultSupersetSchemaGenerator.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/supersetschema/DefaultSupersetSchemaGenerator.java @@ -20,6 +20,6 @@ public boolean compareSchema(Schema s1, Schema s2) { @Override public Schema generateSupersetSchema(Schema existingSchema, Schema newSchema) { - return AvroSupersetSchemaUtils.generateSuperSetSchema(existingSchema, newSchema); + return AvroSupersetSchemaUtils.generateSupersetSchema(existingSchema, newSchema); } } diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/supersetschema/SupersetSchemaGeneratorWithCustomProp.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/supersetschema/SupersetSchemaGeneratorWithCustomProp.java index 92519628b5..f29fcd612c 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/supersetschema/SupersetSchemaGeneratorWithCustomProp.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/supersetschema/SupersetSchemaGeneratorWithCustomProp.java @@ -76,7 +76,7 @@ public boolean compareSchema(Schema s1, Schema s2) { @Override public Schema generateSupersetSchema(Schema existingSchema, Schema newSchema) { - Schema supersetSchema = AvroSupersetSchemaUtils.generateSuperSetSchema(existingSchema, newSchema); + Schema supersetSchema = AvroSupersetSchemaUtils.generateSupersetSchema(existingSchema, newSchema); String customPropInNewSchema = newSchema.getProp(customProp); if (customPropInNewSchema != null && supersetSchema.getProp(customProp) == null) { Schema newSupersetSchema = AvroSchemaParseUtils.parseSchemaFromJSONLooseValidation(supersetSchema.toString()); diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/systemstore/SystemStoreRepairTask.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/systemstore/SystemStoreRepairTask.java index dff7e87124..0524106839 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/systemstore/SystemStoreRepairTask.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/systemstore/SystemStoreRepairTask.java @@ -224,7 +224,6 @@ void checkHeartbeatFromSystemStores( } long retrievedHeartbeatTimestamp = getHeartbeatFromSystemStore(clusterName, entry.getKey()); - LOGGER.info("DEBUGGING: {} {} {}", entry.getKey(), entry.getValue(), retrievedHeartbeatTimestamp); if (retrievedHeartbeatTimestamp < entry.getValue()) { newUnhealthySystemStoreSet.add(entry.getKey()); if (retrievedHeartbeatTimestamp == -1) { From c694d96e54459a511b6ad1a52f0f83dacfd98591 Mon Sep 17 00:00:00 2001 From: Koorous Vargha Date: Thu, 26 Sep 2024 19:03:12 -0700 Subject: [PATCH 16/16] [controller] Ensure empty string from config doesn't result in an array of an empty string (#1202) --- .../venice/utils/VeniceProperties.java | 1 + .../venice/controller/TestHAASController.java | 21 +++++++++++++++++++ .../VeniceControllerClusterConfig.java | 8 ++++++- 3 files changed, 29 insertions(+), 1 deletion(-) diff --git a/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/VeniceProperties.java b/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/VeniceProperties.java index dfafe3ef15..e9855ece93 100644 --- a/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/VeniceProperties.java +++ b/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/VeniceProperties.java @@ -389,6 +389,7 @@ public List getList(String key, List defaultValue) { } String value = get(key); + String[] pieces = value.split("\\s*,\\s*"); return Arrays.asList(pieces); } diff --git a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestHAASController.java b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestHAASController.java index 796edb2aa2..905a853693 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestHAASController.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/controller/TestHAASController.java @@ -78,6 +78,27 @@ public void testClusterResourceInstanceTag() { } } + @Test(timeOut = 60 * Time.MS_PER_SECOND) + public void testClusterResourceEmptyInstanceTag() { + try (VeniceClusterWrapper venice = ServiceFactory.getVeniceCluster(0, 0, 0, 1); + HelixAsAServiceWrapper helixAsAServiceWrapper = startAndWaitForHAASToBeAvailable(venice.getZk().getAddress())) { + String instanceTag = ""; + String controllerClusterName = "venice-controllers"; + + Properties clusterProperties = (Properties) enableControllerAndStorageClusterHAASProperties.clone(); + clusterProperties.put(ConfigKeys.CONTROLLER_RESOURCE_INSTANCE_GROUP_TAG, instanceTag); + clusterProperties.put(ConfigKeys.CONTROLLER_INSTANCE_TAG_LIST, instanceTag); + + VeniceControllerWrapper controllerWrapper = venice.addVeniceController(clusterProperties); + + HelixAdmin helixAdmin = controllerWrapper.getVeniceHelixAdmin().getHelixAdmin(); + List resources = helixAdmin.getResourcesInClusterWithTag(controllerClusterName, instanceTag); + assertEquals(resources.size(), 0); + List instances = helixAdmin.getInstancesInClusterWithTag(controllerClusterName, instanceTag); + assertEquals(instances.size(), 0); + } + } + @Test(timeOut = 60 * Time.MS_PER_SECOND) public void testStartHAASHelixControllerAsControllerClusterLeader() { try (VeniceClusterWrapper venice = ServiceFactory.getVeniceCluster(0, 0, 0, 1); diff --git a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java index e0c1881e05..7d390d3b99 100644 --- a/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java +++ b/services/venice-controller/src/main/java/com/linkedin/venice/controller/VeniceControllerClusterConfig.java @@ -654,7 +654,13 @@ public VeniceControllerClusterConfig(VeniceProperties props) { this.adminCheckReadMethodForKafka = props.getBoolean(ADMIN_CHECK_READ_METHOD_FOR_KAFKA, true); this.controllerClusterName = props.getString(CONTROLLER_CLUSTER, "venice-controllers"); this.controllerResourceInstanceGroupTag = props.getString(CONTROLLER_RESOURCE_INSTANCE_GROUP_TAG, ""); - this.controllerInstanceTagList = props.getList(CONTROLLER_INSTANCE_TAG_LIST, Collections.emptyList()); + + if (props.getString(CONTROLLER_INSTANCE_TAG_LIST, "").isEmpty()) { + this.controllerInstanceTagList = Collections.emptyList(); + } else { + this.controllerInstanceTagList = props.getList(CONTROLLER_INSTANCE_TAG_LIST, Collections.emptyList()); + } + this.controllerClusterReplica = props.getInt(CONTROLLER_CLUSTER_REPLICA, 3); this.controllerClusterZkAddress = props.getString(CONTROLLER_CLUSTER_ZK_ADDRESSS, getZkAddress()); this.parent = props.getBoolean(CONTROLLER_PARENT_MODE, false);