diff --git a/build.gradle b/build.gradle index 57a7d3a75f5..82c5b5cd24d 100644 --- a/build.gradle +++ b/build.gradle @@ -50,9 +50,13 @@ def pulsarGroup = 'org.apache.pulsar' def pulsarVersion = '2.10.3' def alpnAgentVersion = '2.0.10' def hadoopVersion = '2.10.2' +def apacheSparkVersion = '3.1.3' ext.libraries = [ alpnAgent: "org.mortbay.jetty.alpn:jetty-alpn-agent:${alpnAgentVersion}", + apacheSparkAvro: "org.apache.spark:spark-avro_2.12:${apacheSparkVersion}", + apacheSparkCore: "org.apache.spark:spark-core_2.12:${apacheSparkVersion}", + apacheSparkSql: "org.apache.spark:spark-sql_2.12:${apacheSparkVersion}", avro: "org.apache.avro:avro:${avroVersion}", avroCompiler: "org.apache.avro:avro-compiler:${avroVersion}", avroMapred: "org.apache.avro:avro-mapred:${avroVersion}", @@ -368,6 +372,18 @@ subprojects { tasks.withType(Test) { mustRunAfter tasks.withType(SpotBugsTask) + // For Spark to run on Java 17 + jvmArgs "-XX:+IgnoreUnrecognizedVMOptions" + jvmArgs "--add-opens=java.base/java.nio=ALL-UNNAMED" + jvmArgs "--add-opens=java.base/sun.nio.ch=ALL-UNNAMED" + jvmArgs "--add-opens=java.base/java.lang=ALL-UNNAMED" + jvmArgs "--add-opens=java.base/java.lang.invoke=ALL-UNNAMED" + jvmArgs "--add-opens=java.base/jdk.internal.misc=ALL-UNNAMED" + + if (JavaVersion.current() == JavaVersion.VERSION_11) { + jvmArgs "-Dio.netty.tryReflectionSetAccessible=true" + } + forkEvery = Integer.valueOf(System.getProperty('forkEvery', '0')) maxParallelForks = Integer.valueOf(System.getProperty('maxParallelForks', '4')) minHeapSize = System.getProperty('minHeapSize', '1g') diff --git a/clients/venice-push-job/build.gradle b/clients/venice-push-job/build.gradle index 94b70f8389c..c9d31388b1d 100644 --- a/clients/venice-push-job/build.gradle +++ b/clients/venice-push-job/build.gradle @@ -27,7 +27,35 @@ dependencies { exclude group: 'javax.servlet' } - implementation project(':clients:venice-thin-client') // Needed by the Post Bulk-load Analysis Job + 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. + exclude group: 'org.apache.avro', module: 'avro-mapred' + + // Spark 3.3 depends on hadoop-client-runtime and hadoop-client-api, which are shaded jars that were added in Hadoop 3.0.3 + exclude group: 'org.apache.hadoop', module: 'hadoop-client-runtime' + exclude group: 'org.apache.hadoop', module: 'hadoop-client-api' + } + implementation (libraries.apacheSparkCore) { + // 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. + exclude group: 'org.apache.avro', module: 'avro-mapred' + + // Spark 3.3 depends on hadoop-client-runtime and hadoop-client-api, which are shaded jars that were added in Hadoop 3.0.3 + exclude group: 'org.apache.hadoop', module: 'hadoop-client-runtime' + exclude group: 'org.apache.hadoop', module: 'hadoop-client-api' + } + implementation (libraries.apacheSparkSql) { + // 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. + exclude group: 'org.apache.avro', module: 'avro-mapred' + + // Spark 3.3 depends on hadoop-client-runtime and hadoop-client-api, which are shaded jars that were added in Hadoop 3.0.3 + exclude group: 'org.apache.hadoop', module: 'hadoop-client-runtime' + exclude group: 'org.apache.hadoop', module: 'hadoop-client-api' + } + + implementation project(':clients:venice-thin-client') // Needed by the KME SchemaReader implementation libraries.commonsIo implementation libraries.fastUtil 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 2119e5c5f48..ad4940085cc 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 @@ -59,7 +59,7 @@ public class DefaultInputDataInfoProvider implements InputDataInfoProvider { */ private final Lazy hdfsExecutorService; - DefaultInputDataInfoProvider(PushJobSetting pushJobSetting, VeniceProperties props) { + public DefaultInputDataInfoProvider(PushJobSetting pushJobSetting, VeniceProperties props) { this.pushJobSetting = pushJobSetting; this.props = props; this.hdfsExecutorService = diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/InputDataInfoProvider.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/InputDataInfoProvider.java index 6e038401e7f..ee013b9e20d 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/InputDataInfoProvider.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/InputDataInfoProvider.java @@ -16,7 +16,6 @@ public interface InputDataInfoProvider extends Closeable { /** * A POJO that contains input data information (schema information and input data file size) */ - Logger LOGGER = LogManager.getLogger(InputDataInfoProvider.class); class InputDataInfo { private long inputFileDataSizeInBytes; @@ -24,11 +23,15 @@ class InputDataInfo { private final boolean hasRecords; private final long inputModificationTime; - InputDataInfo(long inputFileDataSizeInBytes, int numInputFiles, boolean hasRecords, long inputModificationTime) { + public InputDataInfo( + long inputFileDataSizeInBytes, + int numInputFiles, + boolean hasRecords, + long inputModificationTime) { this(inputFileDataSizeInBytes, numInputFiles, hasRecords, inputModificationTime, true); } - InputDataInfo( + public InputDataInfo( long inputFileDataSizeInBytes, int numInputFiles, boolean hasRecords, @@ -78,6 +81,9 @@ public long getInputModificationTime() { * @param recordIterator The data accessor of input records. */ static void loadZstdTrainingSamples(VeniceRecordIterator recordIterator, PushJobZstdConfig pushJobZstdConfig) { + // It's preferable to make this as "private static final" in the class-level, but it's not possible due to + // "InputDataInfoProvider" being an interface. + final Logger logger = LogManager.getLogger(InputDataInfo.class); int fileSampleSize = 0; while (recordIterator.next()) { if (recordIterator.getCurrentKey() == null) { @@ -93,7 +99,7 @@ static void loadZstdTrainingSamples(VeniceRecordIterator recordIterator, PushJob // At least 1 sample per file should be added until the max sample size is reached if (fileSampleSize > 0) { if (fileSampleSize + value.length > pushJobZstdConfig.getMaxBytesPerFile()) { - LOGGER.debug( + logger.debug( "Read {} to build dictionary. Reached limit per file of {}.", ByteUtils.generateHumanReadableByteCountString(fileSampleSize), ByteUtils.generateHumanReadableByteCountString(pushJobZstdConfig.getMaxBytesPerFile())); @@ -103,7 +109,7 @@ static void loadZstdTrainingSamples(VeniceRecordIterator recordIterator, PushJob // addSample returns false when the data read no longer fits in the 'sample' buffer limit if (!pushJobZstdConfig.getZstdDictTrainer().addSample(value)) { - LOGGER.debug( + logger.debug( "Read {} to build dictionary. Reached sample limit of {}.", ByteUtils.generateHumanReadableByteCountString(fileSampleSize), ByteUtils.generateHumanReadableByteCountString(pushJobZstdConfig.getMaxSampleSize())); @@ -114,7 +120,7 @@ static void loadZstdTrainingSamples(VeniceRecordIterator recordIterator, PushJob pushJobZstdConfig.incrCollectedNumberOfSamples(); } - LOGGER.debug( + logger.debug( "Read {} to build dictionary. Reached EOF.", ByteUtils.generateHumanReadableByteCountString(fileSampleSize)); } 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 310904269b5..5359c7fe407 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 @@ -236,7 +236,7 @@ public class VenicePushJob implements AutoCloseable { AvroProtocolDefinition.PUSH_JOB_DETAILS.getSerializer(); private InputStorageQuotaTracker inputStorageQuotaTracker; - private PushJobHeartbeatSenderFactory pushJobHeartbeatSenderFactory; + private final PushJobHeartbeatSenderFactory pushJobHeartbeatSenderFactory; private boolean pushJobStatusUploadDisabledHasBeenLogged = false; /** @@ -246,7 +246,7 @@ public class VenicePushJob implements AutoCloseable { * 3. Negative enums are error scenarios (Can be user or system errors) */ public enum PushJobCheckpoints { - INITIALIZE_PUSH_JOB(0), NEW_VERSION_CREATED(1), START_MAP_REDUCE_JOB(2), MAP_REDUCE_JOB_COMPLETED(3), + 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), @@ -1003,34 +1003,38 @@ private void validateInputDataSchema(String inputDataSchemaString) { } void runJobAndUpdateStatus() { - updatePushJobDetailsWithCheckpoint(PushJobCheckpoints.START_MAP_REDUCE_JOB); - LOGGER.info("Configuring data writer job"); - dataWriterComputeJob = getDataWriterComputeJob(); - dataWriterComputeJob.configure(props, pushJobSetting); - LOGGER.info("Triggering data writer job"); - dataWriterComputeJob.runJob(); - if (dataWriterComputeJob.getStatus() != ComputeJob.Status.SUCCEEDED) { - if (!pushJobSetting.isSourceKafka) { - try { - checkLastModificationTimeAndLog(); - } catch (IOException e) { - LOGGER.warn("Failed to check last modification time of input file", e); + try { + updatePushJobDetailsWithCheckpoint(PushJobCheckpoints.START_DATA_WRITER_JOB); + LOGGER.info("Configuring data writer job"); + dataWriterComputeJob = getDataWriterComputeJob(); + dataWriterComputeJob.configure(props, pushJobSetting); + LOGGER.info("Triggering data writer job"); + dataWriterComputeJob.runJob(); + if (dataWriterComputeJob.getStatus() != ComputeJob.Status.SUCCEEDED) { + if (!pushJobSetting.isSourceKafka) { + try { + checkLastModificationTimeAndLog(); + } catch (IOException e) { + LOGGER.warn("Failed to check last modification time of input file", e); + } + } + Throwable t = dataWriterComputeJob.getFailureReason(); + if (t == null) { + throw new VeniceException( + "Data writer job failed unexpectedly with status: " + dataWriterComputeJob.getStatus()); + } else { + throwVeniceException(t); } - } - Throwable t = dataWriterComputeJob.getFailureReason(); - if (t == null) { - throw new VeniceException( - "Data writer job failed unexpectedly with status: " + dataWriterComputeJob.getStatus()); } else { - throwVeniceException(t); - } - } else { - String errorMessage = updatePushJobDetailsWithJobDetails(dataWriterComputeJob.getTaskTracker()); - if (errorMessage != null) { - throw new VeniceException(errorMessage); + String errorMessage = updatePushJobDetailsWithJobDetails(dataWriterComputeJob.getTaskTracker()); + if (errorMessage != null) { + throw new VeniceException(errorMessage); + } } + updatePushJobDetailsWithCheckpoint(PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED); + } finally { + Utils.closeQuietlyWithErrorLogged(dataWriterComputeJob); } - updatePushJobDetailsWithCheckpoint(PushJobCheckpoints.MAP_REDUCE_JOB_COMPLETED); } private void runValidateSchemaAndBuildDictJobAndUpdateStatus(JobConf conf) throws Exception { @@ -1646,7 +1650,7 @@ private void updatePushJobDetailsWithDataWriterTracker() { // size of the Zstd with Dict compressed data pushJobDetails.totalZstdWithDictCompressedValueBytes = taskTracker.getTotalZstdCompressedValueSize(); LOGGER.info( - "pushJobDetails MR Counters: " + "\n\tTotal number of records: {} " + "\n\tSize of keys: {} " + "Data writer job summary: " + "\n\tTotal number of records: {} " + "\n\tSize of keys: {} " + "\n\tsize of uncompressed value: {} " + "\n\tConfigured value Compression Strategy: {} " + "\n\tFinal data size stored in Venice based on this compression strategy: {} " + "\n\tCompression Metrics collection is: {} ", diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/recordreader/avro/IdentityVeniceRecordReader.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/recordreader/avro/IdentityVeniceRecordReader.java new file mode 100644 index 00000000000..14dac2747e6 --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/input/recordreader/avro/IdentityVeniceRecordReader.java @@ -0,0 +1,44 @@ +package com.linkedin.venice.hadoop.input.recordreader.avro; + +import com.linkedin.venice.exceptions.VeniceUnsupportedOperationException; +import com.linkedin.venice.hadoop.input.recordreader.AbstractVeniceRecordReader; +import com.linkedin.venice.utils.ByteUtils; +import java.nio.ByteBuffer; +import org.apache.avro.Schema; + + +/** + * A record reader that returns the input key and value as is. + */ +public class IdentityVeniceRecordReader extends AbstractVeniceRecordReader { + private static final IdentityVeniceRecordReader INSTANCE = new IdentityVeniceRecordReader(); + + private IdentityVeniceRecordReader() { + final Schema BYTES_SCHEMA = Schema.create(Schema.Type.BYTES); + configure(BYTES_SCHEMA, BYTES_SCHEMA); + } + + public static IdentityVeniceRecordReader getInstance() { + return INSTANCE; + } + + @Override + public Object getAvroKey(ByteBuffer keyBytes, ByteBuffer valueBytes) { + throw new VeniceUnsupportedOperationException("getAvroKey in IdentityVeniceRecordReader"); + } + + @Override + public byte[] getKeyBytes(ByteBuffer keyBuffer, ByteBuffer valueBuffer) { + return ByteUtils.extractByteArray(keyBuffer); + } + + @Override + public Object getAvroValue(ByteBuffer keyBytes, ByteBuffer valueBytes) { + throw new VeniceUnsupportedOperationException("getAvroValue in IdentityVeniceRecordReader"); + } + + @Override + public byte[] getValueBytes(ByteBuffer keyBuffer, ByteBuffer valueBuffer) { + return ByteUtils.extractByteArray(valueBuffer); + } +} 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/hadoop/jobs/DataWriterComputeJob.java index 2aab45b6a80..b9db0435213 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/hadoop/jobs/DataWriterComputeJob.java @@ -1,9 +1,15 @@ package com.linkedin.venice.hadoop.jobs; +import com.linkedin.venice.ConfigKeys; import com.linkedin.venice.exceptions.VeniceException; import com.linkedin.venice.hadoop.PushJobSetting; +import com.linkedin.venice.hadoop.input.kafka.KafkaInputRecordReader; import com.linkedin.venice.hadoop.task.datawriter.DataWriterTaskTracker; import com.linkedin.venice.utils.VeniceProperties; +import com.linkedin.venice.writer.VeniceWriter; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -14,6 +20,20 @@ public abstract class DataWriterComputeJob implements ComputeJob { private static final Logger LOGGER = LogManager.getLogger(DataWriterComputeJob.class); + /** + * Pass-through the properties whose names start with: + * + **/ + public static final List PASS_THROUGH_CONFIG_PREFIXES = Collections.unmodifiableList( + Arrays.asList( + VeniceWriter.VENICE_WRITER_CONFIG_PREFIX, + ConfigKeys.KAFKA_CONFIG_PREFIX, + KafkaInputRecordReader.KIF_RECORD_READER_KAFKA_CONFIG_PREFIX)); + private Status jobStatus = Status.NOT_STARTED; private Throwable failureReason = null; @@ -98,7 +118,7 @@ public void configure(VeniceProperties properties) { LOGGER.warn("Data writer compute job needs additional configs to be configured."); } - public abstract PushJobSetting getPushJobSetting(); + protected abstract PushJobSetting getPushJobSetting(); public abstract void configure(VeniceProperties props, PushJobSetting pushJobSetting); 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 9946433e736..63fd937554c 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,6 +1,5 @@ package com.linkedin.venice.hadoop.mapreduce.common; -import static com.linkedin.venice.hadoop.InputDataInfoProvider.LOGGER; 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; @@ -21,9 +20,13 @@ import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.RunningJob; import org.apache.hadoop.mapred.lib.NullOutputFormat; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; public final class JobUtils { + private static final Logger LOGGER = LogManager.getLogger(JobUtils.class); + private JobUtils() { } 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 d31f456a2aa..12580c16def 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 @@ -3,7 +3,6 @@ import static com.linkedin.venice.CommonConfigKeys.SSL_FACTORY_CLASS_NAME; import static com.linkedin.venice.ConfigKeys.AMPLIFICATION_FACTOR; import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; -import static com.linkedin.venice.ConfigKeys.KAFKA_CONFIG_PREFIX; import static com.linkedin.venice.ConfigKeys.KAFKA_PRODUCER_DELIVERY_TIMEOUT_MS; import static com.linkedin.venice.ConfigKeys.KAFKA_PRODUCER_REQUEST_TIMEOUT_MS; import static com.linkedin.venice.ConfigKeys.KAFKA_PRODUCER_RETRIES_CONFIG; @@ -58,7 +57,6 @@ import com.linkedin.venice.hadoop.input.kafka.KafkaInputFormatCombiner; import com.linkedin.venice.hadoop.input.kafka.KafkaInputKeyComparator; import com.linkedin.venice.hadoop.input.kafka.KafkaInputMRPartitioner; -import com.linkedin.venice.hadoop.input.kafka.KafkaInputRecordReader; import com.linkedin.venice.hadoop.input.kafka.KafkaInputValueGroupingComparator; import com.linkedin.venice.hadoop.input.kafka.VeniceKafkaInputMapper; import com.linkedin.venice.hadoop.input.kafka.VeniceKafkaInputReducer; @@ -73,8 +71,6 @@ import com.linkedin.venice.utils.VeniceProperties; import com.linkedin.venice.writer.VeniceWriter; import java.io.IOException; -import java.util.Arrays; -import java.util.List; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.mapred.AvroInputFormat; @@ -124,9 +120,6 @@ private void setupDefaultJobConf(JobConf conf, PushJobSetting pushJobSetting, Ve pushJobSetting); conf.set(BATCH_NUM_BYTES_PROP, Integer.toString(pushJobSetting.batchNumBytes)); conf.set(TOPIC_PROP, pushJobSetting.topic); - // We need the two configs with bootstrap servers since VeniceWriterFactory requires kafka.bootstrap.servers while - // the Kafka consumer requires bootstrap.servers. - conf.set(KAFKA_BOOTSTRAP_SERVERS, pushJobSetting.kafkaUrl); conf.set(KAFKA_BOOTSTRAP_SERVERS, pushJobSetting.kafkaUrl); conf.set(PARTITIONER_CLASS, pushJobSetting.partitionerClass); // flatten partitionerParams since JobConf class does not support set an object @@ -158,9 +151,8 @@ private void setupDefaultJobConf(JobConf conf, PushJobSetting pushJobSetting, Ve conf.setBoolean(REPUSH_TTL_ENABLE, pushJobSetting.repushTTLEnabled); conf.setLong(REPUSH_TTL_START_TIMESTAMP, pushJobSetting.repushTTLStartTimeMs); if (pushJobSetting.repushTTLEnabled) { - conf.setInt(REPUSH_TTL_POLICY, TTLResolutionPolicy.RT_WRITE_ONLY.getValue()); // only support one policy - // thus not allow any value passed - // in. + // Currently, we only support one policy. Thus, we don't allow overriding it. + conf.setInt(REPUSH_TTL_POLICY, TTLResolutionPolicy.RT_WRITE_ONLY.getValue()); conf.set(RMD_SCHEMA_DIR, pushJobSetting.rmdSchemaDir); conf.set(VALUE_SCHEMA_DIR, pushJobSetting.valueSchemaDir); } @@ -212,46 +204,21 @@ private void setupDefaultJobConf(JobConf conf, PushJobSetting pushJobSetting, Ve conf.setBoolean(ZSTD_DICTIONARY_CREATION_SUCCESS, pushJobSetting.isZstdDictCreationSuccess); /** - * Pass-through the properties whose names start with: + * Override the configs following the rules: *
    - *
  • {@link VeniceWriter.VENICE_WRITER_CONFIG_PREFIX}
  • - *
  • {@link ApacheKafkaProducerConfig.KAFKA_CONFIG_PREFIX}
  • - *
  • {@link KafkaInputRecordReader.KIF_RECORD_READER_KAFKA_CONFIG_PREFIX}
  • + *
  • Pass-through the properties whose names start with the prefixes defined in {@link PASS_THROUGH_CONFIG_PREFIXES}.
  • + *
  • Override the properties that are specified with the {@link HADOOP_PREFIX} prefix.
  • *
- * - * Override the properties that are specified with the {@link HADOOP_PREFIX} prefix. **/ - List passThroughPrefixList = Arrays.asList( - VeniceWriter.VENICE_WRITER_CONFIG_PREFIX, - KAFKA_CONFIG_PREFIX, - KafkaInputRecordReader.KIF_RECORD_READER_KAFKA_CONFIG_PREFIX); - int passThroughPrefixListSize = passThroughPrefixList.size(); - /** - * The following logic will make sure there is no prefix that is a prefix of another prefix. - */ - for (int i = 0; i < passThroughPrefixListSize; ++i) { - for (int j = i + 1; j < passThroughPrefixListSize; ++j) { - String prefixI = passThroughPrefixList.get(i); - String prefixJ = passThroughPrefixList.get(j); - if (prefixI.startsWith(prefixJ)) { - throw new VeniceException("Prefix: " + prefixJ + " shouldn't be a prefix of another prefix: " + prefixI); - } - - if (prefixJ.startsWith(prefixI)) { - throw new VeniceException("Prefix: " + prefixI + " shouldn't be a prefix of another prefix: " + prefixJ); - } - } - } - - for (String key: props.keySet()) { - String lowerCase = key.toLowerCase(); - if (lowerCase.startsWith(HADOOP_PREFIX)) { - String overrideKey = key.substring(HADOOP_PREFIX.length()); - conf.set(overrideKey, props.getString(key)); + for (String configKey: props.keySet()) { + String lowerCaseConfigKey = configKey.toLowerCase(); + if (lowerCaseConfigKey.startsWith(HADOOP_PREFIX)) { + String overrideKey = configKey.substring(HADOOP_PREFIX.length()); + jobConf.set(overrideKey, props.getString(configKey)); } - for (String prefix: passThroughPrefixList) { - if (lowerCase.startsWith(prefix)) { - conf.set(key, props.getString(key)); + for (String prefix: PASS_THROUGH_CONFIG_PREFIXES) { + if (lowerCaseConfigKey.startsWith(prefix)) { + jobConf.set(configKey, props.getString(configKey)); break; } } diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/mapreduce/datawriter/partition/VeniceMRPartitioner.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/mapreduce/datawriter/partition/VeniceMRPartitioner.java index 24e353dbf7c..1059af5765a 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/mapreduce/datawriter/partition/VeniceMRPartitioner.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/mapreduce/datawriter/partition/VeniceMRPartitioner.java @@ -1,7 +1,6 @@ package com.linkedin.venice.hadoop.mapreduce.datawriter.partition; import com.linkedin.venice.hadoop.utils.HadoopUtils; -import com.linkedin.venice.partitioner.DefaultVenicePartitioner; import com.linkedin.venice.partitioner.VenicePartitioner; import com.linkedin.venice.utils.ByteUtils; import com.linkedin.venice.utils.PartitionUtils; @@ -12,9 +11,9 @@ /** - * {@link VeniceMRPartitioner} will use the exact same partitioner: {@link DefaultVenicePartitioner} inside - * {@link com.linkedin.venice.writer.VeniceWriter} to make sure the messages inside each reducer belong to - * the same Kafka topic partition. + * {@link VeniceMRPartitioner} will use the exact same partitioner as the one that will be used inside + * {@link com.linkedin.venice.writer.VeniceWriter} (e.g. {@link com.linkedin.venice.partitioner.DefaultVenicePartitioner}) + * to make sure the messages inside each partition belong to the same PubSub topic partition. */ public class VeniceMRPartitioner implements Partitioner { protected VenicePartitioner venicePartitioner; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/mapreduce/datawriter/reduce/VeniceReducer.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/mapreduce/datawriter/reduce/VeniceReducer.java index b0e573b41f0..d600245cdd5 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/mapreduce/datawriter/reduce/VeniceReducer.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/mapreduce/datawriter/reduce/VeniceReducer.java @@ -10,11 +10,11 @@ import com.linkedin.venice.hadoop.task.datawriter.AbstractPartitionWriter; import com.linkedin.venice.hadoop.task.datawriter.DataWriterTaskTracker; import com.linkedin.venice.pubsub.api.PubSubProducerCallback; +import com.linkedin.venice.utils.IteratorUtils; import com.linkedin.venice.utils.VeniceProperties; import com.linkedin.venice.writer.AbstractVeniceWriter; import java.io.IOException; import java.util.Iterator; -import java.util.function.Function; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.mapred.Counters; import org.apache.hadoop.mapred.JobClient; @@ -67,7 +67,10 @@ public void reduce( } else { dataWriterTaskTracker = getDataWriterTaskTracker(); } - processValuesForKey(key.copyBytes(), mapIterator(values, BytesWritable::copyBytes), dataWriterTaskTracker); + processValuesForKey( + key.copyBytes(), + IteratorUtils.mapIterator(values, BytesWritable::copyBytes), + dataWriterTaskTracker); } private boolean updatePreviousReporter(Reporter reporter) { @@ -171,18 +174,4 @@ private long getTotalIncomingDataSizeInBytes(JobConf jobConfig) { protected void setHadoopJobClientProvider(HadoopJobClientProvider hadoopJobClientProvider) { this.hadoopJobClientProvider = hadoopJobClientProvider; } - - private Iterator mapIterator(Iterator iterator, Function mapper) { - return new Iterator() { - @Override - public boolean hasNext() { - return iterator.hasNext(); - } - - @Override - public O next() { - return mapper.apply(iterator.next()); - } - }; - } } 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/hadoop/spark/SparkConstants.java new file mode 100644 index 00000000000..657c8d0e79c --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/SparkConstants.java @@ -0,0 +1,48 @@ +package com.linkedin.venice.hadoop.spark; + +import static org.apache.spark.sql.types.DataTypes.BinaryType; +import static org.apache.spark.sql.types.DataTypes.IntegerType; + +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + + +public class SparkConstants { + // Required column names for input dataframes + public static final String KEY_COLUMN_NAME = "key"; + public static final String VALUE_COLUMN_NAME = "value"; + + // Internal column names, hence begins with "_" + public static final String PARTITION_COLUMN_NAME = "__partition__"; + + public static final StructType DEFAULT_SCHEMA = new StructType( + new StructField[] { new StructField(KEY_COLUMN_NAME, BinaryType, false, Metadata.empty()), + new StructField(VALUE_COLUMN_NAME, BinaryType, true, Metadata.empty()) }); + + public static final StructType DEFAULT_SCHEMA_WITH_PARTITION = new StructType( + new StructField[] { new StructField(KEY_COLUMN_NAME, BinaryType, false, Metadata.empty()), + new StructField(VALUE_COLUMN_NAME, BinaryType, true, Metadata.empty()), + new StructField(PARTITION_COLUMN_NAME, IntegerType, false, Metadata.empty()) }); + + /** + * Configs with this prefix will be set when building the spark session. These will get applied to all Spark jobs that + * get triggered as a part of VPJ. It can be used to configure arbitrary cluster properties like cluster address. + */ + public static final String SPARK_SESSION_CONF_PREFIX = "venice.spark.session.conf."; + + public static final String SPARK_APP_NAME_CONFIG = "spark.app.name"; + public static final String SPARK_CASE_SENSITIVE_CONFIG = "spark.sql.caseSensitive"; + + public static final String SPARK_CLUSTER_CONFIG = "venice.spark.cluster"; + public static final String SPARK_LEADER_CONFIG = "spark.master"; + public static final String DEFAULT_SPARK_CLUSTER = "local[*]"; + + /** + * Configs with this prefix will be set when building the data writer spark job and passed as job properties. These + * will only get applied on the DataWriter Spark jobs. It is useful when there are custom input formats which need + * additional configs to be able to read the data. + */ + public static final String SPARK_DATA_WRITER_CONF_PREFIX = "spark.data.writer.conf."; + +} 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/hadoop/spark/datawriter/jobs/AbstractDataWriterSparkJob.java new file mode 100644 index 00000000000..ce7b34b30d0 --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/jobs/AbstractDataWriterSparkJob.java @@ -0,0 +1,458 @@ +package com.linkedin.venice.hadoop.spark.datawriter.jobs; + +import static com.linkedin.venice.ConfigKeys.AMPLIFICATION_FACTOR; +import static com.linkedin.venice.ConfigKeys.KAFKA_BOOTSTRAP_SERVERS; +import static com.linkedin.venice.ConfigKeys.KAFKA_PRODUCER_DELIVERY_TIMEOUT_MS; +import static com.linkedin.venice.ConfigKeys.KAFKA_PRODUCER_REQUEST_TIMEOUT_MS; +import static com.linkedin.venice.ConfigKeys.KAFKA_PRODUCER_RETRIES_CONFIG; +import static com.linkedin.venice.ConfigKeys.PARTITIONER_CLASS; +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.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 com.github.luben.zstd.Zstd; +import com.linkedin.venice.compression.CompressionStrategy; +import com.linkedin.venice.exceptions.VeniceException; +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.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.utils.Utils; +import com.linkedin.venice.utils.VeniceProperties; +import com.linkedin.venice.writer.VeniceWriter; +import java.io.IOException; +import java.util.Objects; +import java.util.Properties; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.DataFrameReader; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RuntimeConfig; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder; +import org.apache.spark.sql.catalyst.encoders.RowEncoder; +import org.apache.spark.sql.functions; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.util.AccumulatorV2; + + +/** + * The implementation of {@link DataWriterComputeJob} for Spark engine. + */ +public abstract class AbstractDataWriterSparkJob extends DataWriterComputeJob { + private static final Logger LOGGER = LogManager.getLogger(AbstractDataWriterSparkJob.class); + + private VeniceProperties props; + private PushJobSetting pushJobSetting; + + private String jobGroupId; + private SparkSession sparkSession; + private Dataset dataFrame; + private DataWriterAccumulators accumulatorsForDataWriterJob; + private SparkDataWriterTaskTracker taskTracker; + + @Override + public void configure(VeniceProperties props, PushJobSetting pushJobSetting) { + this.props = props; + this.pushJobSetting = pushJobSetting; + setupDefaultSparkSessionForDataWriterJob(pushJobSetting, props); + setupSparkDataWriterJobFlow(pushJobSetting); + } + + private void setupSparkDataWriterJobFlow(PushJobSetting pushJobSetting) { + ExpressionEncoder rowEncoder = RowEncoder.apply(DEFAULT_SCHEMA); + ExpressionEncoder rowEncoderWithPartition = RowEncoder.apply(DEFAULT_SCHEMA_WITH_PARTITION); + int numOutputPartitions = pushJobSetting.partitionCount * pushJobSetting.amplificationFactor; + + // Load data from input path + Dataset dataFrameForDataWriterJob = getInputDataFrame(); + Objects.requireNonNull(dataFrameForDataWriterJob, "The input data frame cannot be null"); + + Properties jobProps = new Properties(); + sparkSession.conf().getAll().foreach(entry -> jobProps.setProperty(entry._1, entry._2)); + JavaSparkContext sparkContext = JavaSparkContext.fromSparkContext(sparkSession.sparkContext()); + Broadcast broadcastProperties = sparkContext.broadcast(jobProps); + accumulatorsForDataWriterJob = new DataWriterAccumulators(sparkSession); + taskTracker = new SparkDataWriterTaskTracker(accumulatorsForDataWriterJob); + + // Validate the schema of the input data + validateDataFrameSchema(dataFrameForDataWriterJob); + + // Convert all rows to byte[], byte[] pairs (compressed if compression is enabled) + // We could have worked with "map", but because of spraying all PartitionWriters, we need to use "flatMap" + dataFrameForDataWriterJob = dataFrameForDataWriterJob + .flatMap(new SparkInputRecordProcessorFactory(broadcastProperties, accumulatorsForDataWriterJob), rowEncoder); + + // TODO: Add map-side combiner to reduce the data size before shuffling + + // Partition the data using the custom partitioner and sort the data within that partition + dataFrameForDataWriterJob = SparkPartitionUtils.repartitionAndSortWithinPartitions( + dataFrameForDataWriterJob, + new VeniceSparkPartitioner(broadcastProperties, numOutputPartitions), + new PartitionSorter()); + + // Add a partition column to all rows based on the custom partitioner + dataFrameForDataWriterJob = + dataFrameForDataWriterJob.withColumn(PARTITION_COLUMN_NAME, functions.spark_partition_id()); + + // Write the data to PubSub + dataFrameForDataWriterJob = dataFrameForDataWriterJob.mapPartitions( + new SparkPartitionWriterFactory(broadcastProperties, accumulatorsForDataWriterJob), + rowEncoderWithPartition); + + this.dataFrame = dataFrameForDataWriterJob; + } + + /** + * Common configuration for all the Mapreduce Jobs run as part of VPJ + * + * @param config + */ + private void setupCommonSparkConf(VeniceProperties props, RuntimeConfig config, PushJobSetting pushJobSetting) { + if (pushJobSetting.enableSSL) { + config.set( + SSL_CONFIGURATOR_CLASS_CONFIG, + props.getString(SSL_CONFIGURATOR_CLASS_CONFIG, TempFileSSLConfigurator.class.getName())); + config.set(SSL_KEY_STORE_PROPERTY_NAME, props.getString(SSL_KEY_STORE_PROPERTY_NAME)); + config.set(SSL_TRUST_STORE_PROPERTY_NAME, props.getString(SSL_TRUST_STORE_PROPERTY_NAME)); + config.set(SSL_KEY_PASSWORD_PROPERTY_NAME, props.getString(SSL_KEY_PASSWORD_PROPERTY_NAME)); + } + + /** compression related common configs */ + config + .set(COMPRESSION_METRIC_COLLECTION_ENABLED, String.valueOf(pushJobSetting.compressionMetricCollectionEnabled)); + config.set(ZSTD_DICTIONARY_CREATION_REQUIRED, String.valueOf(pushJobSetting.isZstdDictCreationRequired)); + + config.set(SPARK_CASE_SENSITIVE_CONFIG, String.valueOf(true)); + } + + private void setupDefaultSparkSessionForDataWriterJob(PushJobSetting pushJobSetting, VeniceProperties props) { + this.jobGroupId = pushJobSetting.jobId + ":venice_push_job-" + pushJobSetting.topic; + + SparkConf sparkConf = new SparkConf(); + SparkSession.Builder sparkSessionBuilder = SparkSession.builder().appName(jobGroupId).config(sparkConf); + if (sparkConf.get(SPARK_LEADER_CONFIG, null) == null) { + sparkSessionBuilder.master(props.getString(SPARK_CLUSTER_CONFIG, DEFAULT_SPARK_CLUSTER)); + } + + for (String key: props.keySet()) { + if (key.toLowerCase().startsWith(SPARK_SESSION_CONF_PREFIX)) { + String overrideKey = key.substring(SPARK_SESSION_CONF_PREFIX.length()); + sparkSessionBuilder.config(overrideKey, props.getString(key)); + } + } + + sparkSession = sparkSessionBuilder.getOrCreate(); + SparkContext sparkContext = sparkSession.sparkContext(); + + // Set job group to make the job be killable programmatically + sparkContext.setJobGroup(jobGroupId, "VenicePushJob Data Writer for topic: " + pushJobSetting.topic, true); + + // Some configs to be able to identify the jobs from Spark UI + sparkContext.setCallSite(jobGroupId); + + RuntimeConfig jobConf = sparkSession.conf(); + setupCommonSparkConf(props, jobConf, pushJobSetting); + jobConf.set(BATCH_NUM_BYTES_PROP, pushJobSetting.batchNumBytes); + jobConf.set(TOPIC_PROP, pushJobSetting.topic); + jobConf.set(KAFKA_BOOTSTRAP_SERVERS, pushJobSetting.kafkaUrl); + jobConf.set(PARTITIONER_CLASS, pushJobSetting.partitionerClass); + // flatten partitionerParams since RuntimeConfig class does not support set an object + if (pushJobSetting.partitionerParams != null) { + pushJobSetting.partitionerParams.forEach((key, value) -> jobConf.set(key, value)); + } + jobConf.set(AMPLIFICATION_FACTOR, pushJobSetting.amplificationFactor); + int partitionCount = pushJobSetting.partitionCount * pushJobSetting.amplificationFactor; + jobConf.set(PARTITION_COUNT, partitionCount); + if (pushJobSetting.sslToKafka) { + jobConf.set(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, KAFKA_SECURITY_PROTOCOL); + props.keySet().stream().filter(key -> key.toLowerCase().startsWith(SSL_PREFIX)).forEach(key -> { + jobConf.set(key, props.getString(key)); + }); + } + jobConf.set(ALLOW_DUPLICATE_KEY, pushJobSetting.isDuplicateKeyAllowed); + jobConf.set(VeniceWriter.ENABLE_CHUNKING, pushJobSetting.chunkingEnabled); + jobConf.set(VeniceWriter.ENABLE_RMD_CHUNKING, pushJobSetting.rmdChunkingEnabled); + + jobConf.set(STORAGE_QUOTA_PROP, pushJobSetting.storeStorageQuota); + + if (pushJobSetting.isSourceKafka) { + // Use some fake value schema id here since it won't be used + jobConf.set(VALUE_SCHEMA_ID_PROP, -1); + /** + * Kafka input topic could be inferred from the store name, but absent from the original properties. + * So here will set it up from {@link #pushJobSetting}. + */ + jobConf.set(KAFKA_INPUT_TOPIC, pushJobSetting.kafkaInputTopic); + jobConf.set(KAFKA_INPUT_BROKER_URL, pushJobSetting.kafkaInputBrokerUrl); + jobConf.set(REPUSH_TTL_ENABLE, pushJobSetting.repushTTLEnabled); + jobConf.set(REPUSH_TTL_START_TIMESTAMP, pushJobSetting.repushTTLStartTimeMs); + if (pushJobSetting.repushTTLEnabled) { + // Currently, we only support one policy. Thus, we don't allow overriding it. + jobConf.set(REPUSH_TTL_POLICY, TTLResolutionPolicy.RT_WRITE_ONLY.getValue()); + jobConf.set(RMD_SCHEMA_DIR, pushJobSetting.rmdSchemaDir); + } + // Pass the compression strategy of source version to repush MR job + jobConf.set( + KAFKA_INPUT_SOURCE_COMPRESSION_STRATEGY, + pushJobSetting.sourceKafkaInputVersionInfo.getCompressionStrategy().name()); + jobConf.set( + KAFKA_INPUT_SOURCE_TOPIC_CHUNKING_ENABLED, + pushJobSetting.sourceKafkaInputVersionInfo.isChunkingEnabled()); + } else { + jobConf.set(VALUE_SCHEMA_ID_PROP, pushJobSetting.valueSchemaId); + jobConf.set(DERIVED_SCHEMA_ID_PROP, pushJobSetting.derivedSchemaId); + } + jobConf.set(ENABLE_WRITE_COMPUTE, pushJobSetting.enableWriteCompute); + + if (!props.containsKey(KAFKA_PRODUCER_REQUEST_TIMEOUT_MS)) { + // If the push job plug-in doesn't specify the request timeout config, default will be infinite + jobConf.set(KAFKA_PRODUCER_REQUEST_TIMEOUT_MS, Integer.MAX_VALUE); + } + if (!props.containsKey(KAFKA_PRODUCER_RETRIES_CONFIG)) { + // If the push job plug-in doesn't specify the retries config, default will be infinite + jobConf.set(KAFKA_PRODUCER_RETRIES_CONFIG, Integer.MAX_VALUE); + } + if (!props.containsKey(KAFKA_PRODUCER_DELIVERY_TIMEOUT_MS)) { + // If the push job plug-in doesn't specify the delivery timeout config, default will be infinite + jobConf.set(KAFKA_PRODUCER_DELIVERY_TIMEOUT_MS, Integer.MAX_VALUE); + } + + jobConf.set(TELEMETRY_MESSAGE_INTERVAL, props.getString(TELEMETRY_MESSAGE_INTERVAL, "10000")); + jobConf.set(EXTENDED_SCHEMA_VALIDITY_CHECK_ENABLED, pushJobSetting.extendedSchemaValidityCheckEnabled); + + // Compression related + // Note that COMPRESSION_STRATEGY is from topic creation response as it might be different from the store config + // (eg: for inc push) + jobConf.set( + COMPRESSION_STRATEGY, + pushJobSetting.topicCompressionStrategy != null + ? pushJobSetting.topicCompressionStrategy.name() + : CompressionStrategy.NO_OP.name()); + jobConf.set( + ZSTD_COMPRESSION_LEVEL, + props.getString(ZSTD_COMPRESSION_LEVEL, String.valueOf(Zstd.maxCompressionLevel()))); + jobConf.set(ZSTD_DICTIONARY_CREATION_SUCCESS, pushJobSetting.isZstdDictCreationSuccess); + + /** + * Override the configs following the rules: + *
    + *
  • Pass-through the properties whose names start with the prefixes defined in {@link PASS_THROUGH_CONFIG_PREFIXES}.
  • + *
  • Override the properties that are specified with the {@link SPARK_DATA_WRITER_CONF_PREFIX} prefix.
  • + *
+ **/ + for (String configKey: props.keySet()) { + String lowerCaseConfigKey = configKey.toLowerCase(); + if (lowerCaseConfigKey.startsWith(SPARK_DATA_WRITER_CONF_PREFIX)) { + String overrideKey = configKey.substring(SPARK_DATA_WRITER_CONF_PREFIX.length()); + jobConf.set(overrideKey, props.getString(configKey)); + } + for (String prefix: PASS_THROUGH_CONFIG_PREFIXES) { + if (lowerCaseConfigKey.startsWith(prefix)) { + jobConf.set(configKey, props.getString(configKey)); + break; + } + } + } + } + + protected SparkSession getSparkSession() { + return sparkSession; + } + + /** + * Get the data frame based on the user's input data. The schema of the {@link Row} has the following constraints: + *
    + *
  • Must contain a field "key" with the schema: {@link DataTypes#BinaryType}. This is the key of the record represented in serialized Avro.
  • + *
  • Must contain a field "value" with the schema: {@link DataTypes#BinaryType}. This is the value of the record represented in serialized Avro.
  • + *
  • Must not contain fields with names beginning with "_". These are reserved for internal use.
  • + *
  • Can contain fields that do not violate the above constraints
  • + *
+ * @see {@link #validateDataFrameSchema(StructType)} + * + * @return The data frame based on the user's input data + */ + protected abstract Dataset getUserInputDataFrame(); + + private Dataset getInputDataFrame() { + if (pushJobSetting.isSourceKafka) { + throw new VeniceUnsupportedOperationException("Spark push job doesn't support KIF yet"); + } else { + return getUserInputDataFrame(); + } + } + + // Set configs for both SparkSession (data processing) and DataFrameReader (input format) + protected void setInputConf(SparkSession session, DataFrameReader dataFrameReader, String key, String value) { + session.conf().set(key, value); + dataFrameReader.option(key, value); + } + + @Override + public DataWriterTaskTracker getTaskTracker() { + return taskTracker; + } + + // This is a part of the public API. Do not remove. + @SuppressWarnings("unused") + protected VeniceProperties getJobProperties() { + return props; + } + + // This is a part of the public API. Do not remove. + @Override + public PushJobSetting getPushJobSetting() { + return pushJobSetting; + } + + @Override + protected void runComputeJob() { + LOGGER.info("Triggering Spark job for data writer"); + try { + // For VPJ, we don't care about the output from the DAG. ".count()" is an action that will trigger execution of + // the DAG to completion and will not copy all the rows to the driver to be more memory efficient. + dataFrame.count(); + } finally { + // No matter what, always log the final accumulator values + logAccumulatorValues(); + } + } + + @Override + public void kill() { + super.kill(); + sparkSession.sparkContext().cancelJobGroup(jobGroupId); + } + + @Override + public void close() throws IOException { + Utils.closeQuietlyWithErrorLogged(sparkSession); + } + + private void logAccumulatorValues() { + LOGGER.info("Accumulator values for data writer job:"); + logAccumulatorValue(accumulatorsForDataWriterJob.outputRecordCounter); + logAccumulatorValue(accumulatorsForDataWriterJob.emptyRecordCounter); + logAccumulatorValue(accumulatorsForDataWriterJob.totalKeySizeCounter); + logAccumulatorValue(accumulatorsForDataWriterJob.compressedValueSizeCounter); + logAccumulatorValue(accumulatorsForDataWriterJob.gzipCompressedValueSizeCounter); + logAccumulatorValue(accumulatorsForDataWriterJob.zstdCompressedValueSizeCounter); + logAccumulatorValue(accumulatorsForDataWriterJob.sprayAllPartitionsTriggeredCount); + logAccumulatorValue(accumulatorsForDataWriterJob.partitionWriterCloseCounter); + logAccumulatorValue(accumulatorsForDataWriterJob.repushTtlFilteredRecordCounter); + logAccumulatorValue(accumulatorsForDataWriterJob.writeAclAuthorizationFailureCounter); + logAccumulatorValue(accumulatorsForDataWriterJob.recordTooLargeFailureCounter); + logAccumulatorValue(accumulatorsForDataWriterJob.duplicateKeyWithIdenticalValueCounter); + logAccumulatorValue(accumulatorsForDataWriterJob.duplicateKeyWithDistinctValueCounter); + } + + private void logAccumulatorValue(AccumulatorV2 accumulator) { + LOGGER.info(" {}: {}", accumulator.name().get(), accumulator.value()); + } + + private void validateDataFrameSchema(Dataset dataFrameForDataWriterJob) { + StructType dataSchema = dataFrameForDataWriterJob.schema(); + if (!validateDataFrameSchema(dataSchema)) { + String errorMessage = + String.format("The provided input data schema is not supported. Provided schema: %s.", dataSchema); + throw new VeniceException(errorMessage); + } + } + + private boolean validateDataFrameSchema(StructType dataSchema) { + StructField[] fields = dataSchema.fields(); + + if (fields.length < 2) { + LOGGER.error("The provided input data schema does not have enough fields"); + return false; + } + + int keyFieldIndex = SparkScalaUtils.getFieldIndex(dataSchema, KEY_COLUMN_NAME); + + if (keyFieldIndex == -1) { + LOGGER.error("The provided input data schema does not have a {} field", KEY_COLUMN_NAME); + return false; + } + + if (!fields[keyFieldIndex].dataType().equals(DataTypes.BinaryType)) { + LOGGER.error("The provided input key field's schema must be {}", DataTypes.BinaryType); + return false; + } + + int valueFieldIndex = SparkScalaUtils.getFieldIndex(dataSchema, VALUE_COLUMN_NAME); + + if (valueFieldIndex == -1) { + LOGGER.error("The provided input data schema does not have a {} field", VALUE_COLUMN_NAME); + return false; + } + + if (!fields[valueFieldIndex].dataType().equals(DataTypes.BinaryType)) { + LOGGER.error("The provided input value field's schema must be {}", DataTypes.BinaryType); + return false; + } + + for (StructField field: fields) { + if (field.name().startsWith("_")) { + LOGGER.error("The provided input must not have fields that start with an underscore"); + return false; + } + } + + return true; + } +} 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/hadoop/spark/datawriter/jobs/DataWriterSparkJob.java new file mode 100644 index 00000000000..5cd2e171a19 --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/jobs/DataWriterSparkJob.java @@ -0,0 +1,58 @@ +package com.linkedin.venice.hadoop.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 com.linkedin.venice.hadoop.PushJobSetting; +import com.linkedin.venice.hadoop.spark.input.hdfs.VeniceHdfsSource; +import org.apache.hadoop.fs.Path; +import org.apache.spark.sql.DataFrameReader; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; + + +/** + * The default implementation of {@link AbstractDataWriterSparkJob} for Avro and Vson file input formats. + */ +public class DataWriterSparkJob extends AbstractDataWriterSparkJob { + @Override + protected Dataset getUserInputDataFrame() { + SparkSession sparkSession = getSparkSession(); + PushJobSetting pushJobSetting = getPushJobSetting(); + + DataFrameReader dataFrameReader = sparkSession.read(); + dataFrameReader.format(VeniceHdfsSource.class.getCanonicalName()); + setInputConf(sparkSession, dataFrameReader, INPUT_PATH_PROP, new Path(pushJobSetting.inputURI).toString()); + setInputConf(sparkSession, dataFrameReader, KEY_FIELD_PROP, pushJobSetting.keyField); + setInputConf(sparkSession, dataFrameReader, VALUE_FIELD_PROP, pushJobSetting.valueField); + if (pushJobSetting.etlValueSchemaTransformation != null) { + setInputConf( + sparkSession, + dataFrameReader, + ETL_VALUE_SCHEMA_TRANSFORMATION, + pushJobSetting.etlValueSchemaTransformation.name()); + } + if (pushJobSetting.isAvro) { + setInputConf(sparkSession, dataFrameReader, SCHEMA_STRING_PROP, pushJobSetting.inputDataSchemaString); + if (pushJobSetting.generatePartialUpdateRecordFromInput) { + setInputConf(sparkSession, dataFrameReader, GENERATE_PARTIAL_UPDATE_RECORD_FROM_INPUT, String.valueOf(true)); + setInputConf(sparkSession, dataFrameReader, UPDATE_SCHEMA_STRING_PROP, pushJobSetting.valueSchemaString); + } + setInputConf(sparkSession, dataFrameReader, VSON_PUSH, String.valueOf(false)); + } else { + setInputConf(sparkSession, dataFrameReader, VSON_PUSH, String.valueOf(true)); + setInputConf(sparkSession, dataFrameReader, FILE_KEY_SCHEMA, pushJobSetting.vsonInputKeySchemaString); + setInputConf(sparkSession, dataFrameReader, FILE_VALUE_SCHEMA, pushJobSetting.vsonInputValueSchemaString); + } + return dataFrameReader.load(); + } +} 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/hadoop/spark/datawriter/partition/PartitionSorter.java new file mode 100644 index 00000000000..61ca53d2d0d --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/partition/PartitionSorter.java @@ -0,0 +1,35 @@ +package com.linkedin.venice.hadoop.spark.datawriter.partition; + +import com.linkedin.venice.utils.ArrayUtils; +import java.io.Serializable; +import java.util.Comparator; +import org.apache.spark.sql.Row; + + +/** + * Sort the rows based on the key and value in ascending order using unsigned byte comparison. + *
    + *
  • The sorting on the key is the same as what RocksDB and Shuffle-Sort in MapReduce use.
  • + *
  • The sorting on the value is to make {@link com.linkedin.venice.hadoop.spark.datawriter.writer.SparkPartitionWriter} be able to optimize the de-duping of values.
  • + *
+ */ +public class PartitionSorter implements Comparator, Serializable { + private static final long serialVersionUID = 1L; + + @Override + public int compare(Row r1, Row r2) { + // For primary sort + byte[] key1 = (byte[]) r1.get(0); + byte[] key2 = (byte[]) r2.get(0); + + int keyCompare = ArrayUtils.compareUnsigned(key1, key2); + if (keyCompare != 0) { + return keyCompare; + } + + // For secondary sort + byte[] value1 = (byte[]) r1.get(1); + byte[] value2 = (byte[]) r2.get(1); + return ArrayUtils.compareUnsigned(value1, value2); + } +} 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/hadoop/spark/datawriter/partition/VeniceSparkPartitioner.java new file mode 100644 index 00000000000..d7caa4a2e43 --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/partition/VeniceSparkPartitioner.java @@ -0,0 +1,77 @@ +package com.linkedin.venice.hadoop.spark.datawriter.partition; + +import static com.linkedin.venice.hadoop.VenicePushJobConstants.PARTITION_COUNT; + +import com.linkedin.venice.exceptions.VeniceException; +import com.linkedin.venice.partitioner.VenicePartitioner; +import com.linkedin.venice.utils.ByteUtils; +import com.linkedin.venice.utils.PartitionUtils; +import com.linkedin.venice.utils.VeniceProperties; +import java.util.Properties; +import org.apache.spark.Partitioner; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.Row; + + +/** + * {@link VeniceSparkPartitioner} will use the exact same partitioner as the one that will be used inside + * {@link com.linkedin.venice.writer.VeniceWriter} (e.g. {@link com.linkedin.venice.partitioner.DefaultVenicePartitioner}) + * to make sure the messages inside each partition belong to the same PubSub topic partition. + */ +public class VeniceSparkPartitioner extends Partitioner { + private static final long serialVersionUID = 1L; + private Broadcast broadcastProperties; + private transient VenicePartitioner venicePartitioner = null; + private final int numPartitions; + + public VeniceSparkPartitioner(Broadcast broadcastProperties, int numPartitions) { + this.numPartitions = numPartitions; + this.broadcastProperties = broadcastProperties; + configurePartitioner(); + } + + @Override + public int numPartitions() { + return numPartitions; + } + + private void configurePartitioner() { + if (this.venicePartitioner != null) { + return; + } + + VeniceProperties props = new VeniceProperties(broadcastProperties.getValue()); + /** + * Note: Here needs to use the exact same partitioner being used by {@link com.linkedin.venice.writer.VeniceWriter}. + */ + this.venicePartitioner = PartitionUtils.getVenicePartitioner(props); + + int numPartitions = props.getInt(PARTITION_COUNT); + + if (numPartitions != this.numPartitions) { + throw new VeniceException( + "Partition count mismatch: " + numPartitions + " (from config) != " + this.numPartitions + " (from driver)"); + } + } + + @Override + public int getPartition(Object key) { + if (!(key instanceof Row)) { + throw new VeniceException("VeniceSparkPartitioner only supports Row keys"); + } + + // Since the partitioner is transient, it needs to be re-initialized in each task. + configurePartitioner(); + + Row row = (Row) key; + return getPartition((byte[]) row.get(0), (byte[]) row.get(1), numPartitions); + } + + private int getPartition(byte[] key, byte[] value, int numPartitions) { + if (key.length == 0) { + // Special case, used only to ensure that all PartitionWriters are instantiated, even if starved of actual data. + return ByteUtils.readInt(value, 0); + } + return venicePartitioner.getPartitionId(key, numPartitions); + } +} 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/hadoop/spark/datawriter/recordprocessor/SparkInputRecordProcessor.java new file mode 100644 index 00000000000..256f8d76786 --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/recordprocessor/SparkInputRecordProcessor.java @@ -0,0 +1,56 @@ +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; + +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.utils.VeniceProperties; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Properties; +import java.util.function.BiConsumer; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema; + + +/** + * An implementation of {@link AbstractInputRecordProcessor} for Spark that processes input records from the dataframe + * and emits an {@link Iterator} of {@link Row} with {@link DEFAULT_SCHEMA} as the schema. + */ +public class SparkInputRecordProcessor extends AbstractInputRecordProcessor { + private final DataWriterTaskTracker dataWriterTaskTracker; + + public SparkInputRecordProcessor(Properties jobProperties, DataWriterAccumulators accumulators) { + dataWriterTaskTracker = new SparkDataWriterTaskTracker(accumulators); + super.configure(new SparkEngineTaskConfigProvider(jobProperties)); + } + + 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 valueBB = value == null ? null : ByteBuffer.wrap(value); + super.processRecord(keyBB, valueBB, getRecordEmitter(outputRows), dataWriterTaskTracker); + return outputRows.iterator(); + } + + @Override + protected AbstractVeniceRecordReader getRecordReader(VeniceProperties props) { + return IdentityVeniceRecordReader.getInstance(); + } + + private BiConsumer getRecordEmitter(List rows) { + return (key, value) -> { + rows.add(new GenericRowWithSchema(new Object[] { key, value }, 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/hadoop/spark/datawriter/recordprocessor/SparkInputRecordProcessorFactory.java new file mode 100644 index 00000000000..ba8bb245700 --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/recordprocessor/SparkInputRecordProcessorFactory.java @@ -0,0 +1,34 @@ +package com.linkedin.venice.hadoop.spark.datawriter.recordprocessor; + +import com.linkedin.venice.hadoop.spark.datawriter.task.DataWriterAccumulators; +import java.util.Iterator; +import java.util.Properties; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.Row; + + +/** + * A Factory class to create individual {@link SparkInputRecordProcessor} for processing records in a Spark Dataframe. + */ +public class SparkInputRecordProcessorFactory implements FlatMapFunction { + private static final long serialVersionUID = 1L; + private transient SparkInputRecordProcessor processor = null; + private final Broadcast jobProps; + private final DataWriterAccumulators accumulators; + + public SparkInputRecordProcessorFactory(Broadcast jobProps, DataWriterAccumulators accumulators) { + this.jobProps = jobProps; + this.accumulators = accumulators; + } + + @Override + public Iterator call(Row row) throws Exception { + // Lazily initialize the processor to avoid serialization issues since it is transient + if (processor == null) { + processor = new SparkInputRecordProcessor(jobProps.getValue(), accumulators); + } + + return processor.processRecord(row); + } +} 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/hadoop/spark/datawriter/task/DataWriterAccumulators.java new file mode 100644 index 00000000000..aa5252adfa7 --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/task/DataWriterAccumulators.java @@ -0,0 +1,47 @@ +package com.linkedin.venice.hadoop.spark.datawriter.task; + +import java.io.Serializable; +import org.apache.spark.SparkContext; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.util.LongAccumulator; + + +/** + * All the {@link org.apache.spark.util.AccumulatorV2} objects that are used in the Spark DataWriter jobs. + */ +public class DataWriterAccumulators implements Serializable { + private static final long serialVersionUID = 1L; + + public final LongAccumulator sprayAllPartitionsTriggeredCount; + public final LongAccumulator emptyRecordCounter; + public final LongAccumulator totalKeySizeCounter; + public final LongAccumulator uncompressedValueSizeCounter; + public final LongAccumulator compressedValueSizeCounter; + public final LongAccumulator gzipCompressedValueSizeCounter; + public final LongAccumulator zstdCompressedValueSizeCounter; + public final LongAccumulator outputRecordCounter; + public final LongAccumulator duplicateKeyWithIdenticalValueCounter; + public final LongAccumulator writeAclAuthorizationFailureCounter; + public final LongAccumulator recordTooLargeFailureCounter; + public final LongAccumulator duplicateKeyWithDistinctValueCounter; + public final LongAccumulator partitionWriterCloseCounter; + public final LongAccumulator repushTtlFilteredRecordCounter; + + public DataWriterAccumulators(SparkSession session) { + SparkContext sparkContext = session.sparkContext(); + sprayAllPartitionsTriggeredCount = sparkContext.longAccumulator("Spray All Partitions Triggered"); + emptyRecordCounter = sparkContext.longAccumulator("Empty Records"); + totalKeySizeCounter = sparkContext.longAccumulator("Total Key Size"); + uncompressedValueSizeCounter = sparkContext.longAccumulator("Total Uncompressed Value Size"); + compressedValueSizeCounter = sparkContext.longAccumulator("Total Compressed Value Size"); + gzipCompressedValueSizeCounter = sparkContext.longAccumulator("Total Gzip Compressed Value Size"); + zstdCompressedValueSizeCounter = sparkContext.longAccumulator("Total Zstd Compressed Value Size"); + outputRecordCounter = sparkContext.longAccumulator("Total Output Records"); + partitionWriterCloseCounter = sparkContext.longAccumulator("Partition Writers Closed"); + repushTtlFilteredRecordCounter = sparkContext.longAccumulator("Repush TTL Filtered Records"); + writeAclAuthorizationFailureCounter = sparkContext.longAccumulator("ACL Authorization Failures"); + recordTooLargeFailureCounter = sparkContext.longAccumulator("Record Too Large Failures"); + duplicateKeyWithIdenticalValueCounter = sparkContext.longAccumulator("Duplicate Key With Identical Value"); + duplicateKeyWithDistinctValueCounter = sparkContext.longAccumulator("Duplicate Key With Distinct Value"); + } +} 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/hadoop/spark/datawriter/task/SparkDataWriterTaskTracker.java new file mode 100644 index 00000000000..d6bc19ae75b --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/task/SparkDataWriterTaskTracker.java @@ -0,0 +1,142 @@ +package com.linkedin.venice.hadoop.spark.datawriter.task; + +import com.linkedin.venice.hadoop.task.datawriter.DataWriterTaskTracker; + + +public class SparkDataWriterTaskTracker implements DataWriterTaskTracker { + private final DataWriterAccumulators accumulators; + + public SparkDataWriterTaskTracker(DataWriterAccumulators accumulators) { + this.accumulators = accumulators; + } + + @Override + public void trackSprayAllPartitions() { + accumulators.sprayAllPartitionsTriggeredCount.add(1); + } + + @Override + public void trackEmptyRecord() { + accumulators.emptyRecordCounter.add(1); + } + + @Override + public void trackKeySize(int size) { + accumulators.totalKeySizeCounter.add(size); + } + + @Override + public void trackUncompressedValueSize(int size) { + accumulators.uncompressedValueSizeCounter.add(size); + } + + @Override + public void trackCompressedValueSize(int size) { + accumulators.compressedValueSizeCounter.add(size); + } + + @Override + public void trackGzipCompressedValueSize(int size) { + accumulators.gzipCompressedValueSizeCounter.add(size); + } + + @Override + public void trackZstdCompressedValueSize(int size) { + accumulators.zstdCompressedValueSizeCounter.add(size); + } + + @Override + public void trackWriteAclAuthorizationFailure() { + accumulators.writeAclAuthorizationFailureCounter.add(1); + } + + @Override + public void trackRecordTooLargeFailure() { + accumulators.recordTooLargeFailureCounter.add(1); + } + + @Override + public void trackRecordSentToPubSub() { + accumulators.outputRecordCounter.add(1); + } + + @Override + public void trackDuplicateKeyWithDistinctValue(int count) { + accumulators.duplicateKeyWithDistinctValueCounter.add(count); + } + + @Override + public void trackDuplicateKeyWithIdenticalValue(int count) { + accumulators.duplicateKeyWithIdenticalValueCounter.add(count); + } + + @Override + public void trackRepushTtlFilteredRecord() { + accumulators.repushTtlFilteredRecordCounter.add(1); + } + + @Override + public void trackPartitionWriterClose() { + accumulators.partitionWriterCloseCounter.add(1); + } + + @Override + public long getSprayAllPartitionsCount() { + return accumulators.sprayAllPartitionsTriggeredCount.value(); + } + + @Override + public long getTotalKeySize() { + return accumulators.totalKeySizeCounter.value(); + } + + @Override + public long getTotalValueSize() { + return accumulators.compressedValueSizeCounter.value(); + } + + @Override + public long getTotalUncompressedValueSize() { + return accumulators.uncompressedValueSizeCounter.value(); + } + + @Override + public long getTotalGzipCompressedValueSize() { + return accumulators.gzipCompressedValueSizeCounter.value(); + } + + @Override + public long getTotalZstdCompressedValueSize() { + return accumulators.zstdCompressedValueSizeCounter.value(); + } + + @Override + public long getRecordTooLargeFailureCount() { + return accumulators.recordTooLargeFailureCounter.value(); + } + + @Override + public long getWriteAclAuthorizationFailureCount() { + return accumulators.writeAclAuthorizationFailureCounter.value(); + } + + @Override + public long getDuplicateKeyWithDistinctValueCount() { + return accumulators.duplicateKeyWithDistinctValueCounter.value(); + } + + @Override + public long getOutputRecordsCount() { + return accumulators.outputRecordCounter.value(); + } + + @Override + public long getPartitionWriterCloseCount() { + return accumulators.partitionWriterCloseCounter.value(); + } + + @Override + public long getRepushTtlFilterCount() { + return accumulators.repushTtlFilteredRecordCounter.value(); + } +} 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/hadoop/spark/datawriter/writer/SparkPartitionWriter.java new file mode 100644 index 00000000000..61bb0a32378 --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/writer/SparkPartitionWriter.java @@ -0,0 +1,57 @@ +package com.linkedin.venice.hadoop.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 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 java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.Properties; +import org.apache.spark.sql.Row; + + +public class SparkPartitionWriter extends AbstractPartitionWriter { + private final SparkDataWriterTaskTracker dataWriterTaskTracker; + + public SparkPartitionWriter(Properties jobProperties, DataWriterAccumulators accumulators) { + dataWriterTaskTracker = new SparkDataWriterTaskTracker(accumulators); + super.configure(new SparkEngineTaskConfigProvider(jobProperties)); + } + + @Override + protected long getTotalIncomingDataSizeInBytes() { + // TODO: Explore if this info can be fetched from Spark engine or if it can be passed down via the DAG + return super.getTotalIncomingDataSizeInBytes(); + } + + void processRows(Iterator rows) { + byte[] key = null; + List valuesForKey = null; + while (rows.hasNext()) { + Row row = rows.next(); + byte[] incomingKey = Objects.requireNonNull(row.getAs(KEY_COLUMN_NAME), "Key cannot be null"); + + if (!Arrays.equals(incomingKey, key)) { + if (key != null) { + // Key is different from the prev one and is not null. Write it out to PubSub. + super.processValuesForKey(key, valuesForKey.iterator(), dataWriterTaskTracker); + } + key = incomingKey; + valuesForKey = new ArrayList<>(); + } + + byte[] incomingValue = row.getAs(VALUE_COLUMN_NAME); + valuesForKey.add(incomingValue); + } + + if (key != null) { + super.processValuesForKey(key, valuesForKey.iterator(), dataWriterTaskTracker); + } + } +} 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/hadoop/spark/datawriter/writer/SparkPartitionWriterFactory.java new file mode 100644 index 00000000000..a5f2108fed0 --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/datawriter/writer/SparkPartitionWriterFactory.java @@ -0,0 +1,28 @@ +package com.linkedin.venice.hadoop.spark.datawriter.writer; + +import com.linkedin.venice.hadoop.spark.datawriter.task.DataWriterAccumulators; +import java.util.Iterator; +import java.util.Properties; +import org.apache.spark.api.java.function.MapPartitionsFunction; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.Row; + + +public class SparkPartitionWriterFactory implements MapPartitionsFunction { + private static final long serialVersionUID = 1L; + private final Broadcast jobProps; + private final DataWriterAccumulators accumulators; + + public SparkPartitionWriterFactory(Broadcast jobProps, DataWriterAccumulators accumulators) { + this.jobProps = jobProps; + this.accumulators = accumulators; + } + + @Override + public Iterator call(Iterator rows) throws Exception { + try (SparkPartitionWriter partitionWriter = new SparkPartitionWriter(jobProps.getValue(), accumulators)) { + partitionWriter.processRows(rows); + } + return rows; + } +} 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/hadoop/spark/engine/SparkEngineTaskConfigProvider.java new file mode 100644 index 00000000000..6adb45660b6 --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/engine/SparkEngineTaskConfigProvider.java @@ -0,0 +1,36 @@ +package com.linkedin.venice.hadoop.spark.engine; + +import static com.linkedin.venice.hadoop.spark.SparkConstants.SPARK_APP_NAME_CONFIG; + +import com.linkedin.venice.hadoop.engine.EngineTaskConfigProvider; +import java.util.Properties; +import org.apache.spark.TaskContext; + + +public class SparkEngineTaskConfigProvider implements EngineTaskConfigProvider { + private final TaskContext taskContext; + private final Properties jobProps; + + public SparkEngineTaskConfigProvider(Properties jobProps) { + this.taskContext = TaskContext.get(); + + this.jobProps = new Properties(); + this.jobProps.putAll(jobProps); + this.jobProps.putAll(taskContext.getLocalProperties()); + } + + @Override + public String getJobName() { + return jobProps.getProperty(SPARK_APP_NAME_CONFIG); + } + + @Override + public int getTaskId() { + return taskContext.partitionId(); + } + + @Override + public Properties getJobProps() { + return jobProps; + } +} 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/hadoop/spark/input/VeniceAbstractPartitionReader.java new file mode 100644 index 00000000000..43a31eae8a5 --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/VeniceAbstractPartitionReader.java @@ -0,0 +1,36 @@ +package com.linkedin.venice.hadoop.spark.input; + +import com.linkedin.venice.hadoop.input.recordreader.VeniceRecordIterator; +import com.linkedin.venice.utils.Utils; +import com.linkedin.venice.utils.VeniceProperties; +import java.io.IOException; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.PartitionReader; + + +public abstract class VeniceAbstractPartitionReader implements PartitionReader { + private final VeniceRecordIterator recordIterator; + + public VeniceAbstractPartitionReader(VeniceProperties jobConfig, InputPartition partition) { + this.recordIterator = createRecordIterator(jobConfig, partition); + } + + protected abstract VeniceRecordIterator createRecordIterator(VeniceProperties jobConfig, InputPartition partition); + + @Override + public boolean next() throws IOException { + return recordIterator.next(); + } + + @Override + public InternalRow get() { + return new GenericInternalRow(new Object[] { recordIterator.getCurrentKey(), recordIterator.getCurrentValue() }); + } + + @Override + public void close() throws IOException { + Utils.closeQuietlyWithErrorLogged(recordIterator); + } +} 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/hadoop/spark/input/hdfs/VeniceHdfsInputPartition.java new file mode 100644 index 00000000000..48325ecc345 --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputPartition.java @@ -0,0 +1,28 @@ +package com.linkedin.venice.hadoop.spark.input.hdfs; + +import java.net.URI; +import org.apache.hadoop.fs.Path; +import org.apache.spark.sql.connector.read.InputPartition; + + +public class VeniceHdfsInputPartition implements InputPartition { + private static final long serialVersionUID = 1L; + + private transient Path filePath; + // Path was not serializable till HDFS version 3.0.0, so we use URI instead: + // https://issues.apache.org/jira/browse/HADOOP-13519 + private final URI filePathURI; + + public VeniceHdfsInputPartition(Path filePath) { + this.filePath = filePath; + this.filePathURI = filePath.toUri(); + } + + public Path getFilePath() { + // Transient fields are not serialized, so we need to reinitialize them after deserialization + if (filePath == null) { + filePath = new Path(filePathURI); + } + return filePath; + } +} 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/hadoop/spark/input/hdfs/VeniceHdfsInputPartitionReader.java new file mode 100644 index 00000000000..71d279efde3 --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputPartitionReader.java @@ -0,0 +1,72 @@ +package com.linkedin.venice.hadoop.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.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.KEY_FIELD_PROP; +import static com.linkedin.venice.hadoop.VenicePushJobConstants.SCHEMA_STRING_PROP; +import static com.linkedin.venice.hadoop.VenicePushJobConstants.VALUE_FIELD_PROP; +import static com.linkedin.venice.hadoop.VenicePushJobConstants.VSON_PUSH; + +import com.linkedin.avroutil1.compatibility.AvroCompatibilityHelper; +import com.linkedin.venice.etl.ETLValueSchemaTransformation; +import com.linkedin.venice.exceptions.VeniceException; +import com.linkedin.venice.hadoop.input.recordreader.VeniceRecordIterator; +import com.linkedin.venice.hadoop.input.recordreader.avro.VeniceAvroFileIterator; +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.utils.VeniceProperties; +import java.io.IOException; +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.spark.sql.connector.read.InputPartition; + + +public class VeniceHdfsInputPartitionReader extends VeniceAbstractPartitionReader { + public VeniceHdfsInputPartitionReader(VeniceProperties jobConfig, VeniceHdfsInputPartition partition) { + super(jobConfig, partition); + } + + @Override + protected VeniceRecordIterator createRecordIterator(VeniceProperties jobConfig, InputPartition partition) { + if (!(partition instanceof VeniceHdfsInputPartition)) { + throw new VeniceException("Expected VeniceHdfsInputPartition"); + } + VeniceHdfsInputPartition inputPartition = (VeniceHdfsInputPartition) partition; + + Configuration configuration = new Configuration(); + FileSystem fs; + try { + fs = FileSystem.get(configuration); + } catch (IOException e) { + throw new VeniceException("Unable to get a FileSystem", e); + } + + String keyFieldStr = jobConfig.getString(KEY_FIELD_PROP, DEFAULT_KEY_FIELD_PROP); + String valueFieldStr = jobConfig.getString(VALUE_FIELD_PROP, DEFAULT_VALUE_FIELD_PROP); + ETLValueSchemaTransformation etlValueSchemaTransformation = ETLValueSchemaTransformation + .valueOf(jobConfig.getString(ETL_VALUE_SCHEMA_TRANSFORMATION, ETLValueSchemaTransformation.NONE.name())); + boolean vsonPush = jobConfig.getBoolean(VSON_PUSH, false); + Path filePath = inputPartition.getFilePath(); + + if (vsonPush) { + String fileKeySchema = jobConfig.getString(FILE_KEY_SCHEMA); + String fileValueSchema = jobConfig.getString(FILE_VALUE_SCHEMA); + VeniceVsonRecordReader recordReader = + new VeniceVsonRecordReader(fileKeySchema, fileValueSchema, keyFieldStr, valueFieldStr); + return new VeniceVsonFileIterator(fs, filePath, recordReader); + } + + // Input data is not in Vson format. It must be Avro. + Schema fileSchema = AvroCompatibilityHelper.parse(jobConfig.getString(SCHEMA_STRING_PROP)); + VeniceAvroRecordReader recordReader = + new VeniceAvroRecordReader(fileSchema, keyFieldStr, valueFieldStr, etlValueSchemaTransformation, null); + return new VeniceAvroFileIterator(fs, filePath, recordReader); + } +} 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/hadoop/spark/input/hdfs/VeniceHdfsInputPartitionReaderFactory.java new file mode 100644 index 00000000000..49d9b58d427 --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputPartitionReaderFactory.java @@ -0,0 +1,28 @@ +package com.linkedin.venice.hadoop.spark.input.hdfs; + +import com.linkedin.venice.utils.VeniceProperties; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.PartitionReader; +import org.apache.spark.sql.connector.read.PartitionReaderFactory; + + +public class VeniceHdfsInputPartitionReaderFactory implements PartitionReaderFactory { + private static final long serialVersionUID = 1L; + + private final VeniceProperties jobConfig; + + public VeniceHdfsInputPartitionReaderFactory(VeniceProperties jobConfig) { + this.jobConfig = jobConfig; + } + + @Override + public PartitionReader createReader(InputPartition partition) { + if (!(partition instanceof VeniceHdfsInputPartition)) { + throw new IllegalArgumentException( + "VeniceHdfsInputPartitionReaderFactory can only create readers for VeniceHdfsInputPartition"); + } + + return new VeniceHdfsInputPartitionReader(jobConfig, (VeniceHdfsInputPartition) partition); + } +} 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/hadoop/spark/input/hdfs/VeniceHdfsInputScan.java new file mode 100644 index 00000000000..88afb216f00 --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputScan.java @@ -0,0 +1,67 @@ +package com.linkedin.venice.hadoop.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.utils.VeniceProperties; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.spark.sql.connector.read.Batch; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.PartitionReaderFactory; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.types.StructType; + + +public class VeniceHdfsInputScan implements Scan, Batch { + private final VeniceProperties jobConfig; + + public VeniceHdfsInputScan(VeniceProperties jobConfig) { + this.jobConfig = jobConfig; + } + + @Override + public InputPartition[] planInputPartitions() { + try { + Path inputDirPath = new Path(jobConfig.getString(INPUT_PATH_PROP)); + FileSystem fs = FileSystem.get(new Configuration()); + List inputPartitionList = new ArrayList<>(); + // For now, we create 1 file as 1 InputPartition. This is not the most ideal, because Avro allows splitting files + // to a smaller granularity using sync markers. We can explore later if we feel we need that optimization. + RemoteIterator fileStatusIterator = fs.listFiles(inputDirPath, false); + while (fileStatusIterator.hasNext()) { + LocatedFileStatus fileStatus = fileStatusIterator.next(); + Path filePath = fileStatus.getPath(); + if (PATH_FILTER.accept(filePath)) { + inputPartitionList.add(new VeniceHdfsInputPartition(fileStatus.getPath())); + } + } + return inputPartitionList.toArray(new VeniceHdfsInputPartition[0]); + } catch (IOException e) { + throw new VeniceException("Could not get FileSystem", e); + } + } + + @Override + public PartitionReaderFactory createReaderFactory() { + return new VeniceHdfsInputPartitionReaderFactory(jobConfig); + } + + @Override + public StructType readSchema() { + return SparkConstants.DEFAULT_SCHEMA; + } + + @Override + public Batch toBatch() { + return this; + } +} 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/hadoop/spark/input/hdfs/VeniceHdfsInputScanBuilder.java new file mode 100644 index 00000000000..42e900e12e0 --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputScanBuilder.java @@ -0,0 +1,19 @@ +package com.linkedin.venice.hadoop.spark.input.hdfs; + +import com.linkedin.venice.utils.VeniceProperties; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.connector.read.ScanBuilder; + + +public class VeniceHdfsInputScanBuilder implements ScanBuilder { + private final VeniceProperties jobConfig; + + public VeniceHdfsInputScanBuilder(VeniceProperties jobConfig) { + this.jobConfig = jobConfig; + } + + @Override + public Scan build() { + return new VeniceHdfsInputScan(jobConfig); + } +} 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/hadoop/spark/input/hdfs/VeniceHdfsInputTable.java new file mode 100644 index 00000000000..5b8b3bbca44 --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsInputTable.java @@ -0,0 +1,49 @@ +package com.linkedin.venice.hadoop.spark.input.hdfs; + +import static com.linkedin.venice.hadoop.spark.SparkConstants.DEFAULT_SCHEMA; + +import com.linkedin.venice.utils.VeniceProperties; +import java.util.Collections; +import java.util.Properties; +import java.util.Set; +import org.apache.spark.sql.connector.catalog.SupportsRead; +import org.apache.spark.sql.connector.catalog.TableCapability; +import org.apache.spark.sql.connector.read.ScanBuilder; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + + +/** + * A table format that is used by Spark to read Avro files from HDFS for use in VenicePushJob. + */ +public class VeniceHdfsInputTable implements SupportsRead { + private final VeniceProperties jobConfig; + // Visible for testing + static final String INPUT_TABLE_NAME = "venice_hdfs_table"; + + public VeniceHdfsInputTable(VeniceProperties jobConfig) { + this.jobConfig = jobConfig; + } + + @Override + public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { + Properties properties = jobConfig.getPropertiesCopy(); + properties.putAll(options.asCaseSensitiveMap()); + return new VeniceHdfsInputScanBuilder(new VeniceProperties(properties)); + } + + @Override + public String name() { + return INPUT_TABLE_NAME; + } + + @Override + public StructType schema() { + return DEFAULT_SCHEMA; + } + + @Override + public Set capabilities() { + return Collections.singleton(TableCapability.BATCH_READ); + } +} 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/hadoop/spark/input/hdfs/VeniceHdfsSource.java new file mode 100644 index 00000000000..9e76f4f3c1c --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/input/hdfs/VeniceHdfsSource.java @@ -0,0 +1,33 @@ +package com.linkedin.venice.hadoop.spark.input.hdfs; + +import static com.linkedin.venice.hadoop.spark.SparkConstants.DEFAULT_SCHEMA; + +import com.linkedin.venice.utils.VeniceProperties; +import java.util.Map; +import java.util.Properties; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableProvider; +import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + + +/** + * This is the entrypoint of the Avro input source. It is used by Spark to create a DataFrame from a directory on + * HDFS. The directory must contain either Avro or Vson files. The format of input files must be homogenous, i.e., it + * cannot contain mixed formats or schemas. + */ +@SuppressWarnings("unused") +public class VeniceHdfsSource implements TableProvider { + @Override + public StructType inferSchema(CaseInsensitiveStringMap options) { + return DEFAULT_SCHEMA; + } + + @Override + public Table getTable(StructType schema, Transform[] partitioning, Map configs) { + Properties properties = new Properties(); + properties.putAll(configs); + return new VeniceHdfsInputTable(new VeniceProperties(properties)); + } +} 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/hadoop/spark/utils/SparkPartitionUtils.java new file mode 100644 index 00000000000..9fb1758a33b --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/utils/SparkPartitionUtils.java @@ -0,0 +1,39 @@ +package com.linkedin.venice.hadoop.spark.utils; + +import java.util.Comparator; +import org.apache.spark.Partitioner; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.rdd.RDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import scala.Tuple2; + + +/** + * Spark partitioning functionality in Dataframe and Dataset APIs is not very flexible. This class provides some + * functionality by using the underlying RDD implementation. + */ +public final class SparkPartitionUtils { + private SparkPartitionUtils() { + } + + /** + * This function provides the equivalent of {@link JavaPairRDD#repartitionAndSortWithinPartitions} in Dataframe API. + * 1. Convert to {@link JavaPairRDD} + * 2. Use {@link JavaPairRDD#repartitionAndSortWithinPartitions} to partition and perform primary and secondary sort + * 3. Convert {@link JavaPairRDD} to {@link RDD} + * 4. Convert {@link RDD} back to Dataframe + */ + public static Dataset repartitionAndSortWithinPartitions( + Dataset df, + Partitioner partitioner, + Comparator comparator) { + RDD partitionedRowRDD = df.javaRDD() + .mapToPair((PairFunction) row -> new Tuple2<>(row, row)) + .repartitionAndSortWithinPartitions(partitioner, comparator) + .map(v1 -> v1._1) + .rdd(); + return df.sparkSession().createDataFrame(partitionedRowRDD, df.schema()); + } +} 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/hadoop/spark/utils/SparkScalaUtils.java new file mode 100644 index 00000000000..33e3ac1ba44 --- /dev/null +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/spark/utils/SparkScalaUtils.java @@ -0,0 +1,33 @@ +package com.linkedin.venice.hadoop.spark.utils; + +import org.apache.spark.sql.types.StructType; +import scala.Int; +import scala.Option; + + +/** + * Utility methods for restricting Scala-ism in Spark code in a single class. + */ +public final class SparkScalaUtils { + private SparkScalaUtils() { + } + + /** + * Get the index of a field in a StructType. If the field does not exist, return -1. + */ + public static int getFieldIndex(StructType dataSchema, String fieldName) { + Option fieldIndexOption = dataSchema.getFieldIndex(fieldName); + if (fieldIndexOption.isEmpty()) { + return -1; + } + + Object item = fieldIndexOption.get(); + if (item instanceof Int) { + return ((Int) item).toInt(); + } else if (item instanceof Integer) { + return (Integer) item; + } + + return -1; + } +} 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 1f62e9b0e19..515128d0b98 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 @@ -472,7 +472,18 @@ private void initStorageQuotaFields(VeniceProperties props) { } } - protected abstract long getTotalIncomingDataSizeInBytes(); + /** + * Return the size of serialized key and serialized value in bytes across the entire dataset. This is an optimization + * to skip writing the data to Kafka and reduce the load on Kafka and Venice storage nodes. Not all engines can + * support fetching this information during the execution of the job (eg Spark), but we can live with it for now. The + * quota is checked again in the Driver after the completion of the DataWriter job, and it will kill the VenicePushJob + * soon after. + * + * @return the size of serialized key and serialized value in bytes across the entire dataset + */ + protected long getTotalIncomingDataSizeInBytes() { + return 0; + } private void setSendException(Exception e) { sendException = e; diff --git a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/utils/HadoopUtils.java b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/utils/HadoopUtils.java index ce0b3c9772c..61d0006ec00 100644 --- a/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/utils/HadoopUtils.java +++ b/clients/venice-push-job/src/main/java/com/linkedin/venice/hadoop/utils/HadoopUtils.java @@ -17,6 +17,9 @@ public class HadoopUtils { private static final Logger LOGGER = LogManager.getLogger(HadoopUtils.class); + private HadoopUtils() { + } + public static VeniceProperties getVeniceProps(JobConf conf) { return new VeniceProperties(getProps(conf)); } @@ -51,7 +54,4 @@ public static void cleanUpHDFSPath(String path, boolean recursive) { LOGGER.error("Failed to clean up the HDFS path: {}", path); } } - - private HadoopUtils() { - } } 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 8caf4e87879..09ee6e24655 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 @@ -204,7 +204,7 @@ public void testWithNoMapperToBuildDictionary() throws Exception { Arrays.asList( VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.MAP_REDUCE_JOB_COMPLETED, + VenicePushJob.PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, VenicePushJob.PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), properties -> { properties.setProperty(USE_MAPPER_TO_BUILD_DICTIONARY, "false"); @@ -230,7 +230,7 @@ public void testWithMapperToBuildDictionary() throws Exception { VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, VenicePushJob.PushJobCheckpoints.VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB_COMPLETED, VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.MAP_REDUCE_JOB_COMPLETED, + VenicePushJob.PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, VenicePushJob.PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), properties -> { properties.setProperty(USE_MAPPER_TO_BUILD_DICTIONARY, "true"); @@ -250,7 +250,7 @@ public void testWithCompressionCollectionDisabled() throws Exception { Arrays.asList( VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.MAP_REDUCE_JOB_COMPLETED, + VenicePushJob.PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, VenicePushJob.PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), properties -> { properties.setProperty(USE_MAPPER_TO_BUILD_DICTIONARY, "false"); @@ -278,7 +278,7 @@ public void testWithCompressionCollectionEnabled() throws Exception { VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, VenicePushJob.PushJobCheckpoints.VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB_COMPLETED, VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.MAP_REDUCE_JOB_COMPLETED, + VenicePushJob.PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, VenicePushJob.PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), properties -> { properties.setProperty(USE_MAPPER_TO_BUILD_DICTIONARY, "false"); @@ -310,7 +310,7 @@ public void testWithCompressionCollectionEnabledV1() throws Exception { VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, VenicePushJob.PushJobCheckpoints.VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB_COMPLETED, VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.MAP_REDUCE_JOB_COMPLETED, + VenicePushJob.PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, VenicePushJob.PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), properties -> { properties.setProperty(USE_MAPPER_TO_BUILD_DICTIONARY, "true"); @@ -343,7 +343,7 @@ public void testHandlingFailureWithCompressionCollectionEnabled() throws Excepti VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, VenicePushJob.PushJobCheckpoints.VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB_COMPLETED, VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.MAP_REDUCE_JOB_COMPLETED, + VenicePushJob.PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, VenicePushJob.PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), properties -> { properties.setProperty(USE_MAPPER_TO_BUILD_DICTIONARY, "true"); @@ -407,7 +407,7 @@ public void testHandlingSkippedWithCompressionCollectionEnabled() throws Excepti VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, VenicePushJob.PushJobCheckpoints.VALIDATE_SCHEMA_AND_BUILD_DICT_MAP_JOB_COMPLETED, VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.MAP_REDUCE_JOB_COMPLETED, + VenicePushJob.PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, VenicePushJob.PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), properties -> { properties.setProperty(USE_MAPPER_TO_BUILD_DICTIONARY, "true"); @@ -502,7 +502,7 @@ public void testHandleZeroClosedReducersFailure() throws Exception { Arrays.asList( VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.START_MAP_REDUCE_JOB), + VenicePushJob.PushJobCheckpoints.START_DATA_WRITER_JOB), 10L, // Non-empty input data file properties -> { properties.setProperty(COMPRESSION_METRIC_COLLECTION_ENABLED, "false"); @@ -527,7 +527,7 @@ public void testUnreliableMapReduceCounter() throws Exception { Arrays.asList( VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.START_MAP_REDUCE_JOB), + VenicePushJob.PushJobCheckpoints.START_DATA_WRITER_JOB), 10L, // Non-empty input data file 1, true, @@ -552,7 +552,7 @@ public void testHandleZeroClosedReducersWithNoRecordInputDataFile() throws Excep Arrays.asList( VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.MAP_REDUCE_JOB_COMPLETED, + VenicePushJob.PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, VenicePushJob.PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED // Expect the job to finish successfully ), 10L, @@ -593,7 +593,7 @@ public void testHandleInsufficientClosedReducersFailure() throws Exception { Arrays.asList( VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.START_MAP_REDUCE_JOB), + VenicePushJob.PushJobCheckpoints.START_DATA_WRITER_JOB), properties -> { properties.setProperty(COMPRESSION_METRIC_COLLECTION_ENABLED, "false"); properties.setProperty(USE_MAPPER_TO_BUILD_DICTIONARY, "false"); @@ -618,7 +618,7 @@ public void testCounterValidationWhenSprayAllPartitionsNotTriggeredButWithMismat Arrays.asList( VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.MAP_REDUCE_JOB_COMPLETED, + VenicePushJob.PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, VenicePushJob.PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), properties -> { properties.setProperty(COMPRESSION_METRIC_COLLECTION_ENABLED, "false"); @@ -641,7 +641,7 @@ public void testHandleNoErrorInCounters() throws Exception { // Successful workf Arrays.asList( VenicePushJob.PushJobCheckpoints.INITIALIZE_PUSH_JOB, VenicePushJob.PushJobCheckpoints.NEW_VERSION_CREATED, - VenicePushJob.PushJobCheckpoints.MAP_REDUCE_JOB_COMPLETED, + VenicePushJob.PushJobCheckpoints.DATA_WRITER_JOB_COMPLETED, VenicePushJob.PushJobCheckpoints.JOB_STATUS_POLLING_COMPLETED), properties -> { properties.setProperty(COMPRESSION_METRIC_COLLECTION_ENABLED, "false"); diff --git a/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/recordreader/avro/IdentityVeniceRecordReaderTest.java b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/recordreader/avro/IdentityVeniceRecordReaderTest.java new file mode 100644 index 00000000000..03b23d2ae6f --- /dev/null +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/input/recordreader/avro/IdentityVeniceRecordReaderTest.java @@ -0,0 +1,39 @@ +package com.linkedin.venice.hadoop.input.recordreader.avro; + +import com.linkedin.venice.exceptions.VeniceUnsupportedOperationException; +import com.linkedin.venice.utils.ArrayUtils; +import java.nio.ByteBuffer; +import org.testng.Assert; +import org.testng.annotations.Test; + + +public class IdentityVeniceRecordReaderTest { + private static final IdentityVeniceRecordReader RECORD_READER = IdentityVeniceRecordReader.getInstance(); + private static final byte[] TEST_KEY_BYTES = "RANDOM_KEY".getBytes(); + private static final byte[] TEST_VALUE_BYTES = "RANDOM_VALUE".getBytes(); + + @Test + public void testGetKeyBytes() { + byte[] extractedKey = RECORD_READER.getKeyBytes(ByteBuffer.wrap(TEST_KEY_BYTES), ByteBuffer.wrap(TEST_VALUE_BYTES)); + + Assert.assertEquals(ArrayUtils.compareUnsigned(TEST_KEY_BYTES, extractedKey), 0); + } + + @Test + public void testGetValueBytes() { + byte[] extractedValue = + RECORD_READER.getValueBytes(ByteBuffer.wrap(TEST_KEY_BYTES), ByteBuffer.wrap(TEST_VALUE_BYTES)); + + Assert.assertEquals(ArrayUtils.compareUnsigned(TEST_VALUE_BYTES, extractedValue), 0); + } + + @Test + public void testUnsupportedGetAvroData() { + Assert.assertThrows( + VeniceUnsupportedOperationException.class, + () -> RECORD_READER.getAvroKey(ByteBuffer.wrap(TEST_KEY_BYTES), ByteBuffer.wrap(TEST_VALUE_BYTES))); + Assert.assertThrows( + VeniceUnsupportedOperationException.class, + () -> RECORD_READER.getAvroValue(ByteBuffer.wrap(TEST_KEY_BYTES), ByteBuffer.wrap(TEST_VALUE_BYTES))); + } +} 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/hadoop/jobs/DataWriterComputeJobTest.java new file mode 100644 index 00000000000..58a9bcc3efd --- /dev/null +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/jobs/DataWriterComputeJobTest.java @@ -0,0 +1,30 @@ +package com.linkedin.venice.hadoop.jobs; + +import static com.linkedin.venice.hadoop.jobs.DataWriterComputeJob.PASS_THROUGH_CONFIG_PREFIXES; + +import com.linkedin.venice.exceptions.VeniceException; +import org.testng.annotations.Test; + + +public class DataWriterComputeJobTest { + @Test + public void testNoOverlappingPassThroughConfigPrefixes() { + int passThroughPrefixListSize = PASS_THROUGH_CONFIG_PREFIXES.size(); + /** + * The following logic will make sure there is no prefix that is a prefix of another prefix. + */ + for (int i = 0; i < passThroughPrefixListSize; ++i) { + for (int j = i + 1; j < passThroughPrefixListSize; ++j) { + String prefixI = PASS_THROUGH_CONFIG_PREFIXES.get(i); + String prefixJ = PASS_THROUGH_CONFIG_PREFIXES.get(j); + if (prefixI.startsWith(prefixJ)) { + throw new VeniceException("Prefix: " + prefixJ + " shouldn't be a prefix of another prefix: " + prefixI); + } + + if (prefixJ.startsWith(prefixI)) { + throw new VeniceException("Prefix: " + prefixI + " shouldn't be a prefix of another prefix: " + prefixJ); + } + } + } + } +} 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/hadoop/spark/datawriter/jobs/AbstractDataWriterSparkJobTest.java new file mode 100644 index 00000000000..bcb7abef6d9 --- /dev/null +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/datawriter/jobs/AbstractDataWriterSparkJobTest.java @@ -0,0 +1,261 @@ +package com.linkedin.venice.hadoop.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 org.apache.spark.sql.types.DataTypes.BinaryType; +import static org.apache.spark.sql.types.DataTypes.StringType; + +import com.linkedin.venice.compression.CompressionStrategy; +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.meta.Version; +import com.linkedin.venice.partitioner.DefaultVenicePartitioner; +import com.linkedin.venice.serializer.FastSerializerDeserializerFactory; +import com.linkedin.venice.serializer.RecordSerializer; +import com.linkedin.venice.utils.TestWriteUtils; +import com.linkedin.venice.utils.Utils; +import com.linkedin.venice.utils.VeniceProperties; +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.Properties; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.Path; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.rdd.RDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.RuntimeConfig; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.testng.Assert; +import org.testng.annotations.Test; + + +public class AbstractDataWriterSparkJobTest { + @Test + public void testConfigure() throws IOException { + File inputDir = TestWriteUtils.getTempDataDirectory(); + Schema dataSchema = TestWriteUtils.writeSimpleAvroFileWithStringToStringSchema(inputDir); + + PushJobSetting setting = getDefaultPushJobSetting(inputDir, dataSchema); + String sparkAppNameOverride = "UPDATED_SPARK_APP_NAME"; + String dummyKafkaConfig = "dummy.kafka.config"; + String dummyKafkaConfigValue = "dummy.kafka.config.value"; + String dummyConfig = "some.dummy.config"; + String dummyConfigValue = "some.dummy.config.value"; + + Properties properties = new Properties(); + properties.setProperty(SPARK_SESSION_CONF_PREFIX + SPARK_APP_NAME_CONFIG, sparkAppNameOverride); + properties.setProperty(KAFKA_CONFIG_PREFIX + dummyKafkaConfig, dummyKafkaConfigValue); + properties.setProperty(SPARK_DATA_WRITER_CONF_PREFIX + dummyConfig, dummyConfigValue); + + try (DataWriterSparkJob dataWriterSparkJob = new DataWriterSparkJob()) { + dataWriterSparkJob.configure(new VeniceProperties(properties), setting); + + RuntimeConfig jobConf = dataWriterSparkJob.getSparkSession().conf(); + // Builder configs should get applied + Assert.assertEquals(jobConf.get(SPARK_APP_NAME_CONFIG), sparkAppNameOverride); + + // Pass through properties should get applied without stripping the prefix + Assert.assertEquals(jobConf.get(KAFKA_CONFIG_PREFIX + dummyKafkaConfig), dummyKafkaConfigValue); + + // Properties with SPARK_DATA_WRITER_CONF_PREFIX should get applied after stripping the prefix + Assert.assertEquals(jobConf.get(dummyConfig), dummyConfigValue); + } + } + + @Test + public void testValidateDataFrameSchema() throws IOException { + File inputDir = TestWriteUtils.getTempDataDirectory(); + Schema dataSchema = TestWriteUtils.writeSimpleAvroFileWithStringToStringSchema(inputDir); + PushJobSetting setting = getDefaultPushJobSetting(inputDir, dataSchema); + Properties properties = new Properties(); + try (DataWriterComputeJob computeJob = new InvalidKeySchemaDataWriterSparkJob()) { + Assert.assertThrows(VeniceException.class, () -> computeJob.configure(new VeniceProperties(properties), setting)); + } + + try (DataWriterComputeJob computeJob = new InvalidValueSchemaDataWriterSparkJob()) { + Assert.assertThrows(VeniceException.class, () -> computeJob.configure(new VeniceProperties(properties), setting)); + } + + try (DataWriterComputeJob computeJob = new IncompleteFieldDataWriterSparkJob()) { + Assert.assertThrows(VeniceException.class, () -> computeJob.configure(new VeniceProperties(properties), setting)); + } + + try (DataWriterComputeJob computeJob = new MissingKeyFieldDataWriterSparkJob()) { + Assert.assertThrows(VeniceException.class, () -> computeJob.configure(new VeniceProperties(properties), setting)); + } + + try (DataWriterComputeJob computeJob = new MissingValueFieldDataWriterSparkJob()) { + Assert.assertThrows(VeniceException.class, () -> computeJob.configure(new VeniceProperties(properties), setting)); + } + + try (DataWriterComputeJob computeJob = new SchemaWithRestrictedFieldDataWriterSparkJob()) { + Assert.assertThrows(VeniceException.class, () -> computeJob.configure(new VeniceProperties(properties), setting)); + } + } + + private PushJobSetting getDefaultPushJobSetting(File inputDir, Schema dataSchema) { + PushJobSetting setting = new PushJobSetting(); + setting.storeName = Utils.getUniqueString("TEST_STORE"); + setting.jobId = Utils.getUniqueString("TEST_JOB"); + setting.inputURI = new Path(inputDir.toURI()).toString(); + setting.keyField = DEFAULT_KEY_FIELD_PROP; + setting.valueField = DEFAULT_VALUE_FIELD_PROP; + setting.topic = Version.composeKafkaTopic(setting.storeName, 7); + setting.kafkaUrl = "localhost:9092"; + setting.partitionerClass = DefaultVenicePartitioner.class.getCanonicalName(); + setting.partitionerParams = null; + setting.amplificationFactor = 1; + setting.sslToKafka = false; + setting.isDuplicateKeyAllowed = false; + setting.chunkingEnabled = false; + setting.rmdChunkingEnabled = false; + setting.storeKeySchema = Schema.create(Schema.Type.STRING); + setting.storeStorageQuota = UNLIMITED_STORAGE_QUOTA; + setting.storeCompressionStrategy = CompressionStrategy.NO_OP; + setting.sendControlMessagesDirectly = true; + setting.isSourceETL = false; + setting.isSourceKafka = false; + setting.isAvro = true; + setting.valueSchemaId = 1; + setting.inputDataSchema = dataSchema; + setting.inputDataSchemaString = setting.inputDataSchema.toString(); + setting.keySchema = dataSchema.getField(DEFAULT_KEY_FIELD_PROP).schema(); + setting.keySchemaString = setting.keySchema.toString(); + setting.valueSchema = dataSchema.getField(DEFAULT_VALUE_FIELD_PROP).schema(); + setting.valueSchemaString = setting.valueSchema.toString(); + setting.etlValueSchemaTransformation = ETLValueSchemaTransformation.NONE; + setting.inputHasRecords = true; + setting.inputFileDataSizeInBytes = 1000; + setting.topicCompressionStrategy = setting.storeCompressionStrategy; + return setting; + } + + private static class InvalidKeySchemaDataWriterSparkJob extends AbstractDataWriterSparkJob { + private static final StructType INVALID_SCHEMA = new StructType( + new StructField[] { new StructField(KEY_COLUMN_NAME, StringType, false, Metadata.empty()), + new StructField(VALUE_COLUMN_NAME, BinaryType, true, Metadata.empty()) }); + private static final RecordSerializer serializer = + FastSerializerDeserializerFactory.getFastAvroGenericSerializer(Schema.create(Schema.Type.STRING)); + + @Override + protected Dataset getUserInputDataFrame() { + SparkSession spark = getSparkSession(); + JavaSparkContext sparkContext = new JavaSparkContext(spark.sparkContext()); + RDD rowRDD = sparkContext.parallelize(Arrays.asList("1", "2", "3")) + .map(item -> RowFactory.create(item, serializer.serialize(item))) + .rdd(); + + return spark.createDataFrame(rowRDD, INVALID_SCHEMA); + } + } + + private static class InvalidValueSchemaDataWriterSparkJob extends AbstractDataWriterSparkJob { + private static final StructType INVALID_SCHEMA = new StructType( + new StructField[] { new StructField(KEY_COLUMN_NAME, BinaryType, false, Metadata.empty()), + new StructField(VALUE_COLUMN_NAME, StringType, true, Metadata.empty()) }); + private static final RecordSerializer serializer = + FastSerializerDeserializerFactory.getFastAvroGenericSerializer(Schema.create(Schema.Type.STRING)); + + @Override + protected Dataset getUserInputDataFrame() { + SparkSession spark = getSparkSession(); + JavaSparkContext sparkContext = new JavaSparkContext(spark.sparkContext()); + RDD rowRDD = sparkContext.parallelize(Arrays.asList("1", "2", "3")) + .map(item -> RowFactory.create(serializer.serialize(item), item)) + .rdd(); + + return spark.createDataFrame(rowRDD, INVALID_SCHEMA); + } + } + + private static class IncompleteFieldDataWriterSparkJob extends AbstractDataWriterSparkJob { + private static final StructType INVALID_SCHEMA = + new StructType(new StructField[] { new StructField(KEY_COLUMN_NAME, BinaryType, false, Metadata.empty()) }); + private static final RecordSerializer serializer = + FastSerializerDeserializerFactory.getFastAvroGenericSerializer(Schema.create(Schema.Type.STRING)); + + @Override + protected Dataset getUserInputDataFrame() { + SparkSession spark = getSparkSession(); + JavaSparkContext sparkContext = new JavaSparkContext(spark.sparkContext()); + RDD rowRDD = sparkContext.parallelize(Arrays.asList("1", "2", "3")) + .map(item -> RowFactory.create((Object) serializer.serialize(item))) + .rdd(); + + return spark.createDataFrame(rowRDD, INVALID_SCHEMA); + } + } + + private static class MissingKeyFieldDataWriterSparkJob extends AbstractDataWriterSparkJob { + private static final StructType INVALID_SCHEMA = new StructType( + new StructField[] { new StructField("DUMMY_FIELD", BinaryType, false, Metadata.empty()), + new StructField(VALUE_COLUMN_NAME, BinaryType, true, Metadata.empty()) }); + private static final RecordSerializer serializer = + FastSerializerDeserializerFactory.getFastAvroGenericSerializer(Schema.create(Schema.Type.STRING)); + + @Override + protected Dataset getUserInputDataFrame() { + SparkSession spark = getSparkSession(); + JavaSparkContext sparkContext = new JavaSparkContext(spark.sparkContext()); + RDD rowRDD = sparkContext.parallelize(Arrays.asList("1", "2", "3")) + .map(item -> RowFactory.create(serializer.serialize(item), serializer.serialize(item))) + .rdd(); + + return spark.createDataFrame(rowRDD, INVALID_SCHEMA); + } + } + + private static class MissingValueFieldDataWriterSparkJob extends AbstractDataWriterSparkJob { + private static final StructType INVALID_SCHEMA = new StructType( + new StructField[] { new StructField(KEY_COLUMN_NAME, BinaryType, false, Metadata.empty()), + new StructField("DUMMY_FIELD", BinaryType, true, Metadata.empty()) }); + private static final RecordSerializer serializer = + FastSerializerDeserializerFactory.getFastAvroGenericSerializer(Schema.create(Schema.Type.STRING)); + + @Override + protected Dataset getUserInputDataFrame() { + SparkSession spark = getSparkSession(); + JavaSparkContext sparkContext = new JavaSparkContext(spark.sparkContext()); + RDD rowRDD = sparkContext.parallelize(Arrays.asList("1", "2", "3")) + .map(item -> RowFactory.create(serializer.serialize(item), serializer.serialize(item))) + .rdd(); + + return spark.createDataFrame(rowRDD, INVALID_SCHEMA); + } + } + + private static class SchemaWithRestrictedFieldDataWriterSparkJob extends AbstractDataWriterSparkJob { + private static final StructType INVALID_SCHEMA = new StructType( + new StructField[] { new StructField(KEY_COLUMN_NAME, BinaryType, false, Metadata.empty()), + new StructField(VALUE_COLUMN_NAME, BinaryType, true, Metadata.empty()), + new StructField("_INTERNAL_FIELD", StringType, true, Metadata.empty()) }); + private static final RecordSerializer serializer = + FastSerializerDeserializerFactory.getFastAvroGenericSerializer(Schema.create(Schema.Type.STRING)); + + @Override + protected Dataset getUserInputDataFrame() { + SparkSession spark = getSparkSession(); + JavaSparkContext sparkContext = new JavaSparkContext(spark.sparkContext()); + RDD rowRDD = sparkContext.parallelize(Arrays.asList("1", "2", "3")) + .map(item -> RowFactory.create(serializer.serialize(item), serializer.serialize(item), item)) + .rdd(); + + return spark.createDataFrame(rowRDD, INVALID_SCHEMA); + } + } +} 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/hadoop/spark/datawriter/partition/PartitionSorterTest.java new file mode 100644 index 00000000000..289adee8ddd --- /dev/null +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/datawriter/partition/PartitionSorterTest.java @@ -0,0 +1,42 @@ +package com.linkedin.venice.hadoop.spark.datawriter.partition; + +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.testng.Assert; +import org.testng.annotations.Test; + + +public class PartitionSorterTest { + @Test + public void testCompare() { + PartitionSorter partitionSorter = new PartitionSorter(); + + byte[] key1 = new byte[] { 1, 100 }; + byte[] value1 = new byte[] { 1, 10 }; + Row row1 = RowFactory.create(key1, value1); + + byte[] key2 = new byte[] { 1, 10 }; + byte[] value2 = new byte[] { 1, 10 }; + Row row2 = RowFactory.create(key2, value2); + + // key1 > key2 + Assert.assertEquals(partitionSorter.compare(row1, row2), 90); + + // key2 < key1 + Assert.assertEquals(partitionSorter.compare(row2, row1), -90); + + byte[] key3 = new byte[] { 1, 100 }; + byte[] value3 = new byte[] { 1, 100 }; + Row row3 = RowFactory.create(key3, value3); + + // key1 = key3, value1 < value3 + Assert.assertEquals(partitionSorter.compare(row1, row3), -90); + + byte[] key4 = new byte[] { 1, -100 }; // For unsigned comparison, this is 156. For signed comparison, this is -100. + byte[] value4 = new byte[] { 1, 10 }; + Row row4 = RowFactory.create(key4, value4); + + // key1 < k4 if using unsigned, but key1 > k4 if using signed + Assert.assertEquals(partitionSorter.compare(row1, row4), -56); + } +} 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/hadoop/spark/datawriter/partition/VeniceSparkPartitionerTest.java new file mode 100644 index 00000000000..320ad8d69ce --- /dev/null +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/datawriter/partition/VeniceSparkPartitionerTest.java @@ -0,0 +1,83 @@ +package com.linkedin.venice.hadoop.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.utils.ByteUtils; +import com.linkedin.venice.utils.VeniceProperties; +import java.util.Properties; +import org.apache.commons.lang3.RandomUtils; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.SparkSession; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + + +public class VeniceSparkPartitionerTest { + private SparkSession spark; + private static final int TEST_PARTITION_COUNT = 10; + + private final Properties properties = new Properties(); + private Broadcast broadcastProperties; + private VeniceSparkPartitioner veniceSparkPartitioner; + + @BeforeClass + public void setUp() { + spark = SparkSession.builder().appName("TestApp").master(SparkConstants.DEFAULT_SPARK_CLUSTER).getOrCreate(); + properties.setProperty(PARTITION_COUNT, String.valueOf(TEST_PARTITION_COUNT)); + + JavaSparkContext sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + broadcastProperties = sparkContext.broadcast(properties); + veniceSparkPartitioner = new VeniceSparkPartitioner(broadcastProperties, TEST_PARTITION_COUNT); + } + + @AfterClass(alwaysRun = true) + public void tearDown() { + spark.stop(); + } + + @Test + public void testGetPartition() { + byte[] key = RandomUtils.nextBytes(4); + byte[] value = RandomUtils.nextBytes(5); + Row row = RowFactory.create(key, value); + + VenicePartitioner expectedPartitioner = new DefaultVenicePartitioner(new VeniceProperties(properties)); + Assert.assertEquals( + veniceSparkPartitioner.getPartition(row), + expectedPartitioner.getPartitionId(key, TEST_PARTITION_COUNT)); + + int sprayedPartition = 3; + byte[] sprayedKey = new byte[0]; + byte[] sprayedValue = new byte[Integer.BYTES]; + ByteUtils.writeInt(sprayedValue, sprayedPartition, 0); + Row sprayedRow = RowFactory.create(sprayedKey, sprayedValue); + + Assert.assertEquals(veniceSparkPartitioner.getPartition(sprayedRow), sprayedPartition); + } + + @Test + public void testNumPartitions() { + Assert.assertEquals(veniceSparkPartitioner.numPartitions(), TEST_PARTITION_COUNT); + } + + @Test + public void testGetPartitionWithIncorrectPartitionCount() { + Assert.assertThrows( + VeniceException.class, + () -> new VeniceSparkPartitioner(broadcastProperties, TEST_PARTITION_COUNT + 10)); + } + + @Test + public void testGetPartitionForNonUnsupportedInput() { + Assert.assertThrows(VeniceException.class, () -> veniceSparkPartitioner.getPartition("row")); + } +} 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/hadoop/spark/datawriter/task/SparkDataWriterTaskTrackerTest.java new file mode 100644 index 00000000000..c036f0eb4d4 --- /dev/null +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/datawriter/task/SparkDataWriterTaskTrackerTest.java @@ -0,0 +1,267 @@ +package com.linkedin.venice.hadoop.spark.datawriter.task; + +import com.linkedin.venice.hadoop.spark.SparkConstants; +import org.apache.spark.sql.SparkSession; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + + +public class SparkDataWriterTaskTrackerTest { + private SparkSession spark; + + @BeforeClass + public void setUp() { + spark = SparkSession.builder().appName("TestApp").master(SparkConstants.DEFAULT_SPARK_CLUSTER).getOrCreate(); + } + + @AfterClass(alwaysRun = true) + public void tearDown() { + spark.stop(); + } + + @Test + public void testSprayAllPartitions() { + DataWriterAccumulators accumulators = new DataWriterAccumulators(spark); + SparkDataWriterTaskTracker tracker = new SparkDataWriterTaskTracker(accumulators); + + tracker.trackSprayAllPartitions(); + Assert.assertEquals(tracker.getSprayAllPartitionsCount(), 1); + + DataWriterAccumulators expectedAccumulators = new DataWriterAccumulators(spark); + expectedAccumulators.sprayAllPartitionsTriggeredCount.add(1); + + verifyAllAccumulators(accumulators, expectedAccumulators); + } + + @Test + public void testEmptyRecord() { + DataWriterAccumulators accumulators = new DataWriterAccumulators(spark); + SparkDataWriterTaskTracker tracker = new SparkDataWriterTaskTracker(accumulators); + + tracker.trackEmptyRecord(); + + DataWriterAccumulators expectedAccumulators = new DataWriterAccumulators(spark); + expectedAccumulators.emptyRecordCounter.add(1); + + verifyAllAccumulators(accumulators, expectedAccumulators); + } + + @Test + public void testKeySize() { + DataWriterAccumulators accumulators = new DataWriterAccumulators(spark); + SparkDataWriterTaskTracker tracker = new SparkDataWriterTaskTracker(accumulators); + + tracker.trackKeySize(100); + Assert.assertEquals(tracker.getTotalKeySize(), 100); + + tracker.trackKeySize(100); + Assert.assertEquals(tracker.getTotalKeySize(), 200); + + DataWriterAccumulators expectedAccumulators = new DataWriterAccumulators(spark); + expectedAccumulators.totalKeySizeCounter.add(200); + + verifyAllAccumulators(accumulators, expectedAccumulators); + } + + @Test + public void testUncompressedValueSize() { + DataWriterAccumulators accumulators = new DataWriterAccumulators(spark); + SparkDataWriterTaskTracker tracker = new SparkDataWriterTaskTracker(accumulators); + + tracker.trackUncompressedValueSize(100); + Assert.assertEquals(tracker.getTotalUncompressedValueSize(), 100); + + tracker.trackUncompressedValueSize(100); + Assert.assertEquals(tracker.getTotalUncompressedValueSize(), 200); + + DataWriterAccumulators expectedAccumulators = new DataWriterAccumulators(spark); + expectedAccumulators.uncompressedValueSizeCounter.add(200); + + verifyAllAccumulators(accumulators, expectedAccumulators); + } + + @Test + public void testCompressedValueSize() { + DataWriterAccumulators accumulators = new DataWriterAccumulators(spark); + SparkDataWriterTaskTracker tracker = new SparkDataWriterTaskTracker(accumulators); + + tracker.trackCompressedValueSize(100); + Assert.assertEquals(tracker.getTotalValueSize(), 100); + + tracker.trackCompressedValueSize(100); + Assert.assertEquals(tracker.getTotalValueSize(), 200); + + DataWriterAccumulators expectedAccumulators = new DataWriterAccumulators(spark); + expectedAccumulators.compressedValueSizeCounter.add(200); + + verifyAllAccumulators(accumulators, expectedAccumulators); + } + + @Test + public void testGzipCompressedValueSize() { + DataWriterAccumulators accumulators = new DataWriterAccumulators(spark); + SparkDataWriterTaskTracker tracker = new SparkDataWriterTaskTracker(accumulators); + + tracker.trackGzipCompressedValueSize(100); + Assert.assertEquals(tracker.getTotalGzipCompressedValueSize(), 100); + + tracker.trackGzipCompressedValueSize(100); + Assert.assertEquals(tracker.getTotalGzipCompressedValueSize(), 200); + + DataWriterAccumulators expectedAccumulators = new DataWriterAccumulators(spark); + expectedAccumulators.gzipCompressedValueSizeCounter.add(200); + + verifyAllAccumulators(accumulators, expectedAccumulators); + } + + @Test + public void testZstdCompressedValueSize() { + DataWriterAccumulators accumulators = new DataWriterAccumulators(spark); + SparkDataWriterTaskTracker tracker = new SparkDataWriterTaskTracker(accumulators); + + tracker.trackZstdCompressedValueSize(100); + Assert.assertEquals(tracker.getTotalZstdCompressedValueSize(), 100); + + tracker.trackZstdCompressedValueSize(100); + Assert.assertEquals(tracker.getTotalZstdCompressedValueSize(), 200); + + DataWriterAccumulators expectedAccumulators = new DataWriterAccumulators(spark); + expectedAccumulators.zstdCompressedValueSizeCounter.add(200); + + verifyAllAccumulators(accumulators, expectedAccumulators); + } + + @Test + public void testWriteAclAuthorizationFailure() { + DataWriterAccumulators accumulators = new DataWriterAccumulators(spark); + SparkDataWriterTaskTracker tracker = new SparkDataWriterTaskTracker(accumulators); + + tracker.trackWriteAclAuthorizationFailure(); + tracker.trackWriteAclAuthorizationFailure(); + Assert.assertEquals(tracker.getWriteAclAuthorizationFailureCount(), 2); + + DataWriterAccumulators expectedAccumulators = new DataWriterAccumulators(spark); + expectedAccumulators.writeAclAuthorizationFailureCounter.add(2); + + verifyAllAccumulators(accumulators, expectedAccumulators); + } + + @Test + public void testRecordTooLargeFailure() { + DataWriterAccumulators accumulators = new DataWriterAccumulators(spark); + SparkDataWriterTaskTracker tracker = new SparkDataWriterTaskTracker(accumulators); + + tracker.trackRecordTooLargeFailure(); + tracker.trackRecordTooLargeFailure(); + Assert.assertEquals(tracker.getRecordTooLargeFailureCount(), 2); + + DataWriterAccumulators expectedAccumulators = new DataWriterAccumulators(spark); + expectedAccumulators.recordTooLargeFailureCounter.add(2); + + verifyAllAccumulators(accumulators, expectedAccumulators); + } + + @Test + public void testRecordSentToPubSub() { + DataWriterAccumulators accumulators = new DataWriterAccumulators(spark); + SparkDataWriterTaskTracker tracker = new SparkDataWriterTaskTracker(accumulators); + + tracker.trackRecordSentToPubSub(); + Assert.assertEquals(tracker.getOutputRecordsCount(), 1); + + DataWriterAccumulators expectedAccumulators = new DataWriterAccumulators(spark); + expectedAccumulators.outputRecordCounter.add(1); + + verifyAllAccumulators(accumulators, expectedAccumulators); + } + + @Test + public void testDuplicateKeyWithDistinctValue() { + DataWriterAccumulators accumulators = new DataWriterAccumulators(spark); + SparkDataWriterTaskTracker tracker = new SparkDataWriterTaskTracker(accumulators); + + tracker.trackDuplicateKeyWithDistinctValue(10); + Assert.assertEquals(tracker.getDuplicateKeyWithDistinctValueCount(), 10); + tracker.trackDuplicateKeyWithDistinctValue(20); + Assert.assertEquals(tracker.getDuplicateKeyWithDistinctValueCount(), 30); + + DataWriterAccumulators expectedAccumulators = new DataWriterAccumulators(spark); + expectedAccumulators.duplicateKeyWithDistinctValueCounter.add(30); + + verifyAllAccumulators(accumulators, expectedAccumulators); + } + + @Test + public void testDuplicateKeyWithIdenticalValue() { + DataWriterAccumulators accumulators = new DataWriterAccumulators(spark); + SparkDataWriterTaskTracker tracker = new SparkDataWriterTaskTracker(accumulators); + + tracker.trackDuplicateKeyWithIdenticalValue(10); + tracker.trackDuplicateKeyWithIdenticalValue(20); + + DataWriterAccumulators expectedAccumulators = new DataWriterAccumulators(spark); + expectedAccumulators.duplicateKeyWithIdenticalValueCounter.add(30); + + verifyAllAccumulators(accumulators, expectedAccumulators); + } + + @Test + public void testRepushTtlFilteredRecord() { + DataWriterAccumulators accumulators = new DataWriterAccumulators(spark); + SparkDataWriterTaskTracker tracker = new SparkDataWriterTaskTracker(accumulators); + + tracker.trackRepushTtlFilteredRecord(); + tracker.trackRepushTtlFilteredRecord(); + + Assert.assertEquals(tracker.getRepushTtlFilterCount(), 2); + + DataWriterAccumulators expectedAccumulators = new DataWriterAccumulators(spark); + expectedAccumulators.repushTtlFilteredRecordCounter.add(2); + + verifyAllAccumulators(accumulators, expectedAccumulators); + } + + @Test + public void testPartitionWriterClose() { + DataWriterAccumulators accumulators = new DataWriterAccumulators(spark); + SparkDataWriterTaskTracker tracker = new SparkDataWriterTaskTracker(accumulators); + + tracker.trackPartitionWriterClose(); + tracker.trackPartitionWriterClose(); + + Assert.assertEquals(tracker.getPartitionWriterCloseCount(), 2); + + DataWriterAccumulators expectedAccumulators = new DataWriterAccumulators(spark); + expectedAccumulators.partitionWriterCloseCounter.add(2); + + verifyAllAccumulators(accumulators, expectedAccumulators); + } + + // Verify values of all accumulators to ensure that they don't get updated through side effects + private void verifyAllAccumulators(DataWriterAccumulators actual, DataWriterAccumulators expected) { + Assert.assertEquals( + actual.sprayAllPartitionsTriggeredCount.value(), + expected.sprayAllPartitionsTriggeredCount.value()); + Assert.assertEquals(actual.emptyRecordCounter.value(), expected.emptyRecordCounter.value()); + Assert.assertEquals(actual.totalKeySizeCounter.value(), expected.totalKeySizeCounter.value()); + Assert.assertEquals(actual.uncompressedValueSizeCounter.value(), expected.uncompressedValueSizeCounter.value()); + Assert.assertEquals(actual.compressedValueSizeCounter.value(), expected.compressedValueSizeCounter.value()); + Assert.assertEquals(actual.gzipCompressedValueSizeCounter.value(), expected.gzipCompressedValueSizeCounter.value()); + Assert.assertEquals(actual.zstdCompressedValueSizeCounter.value(), expected.zstdCompressedValueSizeCounter.value()); + Assert.assertEquals(actual.outputRecordCounter.value(), expected.outputRecordCounter.value()); + Assert.assertEquals( + actual.duplicateKeyWithIdenticalValueCounter.value(), + expected.duplicateKeyWithIdenticalValueCounter.value()); + Assert.assertEquals( + actual.writeAclAuthorizationFailureCounter.value(), + expected.writeAclAuthorizationFailureCounter.value()); + Assert.assertEquals(actual.recordTooLargeFailureCounter.value(), expected.recordTooLargeFailureCounter.value()); + Assert.assertEquals( + actual.duplicateKeyWithDistinctValueCounter.value(), + expected.duplicateKeyWithDistinctValueCounter.value()); + Assert.assertEquals(actual.partitionWriterCloseCounter.value(), expected.partitionWriterCloseCounter.value()); + Assert.assertEquals(actual.repushTtlFilteredRecordCounter.value(), expected.repushTtlFilteredRecordCounter.value()); + } +} 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/hadoop/spark/engine/SparkEngineTaskConfigProviderTest.java new file mode 100644 index 00000000000..cb244cd6704 --- /dev/null +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/engine/SparkEngineTaskConfigProviderTest.java @@ -0,0 +1,71 @@ +package com.linkedin.venice.hadoop.spark.engine; + +import com.linkedin.venice.hadoop.spark.SparkConstants; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Properties; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.SparkSession; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + + +public class SparkEngineTaskConfigProviderTest { + private static final String TEST_APP_NAME = "SparkEngineTaskConfigProviderTest"; + + private SparkSession spark; + private JavaSparkContext sparkContext; + + @BeforeClass + public void setUp() { + spark = SparkSession.builder().appName(TEST_APP_NAME).master(SparkConstants.DEFAULT_SPARK_CLUSTER).getOrCreate(); + + // Will be closed when the spark session is closed + sparkContext = new JavaSparkContext(spark.sparkContext()); + } + + @AfterClass(alwaysRun = true) + public void tearDown() { + sparkContext.close(); + spark.stop(); + } + + @Test + public void testGetJobName() { + Properties jobProps = new Properties(); + String propKey1 = "TestPropKey"; + String propValue1 = "TestPropValue"; + jobProps.setProperty(propKey1, propValue1); + + List data = Arrays.asList("1", "2", "3", "4", "5"); + + int numPartitions = 2; + JavaRDD rowRDD = sparkContext.parallelize(data).repartition(numPartitions).map(RowFactory::create); + + List outputData = rowRDD.map((Row row) -> { + SparkEngineTaskConfigProvider sparkEngineTaskConfigProvider = new SparkEngineTaskConfigProvider(jobProps); + + // TODO: Why does this not work? + // Assert.assertEquals(sparkEngineTaskConfigProvider.getJobName(), TEST_APP_NAME); + + Properties taskJobProps = sparkEngineTaskConfigProvider.getJobProps(); + jobProps.forEach((key, value) -> Assert.assertEquals(taskJobProps.getProperty((String) key), value)); + + return sparkEngineTaskConfigProvider.getTaskId(); + }).collect(); + + List mutableOutput = new ArrayList<>(outputData); + mutableOutput.sort(Comparator.comparingInt(Integer::intValue)); + + Assert.assertEquals(mutableOutput.get(0).intValue(), 0); // Check min partition id + Assert.assertEquals(mutableOutput.get(mutableOutput.size() - 1).intValue(), numPartitions - 1); // Check max + // partition id + } +} 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/hadoop/spark/input/hdfs/TestSparkInputFromHdfs.java new file mode 100644 index 00000000000..6d2e48e19d4 --- /dev/null +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/input/hdfs/TestSparkInputFromHdfs.java @@ -0,0 +1,264 @@ +package com.linkedin.venice.hadoop.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.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.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.utils.TestWriteUtils.DEFAULT_USER_DATA_VALUE_PREFIX; +import static com.linkedin.venice.utils.TestWriteUtils.STRING_TO_STRING_SCHEMA; + +import com.linkedin.venice.schema.vson.VsonAvroSchemaAdapter; +import com.linkedin.venice.schema.vson.VsonAvroSerializer; +import com.linkedin.venice.serializer.FastSerializerDeserializerFactory; +import com.linkedin.venice.serializer.RecordDeserializer; +import com.linkedin.venice.utils.TestWriteUtils; +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.DatumWriter; +import org.apache.commons.lang3.SerializationUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableCapability; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.PartitionReader; +import org.apache.spark.sql.connector.read.PartitionReaderFactory; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.connector.read.ScanBuilder; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.testng.Assert; +import org.testng.annotations.Test; + + +public class TestSparkInputFromHdfs { + private static final Schema AVRO_FILE_SCHEMA = STRING_TO_STRING_SCHEMA; + private static final String VSON_STRING_SCHEMA = "\"string\""; + + @Test + public void testHdfsInputSchema() { + VeniceHdfsSource source = new VeniceHdfsSource(); + Assert.assertEquals(source.inferSchema(null), DEFAULT_SCHEMA); + } + + @Test + public void testAvroInputSource() throws IOException { + File inputDir = TestWriteUtils.getTempDataDirectory(); + Map config = getDefaultConfigs(inputDir); + + String file1Path = "string2string1.avro"; + writeAvroFile(inputDir, file1Path, 1, 100); + String file2Path = "string2string2.avro"; + writeAvroFile(inputDir, file2Path, 101, 200); + + config.put(SCHEMA_STRING_PROP, AVRO_FILE_SCHEMA.toString()); + config.put(VSON_PUSH, String.valueOf(false)); + + CaseInsensitiveStringMap caseInsensitiveConfig = new CaseInsensitiveStringMap(config); + + VeniceHdfsSource source = new VeniceHdfsSource(); + StructType schema = source.inferSchema(caseInsensitiveConfig); + + Table table = source.getTable(schema, source.inferPartitioning(caseInsensitiveConfig), config); + Assert.assertTrue(table instanceof VeniceHdfsInputTable); + Assert.assertTrue(table.capabilities().contains(TableCapability.BATCH_READ)); + Assert.assertEquals(table.schema(), DEFAULT_SCHEMA); + // Spark docs mention that table's and source's partitioning should be the same + Assert.assertEquals(table.partitioning(), source.inferPartitioning(caseInsensitiveConfig)); + Assert.assertEquals(table.name(), INPUT_TABLE_NAME); + + VeniceHdfsInputTable hdfsTable = (VeniceHdfsInputTable) table; + ScanBuilder scanBuilder = hdfsTable.newScanBuilder(caseInsensitiveConfig); + Assert.assertTrue(scanBuilder instanceof VeniceHdfsInputScanBuilder); + + Scan scan = scanBuilder.build(); + Assert.assertTrue(scan instanceof VeniceHdfsInputScan); + Assert.assertEquals(scan.readSchema(), DEFAULT_SCHEMA); + Assert.assertSame(scan.toBatch(), scan); + + VeniceHdfsInputScan hdfsScan = (VeniceHdfsInputScan) scan; + InputPartition[] partitions = hdfsScan.planInputPartitions(); + Assert.assertEquals(partitions.length, 2); + + for (InputPartition partition: partitions) { + Assert.assertTrue(partition instanceof VeniceHdfsInputPartition); + + PartitionReaderFactory readerFactory = hdfsScan.createReaderFactory(); + Assert.assertTrue(readerFactory instanceof VeniceHdfsInputPartitionReaderFactory); + + try (PartitionReader reader = readerFactory.createReader(partition)) { + Assert.assertTrue(reader instanceof VeniceHdfsInputPartitionReader); + if (((VeniceHdfsInputPartition) partition).getFilePath().getName().equals(file1Path)) { + verifyAvroData(reader, 1, 100); + } else if (((VeniceHdfsInputPartition) partition).getFilePath().getName().equals(file2Path)) { + verifyAvroData(reader, 101, 200); + } else { + Assert.fail("Unexpected partition: " + partition); + } + } + } + } + + @Test + public void testVsonInputSource() throws IOException { + File inputDir = TestWriteUtils.getTempDataDirectory(); + Map config = getDefaultConfigs(inputDir); + + String file1Path = "string2string1_vson"; + writeVsonFile(inputDir, file1Path, 1, 100); + String file2Path = "string2string2_vson"; + writeVsonFile(inputDir, file2Path, 101, 200); + + config.put(FILE_KEY_SCHEMA, VSON_STRING_SCHEMA); + config.put(FILE_VALUE_SCHEMA, VSON_STRING_SCHEMA); + config.put(KEY_FIELD_PROP, ""); + config.put(VALUE_FIELD_PROP, ""); + config.put(VSON_PUSH, String.valueOf(true)); + + CaseInsensitiveStringMap caseInsensitiveConfig = new CaseInsensitiveStringMap(config); + + VeniceHdfsSource source = new VeniceHdfsSource(); + StructType schema = source.inferSchema(caseInsensitiveConfig); + + Table table = source.getTable(schema, source.inferPartitioning(caseInsensitiveConfig), config); + Assert.assertTrue(table instanceof VeniceHdfsInputTable); + Assert.assertTrue(table.capabilities().contains(TableCapability.BATCH_READ)); + Assert.assertEquals(table.schema(), DEFAULT_SCHEMA); + // Spark docs mention that table's and source's partitioning should be the same + Assert.assertEquals(table.partitioning(), source.inferPartitioning(caseInsensitiveConfig)); + Assert.assertEquals(table.name(), INPUT_TABLE_NAME); + + VeniceHdfsInputTable hdfsTable = (VeniceHdfsInputTable) table; + ScanBuilder scanBuilder = hdfsTable.newScanBuilder(caseInsensitiveConfig); + Assert.assertTrue(scanBuilder instanceof VeniceHdfsInputScanBuilder); + + Scan scan = scanBuilder.build(); + Assert.assertTrue(scan instanceof VeniceHdfsInputScan); + Assert.assertEquals(scan.readSchema(), DEFAULT_SCHEMA); + Assert.assertSame(scan.toBatch(), scan); + + VeniceHdfsInputScan hdfsScan = (VeniceHdfsInputScan) scan; + InputPartition[] partitions = hdfsScan.planInputPartitions(); + Assert.assertEquals(partitions.length, 2); + + for (InputPartition partition: partitions) { + Assert.assertTrue(partition instanceof VeniceHdfsInputPartition); + + PartitionReaderFactory readerFactory = hdfsScan.createReaderFactory(); + Assert.assertTrue(readerFactory instanceof VeniceHdfsInputPartitionReaderFactory); + + try (PartitionReader reader = readerFactory.createReader(partition)) { + Assert.assertTrue(reader instanceof VeniceHdfsInputPartitionReader); + if (((VeniceHdfsInputPartition) partition).getFilePath().getName().equals(file1Path)) { + verifyVsonData(reader, 1, 100); + } else if (((VeniceHdfsInputPartition) partition).getFilePath().getName().equals(file2Path)) { + verifyVsonData(reader, 101, 200); + } else { + Assert.fail("Unexpected partition: " + partition); + } + } + } + } + + @Test + public void testVeniceHdfsInputPartitionCanHandleSerDe() { + Path filePath = new Path("test"); + VeniceHdfsInputPartition partition = new VeniceHdfsInputPartition(filePath); + Assert.assertEquals(partition.getFilePath(), filePath); + VeniceHdfsInputPartition deserialized = SerializationUtils.deserialize(SerializationUtils.serialize(partition)); + // Check if deserialized object has the same filePath since it is a transient field + Assert.assertEquals(deserialized.getFilePath(), filePath); + } + + private Map getDefaultConfigs(File inputDir) { + Map config = new HashMap<>(); + config.put(INPUT_PATH_PROP, inputDir.getAbsolutePath()); + return config; + } + + private void writeAvroFile(File inputDir, String fileName, int start, int end) throws IOException { + File file = new File(inputDir, fileName); + + DatumWriter datumWriter = new GenericDatumWriter<>(AVRO_FILE_SCHEMA); + try (DataFileWriter dataFileWriter = new DataFileWriter<>(datumWriter)) { + dataFileWriter.create(AVRO_FILE_SCHEMA, file); + for (int i = start; i <= end; ++i) { + GenericRecord user = new GenericData.Record(AVRO_FILE_SCHEMA); + user.put(DEFAULT_KEY_FIELD_PROP, Integer.toString(i)); + user.put(DEFAULT_VALUE_FIELD_PROP, DEFAULT_USER_DATA_VALUE_PREFIX + i); + dataFileWriter.append(user); + } + } + } + + private void writeVsonFile(File parentDir, String fileName, int start, int end) throws IOException { + SequenceFile.Metadata metadata = new SequenceFile.Metadata(); + metadata.set(new Text("key.schema"), new Text(VSON_STRING_SCHEMA)); + metadata.set(new Text("value.schema"), new Text(VSON_STRING_SCHEMA)); + + VsonAvroSerializer stringSerializer = VsonAvroSerializer.fromSchemaStr(VSON_STRING_SCHEMA); + + try (SequenceFile.Writer writer = SequenceFile.createWriter( + new Configuration(), + SequenceFile.Writer.file(new Path(parentDir.toString(), fileName)), + SequenceFile.Writer.keyClass(BytesWritable.class), + SequenceFile.Writer.valueClass(BytesWritable.class), + SequenceFile.Writer.metadata(metadata))) { + for (int i = start; i <= end; i++) { + writer.append( + new BytesWritable(stringSerializer.toBytes(String.valueOf(i))), + new BytesWritable(stringSerializer.toBytes(DEFAULT_USER_DATA_VALUE_PREFIX + i))); + } + } + } + + private void verifyAvroData(PartitionReader reader, int start, int end) throws IOException { + Schema avroStringSchema = Schema.create(Schema.Type.STRING); + RecordDeserializer deserializer = + FastSerializerDeserializerFactory.getFastAvroGenericDeserializer(avroStringSchema, avroStringSchema); + int currentIdx = start; + while (reader.next()) { + InternalRow row = reader.get(); + Assert.assertEquals(deserializer.deserialize(row.getBinary(0)).toString(), Integer.toString(currentIdx)); + Assert.assertEquals( + deserializer.deserialize(row.getBinary(1)).toString(), + DEFAULT_USER_DATA_VALUE_PREFIX + currentIdx); + currentIdx++; + } + Assert.assertEquals(currentIdx, end + 1); + } + + private void verifyVsonData(PartitionReader reader, int start, int end) throws IOException { + Schema vsonAvroStringSchema = VsonAvroSchemaAdapter.nullableUnion(Schema.create(Schema.Type.STRING)); + RecordDeserializer deserializer = + FastSerializerDeserializerFactory.getFastAvroGenericDeserializer(vsonAvroStringSchema, vsonAvroStringSchema); + int currentIdx = start; + while (reader.next()) { + InternalRow row = reader.get(); + Assert.assertEquals(deserializer.deserialize(row.getBinary(0)).toString(), Integer.toString(currentIdx)); + Assert.assertEquals( + deserializer.deserialize(row.getBinary(1)).toString(), + DEFAULT_USER_DATA_VALUE_PREFIX + currentIdx); + currentIdx++; + } + Assert.assertEquals(currentIdx, end + 1); + } +} 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/hadoop/spark/utils/SparkScalaUtilsTest.java new file mode 100644 index 00000000000..7081393d265 --- /dev/null +++ b/clients/venice-push-job/src/test/java/com/linkedin/venice/hadoop/spark/utils/SparkScalaUtilsTest.java @@ -0,0 +1,25 @@ +package com.linkedin.venice.hadoop.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 org.apache.spark.sql.types.DataTypes.BinaryType; + +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.testng.Assert; +import org.testng.annotations.Test; + + +public class SparkScalaUtilsTest { + @Test + public void testGetFieldIndex() { + final StructType dataSchema = new StructType( + new StructField[] { new StructField(KEY_COLUMN_NAME, BinaryType, false, Metadata.empty()), + new StructField(VALUE_COLUMN_NAME, BinaryType, true, Metadata.empty()) }); + + Assert.assertEquals(SparkScalaUtils.getFieldIndex(dataSchema, KEY_COLUMN_NAME), 0); + Assert.assertEquals(SparkScalaUtils.getFieldIndex(dataSchema, VALUE_COLUMN_NAME), 1); + Assert.assertEquals(SparkScalaUtils.getFieldIndex(dataSchema, "DUMMY_FILED"), -1); + } +} diff --git a/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/ArrayUtils.java b/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/ArrayUtils.java new file mode 100644 index 00000000000..bdb8b216e1f --- /dev/null +++ b/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/ArrayUtils.java @@ -0,0 +1,31 @@ +package com.linkedin.venice.utils; + +public final class ArrayUtils { + private ArrayUtils() { + } + + /** + * This is mostly a copy of Arrays.compareUnsigned(T[], T[]) from Java 9. + */ + public static int compareUnsigned(byte[] a, byte[] b) { + if (a == b) { + return 0; + } + + // A null array is less than a non-null array + if (a == null || b == null) { + return a == null ? -1 : 1; + } + + int length = Math.min(a.length, b.length); + for (int i = 0; i < length; i++) { + byte oa = a[i]; + byte ob = b[i]; + if (oa != ob) { + return Byte.toUnsignedInt(oa) - Byte.toUnsignedInt(ob); + } + } + + return a.length - b.length; + } +} diff --git a/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/IteratorUtils.java b/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/IteratorUtils.java new file mode 100644 index 00000000000..01ea38ab134 --- /dev/null +++ b/internal/venice-client-common/src/main/java/com/linkedin/venice/utils/IteratorUtils.java @@ -0,0 +1,32 @@ +package com.linkedin.venice.utils; + +import java.util.Iterator; +import java.util.function.Function; + + +public final class IteratorUtils { + private IteratorUtils() { + } + + public static Iterator mapIterator(Iterator iterator, Function mapper) { + if (iterator == null) { + return null; + } + + if (mapper == null) { + throw new NullPointerException("Expected 'mapper' to not be null"); + } + + return new Iterator() { + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public O next() { + return mapper.apply(iterator.next()); + } + }; + } +} 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 a0a87c46ebc..dfafe3ef155 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 @@ -7,6 +7,7 @@ import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStream; +import java.io.Serializable; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -19,7 +20,8 @@ import java.util.function.Supplier; -public class VeniceProperties { +public class VeniceProperties implements Serializable { + private static final long serialVersionUID = 1L; private static final VeniceProperties EMPTY = new VeniceProperties(); private final Map props; diff --git a/internal/venice-client-common/src/test/java/com/linkedin/venice/utils/ArrayUtilsTest.java b/internal/venice-client-common/src/test/java/com/linkedin/venice/utils/ArrayUtilsTest.java new file mode 100644 index 00000000000..b8c0c3a9b97 --- /dev/null +++ b/internal/venice-client-common/src/test/java/com/linkedin/venice/utils/ArrayUtilsTest.java @@ -0,0 +1,60 @@ +package com.linkedin.venice.utils; + +import org.testng.Assert; +import org.testng.annotations.Test; + + +public class ArrayUtilsTest { + @Test + public void testCompareUnsignedSameArrays() { + byte[] a = new byte[] { 1, 2, 3 }; + Assert.assertEquals(ArrayUtils.compareUnsigned(a, a), 0); + } + + @Test + public void testCompareUnsignedIdenticalArrays() { + byte[] a = new byte[] { 1, 2, 3 }; + byte[] b = new byte[] { 1, 2, 3 }; + Assert.assertEquals(ArrayUtils.compareUnsigned(a, b), 0); + } + + @Test + public void testCompareUnsignedFirstLarger() { + byte[] a = new byte[] { 1, 2, 3 }; + byte[] b = new byte[] { 1, 2 }; + Assert.assertEquals(ArrayUtils.compareUnsigned(a, b), 1); + + byte[] c = new byte[] { 1, 4 }; + byte[] d = new byte[] { 1, 2, 3 }; + Assert.assertEquals(ArrayUtils.compareUnsigned(c, d), 2); + } + + @Test + public void testCompareUnsignedHonorsUnsigned() { + byte[] a = new byte[] { 1, -100 }; // For unsigned comparison, this is 156. For signed comparison, this is -100. + byte[] b = new byte[] { 1, 2 }; + Assert.assertEquals(ArrayUtils.compareUnsigned(a, b), 154); + } + + @Test + public void testCompareUnsignedSecondLarger() { + byte[] a = new byte[] { 1, 2 }; + byte[] b = new byte[] { 1, 2, 3 }; + Assert.assertEquals(ArrayUtils.compareUnsigned(a, b), -1); + + byte[] c = new byte[] { 1, 2, 3 }; + byte[] d = new byte[] { 1, 4 }; + Assert.assertEquals(ArrayUtils.compareUnsigned(c, d), -2); + } + + @Test + public void testCompareUnsignedNullArrays() { + byte[] a = new byte[] { 1, 2, 3 }; + + Assert.assertEquals(ArrayUtils.compareUnsigned(a, null), 1); + + Assert.assertEquals(ArrayUtils.compareUnsigned(null, a), -1); + + Assert.assertEquals(ArrayUtils.compareUnsigned(null, null), 0); + } +} diff --git a/internal/venice-client-common/src/test/java/com/linkedin/venice/utils/IteratorUtilsTest.java b/internal/venice-client-common/src/test/java/com/linkedin/venice/utils/IteratorUtilsTest.java new file mode 100644 index 00000000000..3b097d0b05e --- /dev/null +++ b/internal/venice-client-common/src/test/java/com/linkedin/venice/utils/IteratorUtilsTest.java @@ -0,0 +1,44 @@ +package com.linkedin.venice.utils; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import org.testng.Assert; +import org.testng.annotations.Test; + + +public class IteratorUtilsTest { + @Test + public void testMapIterator() { + Iterator iterator = Arrays.asList(1, 2, 3, 4, 5).iterator(); + Iterator mappedIterator = IteratorUtils.mapIterator(iterator, Object::toString); + + Arrays.asList("1", "2", "3", "4", "5").forEach(expected -> { + Assert.assertTrue(mappedIterator.hasNext()); + Assert.assertEquals(mappedIterator.next(), expected); + }); + + Assert.assertFalse(mappedIterator.hasNext()); + } + + @Test + public void testMapEmptyIterator() { + List list = Collections.emptyList(); + Iterator iterator = list.iterator(); + Iterator mappedIterator = IteratorUtils.mapIterator(iterator, Object::toString); + + Assert.assertFalse(mappedIterator.hasNext()); + } + + @Test + public void testMapIteratorWithNullMapper() { + Iterator iterator = Arrays.asList(1, 2, 3, 4, 5).iterator(); + Assert.assertThrows(NullPointerException.class, () -> IteratorUtils.mapIterator(iterator, null)); + } + + @Test + public void testMapIteratorWithNullIterator() { + Assert.assertNull(IteratorUtils.mapIterator(null, Object::toString)); + } +} 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 a962ba3d34c..dc5882a5d3a 100644 --- a/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/PushJobDetailsTest.java +++ b/internal/venice-test-common/src/integrationTest/java/com/linkedin/venice/endToEnd/PushJobDetailsTest.java @@ -3,7 +3,7 @@ import static com.linkedin.davinci.store.rocksdb.RocksDBServerConfig.ROCKSDB_PLAIN_TABLE_FORMAT_ENABLED; import static com.linkedin.venice.ConfigKeys.SERVER_DATABASE_CHECKSUM_VERIFICATION_ENABLED; import static com.linkedin.venice.ConfigKeys.SERVER_DATABASE_SYNC_BYTES_INTERNAL_FOR_DEFERRED_WRITE_MODE; -import static com.linkedin.venice.hadoop.VenicePushJob.PushJobCheckpoints.START_MAP_REDUCE_JOB; +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; @@ -256,7 +256,7 @@ public void testPushJobDetailsFailureTags() throws ExecutionException, Interrupt PushJobDetails value = client.get(key).get(); assertEquals( value.pushJobLatestCheckpoint.intValue(), - START_MAP_REDUCE_JOB.getValue(), + START_DATA_WRITER_JOB.getValue(), "Unexpected latest push job checkpoint reported"); assertFalse(value.failureDetails.toString().isEmpty()); } 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 37fac4a880f..d1211ecf76a 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,6 +3,7 @@ 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.EXTENDED_SCHEMA_VALIDITY_CHECK_ENABLED; @@ -53,6 +54,7 @@ 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; @@ -815,6 +817,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()); extraProps.accept(props); if (StringUtils.isEmpty(existingStore)) { 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 835810a843c..932e5ee8d5b 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,5 +1,6 @@ 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; @@ -22,6 +23,7 @@ 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; @@ -319,6 +321,7 @@ private String testBatchStore( try { String inputDirPath = "file://" + inputDir.getAbsolutePath(); Properties props = defaultVPJPropsWithoutD2Routing(veniceCluster, inputDirPath, storeName); + props.setProperty(DATA_WRITER_COMPUTE_JOB_CLASS, DataWriterSparkJob.class.getCanonicalName()); extraProps.accept(props); if (!storeNameOptional.isPresent()) {