-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-1351] Improvements to the hudi test suite for scalability and repeated testing. #2197
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -28,9 +28,17 @@ | |
| import java.util.Iterator; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.UUID; | ||
| import java.util.stream.Collectors; | ||
| import java.util.stream.IntStream; | ||
| import java.util.stream.StreamSupport; | ||
| import org.apache.avro.generic.GenericRecord; | ||
| import org.apache.hadoop.fs.FileSystem; | ||
| import org.apache.hadoop.fs.Path; | ||
| import org.apache.hudi.common.fs.FSUtils; | ||
| import org.apache.hudi.common.fs.HoodieWrapperFileSystem; | ||
| import org.apache.hudi.common.util.Option; | ||
| import org.apache.hudi.common.util.StringUtils; | ||
| import org.apache.hudi.integ.testsuite.converter.UpdateConverter; | ||
| import org.apache.hudi.integ.testsuite.reader.DFSAvroDeltaInputReader; | ||
| import org.apache.hudi.integ.testsuite.reader.DFSHoodieDatasetInputReader; | ||
|
|
@@ -41,7 +49,6 @@ | |
| import org.apache.hudi.integ.testsuite.writer.DeltaWriterFactory; | ||
| import org.apache.hudi.keygen.BuiltinKeyGenerator; | ||
| import org.apache.hudi.integ.testsuite.configuration.DFSDeltaConfig; | ||
| import org.apache.hudi.integ.testsuite.configuration.DeltaConfig; | ||
| import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config; | ||
| import org.apache.spark.api.java.JavaRDD; | ||
| import org.apache.spark.api.java.JavaSparkContext; | ||
|
|
@@ -58,15 +65,15 @@ public class DeltaGenerator implements Serializable { | |
|
|
||
| private static Logger log = LoggerFactory.getLogger(DeltaGenerator.class); | ||
|
|
||
| private DeltaConfig deltaOutputConfig; | ||
| private DFSDeltaConfig deltaOutputConfig; | ||
| private transient JavaSparkContext jsc; | ||
| private transient SparkSession sparkSession; | ||
| private String schemaStr; | ||
| private List<String> recordRowKeyFieldNames; | ||
| private List<String> partitionPathFieldNames; | ||
| private int batchId; | ||
|
|
||
| public DeltaGenerator(DeltaConfig deltaOutputConfig, JavaSparkContext jsc, SparkSession sparkSession, | ||
| public DeltaGenerator(DFSDeltaConfig deltaOutputConfig, JavaSparkContext jsc, SparkSession sparkSession, | ||
| String schemaStr, BuiltinKeyGenerator keyGenerator) { | ||
| this.deltaOutputConfig = deltaOutputConfig; | ||
| this.jsc = jsc; | ||
|
|
@@ -77,6 +84,16 @@ public DeltaGenerator(DeltaConfig deltaOutputConfig, JavaSparkContext jsc, Spark | |
| } | ||
|
|
||
| public JavaRDD<DeltaWriteStats> writeRecords(JavaRDD<GenericRecord> records) { | ||
| if (deltaOutputConfig.shouldDeleteOldInputData() && batchId > 1) { | ||
| Path oldInputDir = new Path(deltaOutputConfig.getDeltaBasePath(), Integer.toString(batchId - 1)); | ||
|
||
| try { | ||
| FileSystem fs = FSUtils.getFs(oldInputDir.toString(), deltaOutputConfig.getConfiguration()); | ||
| fs.delete(oldInputDir, true); | ||
| } catch (IOException e) { | ||
| log.error("Failed to delete older input data direcory " + oldInputDir, e); | ||
| } | ||
| } | ||
|
|
||
| // The following creates a new anonymous function for iterator and hence results in serialization issues | ||
| JavaRDD<DeltaWriteStats> ws = records.mapPartitions(itr -> { | ||
| try { | ||
|
|
@@ -95,11 +112,22 @@ public JavaRDD<GenericRecord> generateInserts(Config operation) { | |
| int numPartitions = operation.getNumInsertPartitions(); | ||
| long recordsPerPartition = operation.getNumRecordsInsert() / numPartitions; | ||
| int minPayloadSize = operation.getRecordSize(); | ||
| JavaRDD<GenericRecord> inputBatch = jsc.parallelize(Collections.EMPTY_LIST) | ||
| .repartition(numPartitions).mapPartitions(p -> { | ||
| int startPartition = operation.getStartPartition(); | ||
|
||
|
|
||
| // Each spark partition below will generate records for a single partition given by the integer index. | ||
| List<Integer> partitionIndexes = IntStream.rangeClosed(0 + startPartition, numPartitions + startPartition) | ||
| .boxed().collect(Collectors.toList()); | ||
|
|
||
| JavaRDD<GenericRecord> inputBatch = jsc.parallelize(partitionIndexes, numPartitions) | ||
| .mapPartitionsWithIndex((index, p) -> { | ||
| return new LazyRecordGeneratorIterator(new FlexibleSchemaRecordGenerationIterator(recordsPerPartition, | ||
| minPayloadSize, schemaStr, partitionPathFieldNames, numPartitions)); | ||
| }); | ||
| minPayloadSize, schemaStr, partitionPathFieldNames, (Integer)index)); | ||
| }, true); | ||
|
|
||
| if (deltaOutputConfig.getInputParallelism() < numPartitions) { | ||
| inputBatch = inputBatch.coalesce(deltaOutputConfig.getInputParallelism()); | ||
| } | ||
|
|
||
| return inputBatch; | ||
| } | ||
|
|
||
|
|
@@ -131,9 +159,11 @@ public JavaRDD<GenericRecord> generateUpdates(Config config) throws IOException | |
| } | ||
| } | ||
|
|
||
| log.info("Repartitioning records"); | ||
| // persist this since we will make multiple passes over this | ||
| adjustedRDD = adjustedRDD.repartition(jsc.defaultParallelism()); | ||
| int numPartition = Math.min(deltaOutputConfig.getInputParallelism(), | ||
| Math.max(1, config.getNumUpsertPartitions())); | ||
| log.info("Repartitioning records into " + numPartition + " partitions"); | ||
| adjustedRDD = adjustedRDD.repartition(numPartition); | ||
| log.info("Repartitioning records done"); | ||
| UpdateConverter converter = new UpdateConverter(schemaStr, config.getRecordSize(), | ||
| partitionPathFieldNames, recordRowKeyFieldNames); | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
What is the purpose behind this change ?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
DFSDeltaConfig extends DeltaConfig
The two settings I have added (getInputParallelism and shouldDeleteOldInputData) are in DFSDeltaConfig.