nsivabalan commented on code in PR #9852:
URL: https://github.com/apache/hudi/pull/9852#discussion_r1365976614
##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java:
##########
@@ -316,13 +314,15 @@ public StreamSync(HoodieStreamer.Config cfg, SparkSession
sparkSession, SchemaPr
this.errorTableWriter = ErrorTableUtils.getErrorTableWriter(cfg,
sparkSession, props, hoodieSparkContext, fs);
this.errorWriteFailureStrategy =
ErrorTableUtils.getErrorWriteFailureStrategy(props);
}
- this.formatAdapter = new SourceFormatAdapter(
- UtilHelpers.createSource(cfg.sourceClassName, props,
hoodieSparkContext.jsc(), sparkSession, schemaProvider, metrics),
- this.errorTableWriter, Option.of(props));
+ Source source = UtilHelpers.createSource(cfg.sourceClassName, props,
hoodieSparkContext.jsc(), sparkSession, schemaProvider, metrics);
+ this.formatAdapter = new SourceFormatAdapter(source,
this.errorTableWriter, Option.of(props));
this.transformer =
UtilHelpers.createTransformer(Option.ofNullable(cfg.transformerClassNames),
Option.ofNullable(schemaProvider).map(SchemaProvider::getSourceSchema),
this.errorTableWriter.isPresent());
+ if (this.cfg.operation == WriteOperationType.BULK_INSERT &&
source.getSourceType() != Source.SourceType.AVRO) {
+ this.props.setProperty(RECORD_MERGER_IMPLS.key(),
HoodieSparkRecordMerger.class.getName());
Review Comment:
if we found any bugs w/ spark record flow, is there a way to fallback.
##########
hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java:
##########
@@ -2757,6 +2757,79 @@ private Set<String> getAllFileIDsInTable(String
tableBasePath, Option<String> pa
return
baseFileStream.map(HoodieBaseFile::getFileId).collect(Collectors.toSet());
}
+ private void testBulkInsertOptRowWriterParquetDFSSource(boolean
useSchemaProvider, List<String> transformerClassNames, boolean testEmptyBatch)
throws Exception {
+ PARQUET_SOURCE_ROOT = basePath + "/parquetFilesDfs" + testNum;
+ int parquetRecordsCount = 10;
+ boolean hasTransformer = transformerClassNames != null &&
!transformerClassNames.isEmpty();
+ prepareParquetDFSFiles(parquetRecordsCount, PARQUET_SOURCE_ROOT,
FIRST_PARQUET_FILE_NAME, false, null, null);
+ prepareParquetDFSSource(useSchemaProvider, hasTransformer, "source.avsc",
"target.avsc", PROPS_FILENAME_TEST_PARQUET,
+ PARQUET_SOURCE_ROOT, false, "partition_path", testEmptyBatch ? "1" :
"");
+
+ String tableBasePath = basePath + "/test_parquet_table" + testNum;
+ HoodieDeltaStreamer deltaStreamer = new HoodieDeltaStreamer(
+ TestHelpers.makeConfig(tableBasePath, WriteOperationType.BULK_INSERT,
testEmptyBatch ? TestParquetDFSSourceEmptyBatch.class.getName()
+ : ParquetDFSSource.class.getName(),
+ transformerClassNames, PROPS_FILENAME_TEST_PARQUET, false,
+ useSchemaProvider, 100000, false, null, null, "timestamp", null),
jsc);
+ deltaStreamer.sync();
+ TestHelpers.assertRecordCount(parquetRecordsCount, tableBasePath,
sqlContext);
+
+ if (testEmptyBatch) {
+ prepareParquetDFSFiles(100, PARQUET_SOURCE_ROOT, "2.parquet", false,
null, null);
+ deltaStreamer.sync();
+ // since we mimic'ed empty batch, total records should be same as first
sync().
+ TestHelpers.assertRecordCount(parquetRecordsCount, tableBasePath,
sqlContext);
+ HoodieTableMetaClient metaClient =
HoodieTableMetaClient.builder().setBasePath(tableBasePath).setConf(jsc.hadoopConfiguration()).build();
+
+ // validate table schema fetches valid schema from last but one commit.
+ TableSchemaResolver tableSchemaResolver = new
TableSchemaResolver(metaClient);
+ assertNotEquals(tableSchemaResolver.getTableAvroSchema(),
Schema.create(Schema.Type.NULL).toString());
+ }
+
+ // proceed w/ non empty batch.
+ prepareParquetDFSFiles(100, PARQUET_SOURCE_ROOT, "3.parquet", false, null,
null);
+ deltaStreamer.sync();
+ TestHelpers.assertRecordCount(parquetRecordsCount + 100, tableBasePath,
sqlContext);
+ // validate commit metadata for all completed commits to have valid schema
in extra metadata.
+ HoodieTableMetaClient metaClient =
HoodieTableMetaClient.builder().setBasePath(tableBasePath).setConf(jsc.hadoopConfiguration()).build();
+
metaClient.reloadActiveTimeline().getCommitsTimeline().filterCompletedInstants().getInstants().forEach(entry
-> assertValidSchemaInCommitMetadata(entry, metaClient));
+ testNum++;
+ }
+
+ private void testBulkInsertOptRowWriterParquetDFSSource(boolean
useSchemaProvider, List<String> transformerClassNames) throws Exception {
+ testBulkInsertOptRowWriterParquetDFSSource(useSchemaProvider,
transformerClassNames, false);
+ }
+
+ @Test
+ public void testBulkInsertOptBasic() throws Exception {
Review Comment:
similarly, for other operations, its good to validate that we are going via
avro record and not spark record
##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java:
##########
@@ -597,16 +578,56 @@ private Pair<SchemaProvider, Pair<String,
JavaRDD<HoodieRecord>>> fetchFromSourc
(SchemaProvider) new DelegatingSchemaProvider(props,
hoodieSparkContext.jsc(), dataAndCheckpoint.getSchemaProvider(),
new SimpleSchemaProvider(hoodieSparkContext.jsc(),
targetSchema, props)))
.orElse(dataAndCheckpoint.getSchemaProvider());
- // Rewrite transformed records into the expected target schema
- avroRDDOptional = transformed.map(t -> getTransformedRDD(t,
reconcileSchema, schemaProvider.getTargetSchema()));
+ }
+
+ // Short circuit for bulk insert to gain performance benefits of row
writing
+ if (recordType == HoodieRecordType.SPARK) {
+ rowDatasetOptional = transformed;
+ noNewData = (!rowDatasetOptional.isPresent()) ||
(rowDatasetOptional.get().isEmpty());
+ } else {
+ // If the target schema is specified through Avro schema,
+ // pass in the schema for the Row-to-Avro conversion
+ // to avoid nullability mismatch between Avro schema and Row schema
+ if (errorTableWriter.isPresent()
Review Comment:
I guess we can't skip this for SPARK record. if error table is enabled and
if target schema is enforced, we might have to make this call for all record
type.
##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java:
##########
@@ -618,65 +639,64 @@ private Pair<SchemaProvider, Pair<String,
JavaRDD<HoodieRecord>>> fetchFromSourc
}
hoodieSparkContext.setJobStatus(this.getClass().getSimpleName(), "Checking
if input is empty");
- if ((!avroRDDOptional.isPresent()) || (avroRDDOptional.get().isEmpty())) {
+ if (noNewData) {
LOG.info("No new data, perform empty commit.");
return Pair.of(schemaProvider, Pair.of(checkpointStr,
hoodieSparkContext.emptyRDD()));
}
boolean shouldCombine = cfg.filterDupes ||
cfg.operation.equals(WriteOperationType.UPSERT);
Set<String> partitionColumns = getPartitionColumns(props);
- JavaRDD<GenericRecord> avroRDD = avroRDDOptional.get();
-
- JavaRDD<HoodieRecord> records;
+ JavaRDD<HoodieRecord> records = null;
SerializableSchema avroSchema = new
SerializableSchema(schemaProvider.getTargetSchema());
SerializableSchema processedAvroSchema = new
SerializableSchema(isDropPartitionColumns() ?
HoodieAvroUtils.removeMetadataFields(avroSchema.get()) : avroSchema.get());
- if (recordType == HoodieRecordType.AVRO) {
- records = avroRDD.mapPartitions(
- (FlatMapFunction<Iterator<GenericRecord>, HoodieRecord>)
genericRecordIterator -> {
- if (autoGenerateRecordKeys) {
-
props.setProperty(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG,
String.valueOf(TaskContext.getPartitionId()));
-
props.setProperty(KeyGenUtils.RECORD_KEY_GEN_INSTANT_TIME_CONFIG, instantTime);
- }
- BuiltinKeyGenerator builtinKeyGenerator = (BuiltinKeyGenerator)
HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
- List<HoodieRecord> avroRecords = new ArrayList<>();
- while (genericRecordIterator.hasNext()) {
- GenericRecord genRec = genericRecordIterator.next();
- HoodieKey hoodieKey = new
HoodieKey(builtinKeyGenerator.getRecordKey(genRec),
builtinKeyGenerator.getPartitionPath(genRec));
- GenericRecord gr = isDropPartitionColumns() ?
HoodieAvroUtils.removeFields(genRec, partitionColumns) : genRec;
- HoodieRecordPayload payload = shouldCombine ?
DataSourceUtils.createPayload(cfg.payloadClassName, gr,
- (Comparable) HoodieAvroUtils.getNestedFieldVal(gr,
cfg.sourceOrderingField, false, props.getBoolean(
-
KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
-
Boolean.parseBoolean(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()))))
- : DataSourceUtils.createPayload(cfg.payloadClassName, gr);
- avroRecords.add(new HoodieAvroRecord<>(hoodieKey, payload));
- }
- return avroRecords.iterator();
- });
- } else if (recordType == HoodieRecordType.SPARK) {
- // TODO we should remove it if we can read InternalRow from source.
- records = avroRDD.mapPartitions(itr -> {
- if (autoGenerateRecordKeys) {
- props.setProperty(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG,
String.valueOf(TaskContext.getPartitionId()));
- props.setProperty(KeyGenUtils.RECORD_KEY_GEN_INSTANT_TIME_CONFIG,
instantTime);
- }
- BuiltinKeyGenerator builtinKeyGenerator = (BuiltinKeyGenerator)
HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
+ if (autoGenerateRecordKeys) {
+ props.setProperty(KeyGenUtils.RECORD_KEY_GEN_PARTITION_ID_CONFIG,
String.valueOf(TaskContext.getPartitionId()));
Review Comment:
these two lines need to be called from within executor. Guess here we are
calling it from driver.
##########
hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java:
##########
@@ -2757,6 +2757,79 @@ private Set<String> getAllFileIDsInTable(String
tableBasePath, Option<String> pa
return
baseFileStream.map(HoodieBaseFile::getFileId).collect(Collectors.toSet());
}
+ private void testBulkInsertOptRowWriterParquetDFSSource(boolean
useSchemaProvider, List<String> transformerClassNames, boolean testEmptyBatch)
throws Exception {
+ PARQUET_SOURCE_ROOT = basePath + "/parquetFilesDfs" + testNum;
+ int parquetRecordsCount = 10;
+ boolean hasTransformer = transformerClassNames != null &&
!transformerClassNames.isEmpty();
+ prepareParquetDFSFiles(parquetRecordsCount, PARQUET_SOURCE_ROOT,
FIRST_PARQUET_FILE_NAME, false, null, null);
+ prepareParquetDFSSource(useSchemaProvider, hasTransformer, "source.avsc",
"target.avsc", PROPS_FILENAME_TEST_PARQUET,
+ PARQUET_SOURCE_ROOT, false, "partition_path", testEmptyBatch ? "1" :
"");
+
+ String tableBasePath = basePath + "/test_parquet_table" + testNum;
+ HoodieDeltaStreamer deltaStreamer = new HoodieDeltaStreamer(
+ TestHelpers.makeConfig(tableBasePath, WriteOperationType.BULK_INSERT,
testEmptyBatch ? TestParquetDFSSourceEmptyBatch.class.getName()
+ : ParquetDFSSource.class.getName(),
+ transformerClassNames, PROPS_FILENAME_TEST_PARQUET, false,
+ useSchemaProvider, 100000, false, null, null, "timestamp", null),
jsc);
+ deltaStreamer.sync();
+ TestHelpers.assertRecordCount(parquetRecordsCount, tableBasePath,
sqlContext);
+
+ if (testEmptyBatch) {
+ prepareParquetDFSFiles(100, PARQUET_SOURCE_ROOT, "2.parquet", false,
null, null);
+ deltaStreamer.sync();
+ // since we mimic'ed empty batch, total records should be same as first
sync().
+ TestHelpers.assertRecordCount(parquetRecordsCount, tableBasePath,
sqlContext);
+ HoodieTableMetaClient metaClient =
HoodieTableMetaClient.builder().setBasePath(tableBasePath).setConf(jsc.hadoopConfiguration()).build();
+
+ // validate table schema fetches valid schema from last but one commit.
+ TableSchemaResolver tableSchemaResolver = new
TableSchemaResolver(metaClient);
+ assertNotEquals(tableSchemaResolver.getTableAvroSchema(),
Schema.create(Schema.Type.NULL).toString());
+ }
+
+ // proceed w/ non empty batch.
+ prepareParquetDFSFiles(100, PARQUET_SOURCE_ROOT, "3.parquet", false, null,
null);
+ deltaStreamer.sync();
+ TestHelpers.assertRecordCount(parquetRecordsCount + 100, tableBasePath,
sqlContext);
+ // validate commit metadata for all completed commits to have valid schema
in extra metadata.
+ HoodieTableMetaClient metaClient =
HoodieTableMetaClient.builder().setBasePath(tableBasePath).setConf(jsc.hadoopConfiguration()).build();
+
metaClient.reloadActiveTimeline().getCommitsTimeline().filterCompletedInstants().getInstants().forEach(entry
-> assertValidSchemaInCommitMetadata(entry, metaClient));
+ testNum++;
+ }
+
+ private void testBulkInsertOptRowWriterParquetDFSSource(boolean
useSchemaProvider, List<String> transformerClassNames) throws Exception {
+ testBulkInsertOptRowWriterParquetDFSSource(useSchemaProvider,
transformerClassNames, false);
+ }
+
+ @Test
+ public void testBulkInsertOptBasic() throws Exception {
Review Comment:
is it possible to verify that we went through Spark Record for bulk_insert
and not AVRO record.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]