1
0

[HUDI-3213] Making commit preserve metadata to true for compaction (#4811)

* Making commit preserve metadata to true

* Fixing integ tests

* Fixing preserve commit metadata for metadata table

* fixed bootstrap tests

* temp diff

* Fixing merge handle

* renaming fallback record

* fixing build issue

* Fixing test failures
This commit is contained in:
Sivabalan Narayanan
2022-03-07 07:32:05 -05:00
committed by GitHub
parent 6f57bbfac4
commit 3539578ccb
15 changed files with 88 additions and 48 deletions

View File

@@ -395,7 +395,8 @@ object HoodieSparkSqlWriter {
val partitionColumns = HoodieWriterUtils.getPartitionColumns(parameters)
val recordKeyFields = hoodieConfig.getString(DataSourceWriteOptions.RECORDKEY_FIELD)
val keyGenProp = hoodieConfig.getString(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME)
val populateMetaFields = parameters.getOrElse(HoodieTableConfig.POPULATE_META_FIELDS.key(), HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()).toBoolean
val populateMetaFields = java.lang.Boolean.parseBoolean((parameters.getOrElse(HoodieTableConfig.POPULATE_META_FIELDS.key(),
String.valueOf(HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()))))
val baseFileFormat = hoodieConfig.getStringOrDefault(HoodieTableConfig.BASE_FILE_FORMAT)
HoodieTableMetaClient.withPropertyBuilder()
@@ -447,8 +448,8 @@ object HoodieSparkSqlWriter {
instantTime: String,
partitionColumns: String): (Boolean, common.util.Option[String]) = {
val sparkContext = sqlContext.sparkContext
val populateMetaFields = parameters.getOrElse(HoodieTableConfig.POPULATE_META_FIELDS.key(),
HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()).toBoolean
val populateMetaFields = java.lang.Boolean.parseBoolean((parameters.getOrElse(HoodieTableConfig.POPULATE_META_FIELDS.key(),
String.valueOf(HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()))))
val dropPartitionColumns =
parameters.getOrElse(DataSourceWriteOptions.DROP_PARTITION_COLUMNS.key(), DataSourceWriteOptions.DROP_PARTITION_COLUMNS.defaultValue()).toBoolean
// register classes & schemas

View File

@@ -364,6 +364,7 @@ public class HoodieJavaStreamingApp {
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key(), "1")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE().key(), "true")
.option(DataSourceWriteOptions.ASYNC_CLUSTERING_ENABLE().key(), "true")
.option(HoodieCompactionConfig.PRESERVE_COMMIT_METADATA.key(), "false")
.option(HoodieWriteConfig.TBL_NAME.key(), tableName).option("checkpointLocation", checkpointLocation)
.outputMode(OutputMode.Append());

View File

@@ -256,7 +256,7 @@ public class TestBootstrap extends HoodieClientTestBase {
SparkRDDWriteClient client = new SparkRDDWriteClient(context, config);
client.bootstrap(Option.empty());
checkBootstrapResults(totalRecords, schema, bootstrapCommitInstantTs, checkNumRawFiles, numInstantsAfterBootstrap,
numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants);
numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants, true);
// Rollback Bootstrap
HoodieActiveTimeline.deleteInstantFile(metaClient.getFs(), metaClient.getMetaPath(), new HoodieInstant(State.COMPLETED,
@@ -284,7 +284,7 @@ public class TestBootstrap extends HoodieClientTestBase {
}
checkBootstrapResults(totalRecords, schema, bootstrapCommitInstantTs, checkNumRawFiles, numInstantsAfterBootstrap,
numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants);
numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants, true);
// Upsert case
long updateTimestamp = Instant.now().toEpochMilli();
@@ -296,7 +296,7 @@ public class TestBootstrap extends HoodieClientTestBase {
String newInstantTs = client.startCommit();
client.upsert(updateBatch, newInstantTs);
checkBootstrapResults(totalRecords, schema, newInstantTs, false, numInstantsAfterBootstrap + 1,
updateTimestamp, deltaCommit ? timestamp : updateTimestamp, deltaCommit);
updateTimestamp, deltaCommit ? timestamp : updateTimestamp, deltaCommit, true);
if (deltaCommit) {
Option<String> compactionInstant = client.scheduleCompaction(Option.empty());
@@ -304,7 +304,7 @@ public class TestBootstrap extends HoodieClientTestBase {
client.compact(compactionInstant.get());
checkBootstrapResults(totalRecords, schema, compactionInstant.get(), checkNumRawFiles,
numInstantsAfterBootstrap + 2, 2, updateTimestamp, updateTimestamp, !deltaCommit,
Arrays.asList(compactionInstant.get()));
Arrays.asList(compactionInstant.get()), !config.isPreserveHoodieCommitMetadataForCompaction());
}
}
@@ -334,14 +334,14 @@ public class TestBootstrap extends HoodieClientTestBase {
}
private void checkBootstrapResults(int totalRecords, Schema schema, String maxInstant, boolean checkNumRawFiles,
int expNumInstants, long expTimestamp, long expROTimestamp, boolean isDeltaCommit) throws Exception {
int expNumInstants, long expTimestamp, long expROTimestamp, boolean isDeltaCommit, boolean validateRecordsForCommitTime) throws Exception {
checkBootstrapResults(totalRecords, schema, maxInstant, checkNumRawFiles, expNumInstants, expNumInstants,
expTimestamp, expROTimestamp, isDeltaCommit, Arrays.asList(maxInstant));
expTimestamp, expROTimestamp, isDeltaCommit, Arrays.asList(maxInstant), validateRecordsForCommitTime);
}
private void checkBootstrapResults(int totalRecords, Schema schema, String instant, boolean checkNumRawFiles,
int expNumInstants, int numVersions, long expTimestamp, long expROTimestamp, boolean isDeltaCommit,
List<String> instantsWithValidRecords) throws Exception {
List<String> instantsWithValidRecords, boolean validateRecordsForCommitTime) throws Exception {
metaClient.reloadActiveTimeline();
assertEquals(expNumInstants, metaClient.getCommitsTimeline().filterCompletedInstants().countInstants());
assertEquals(instant, metaClient.getActiveTimeline()
@@ -361,8 +361,10 @@ public class TestBootstrap extends HoodieClientTestBase {
if (!isDeltaCommit) {
String predicate = String.join(", ",
instantsWithValidRecords.stream().map(p -> "\"" + p + "\"").collect(Collectors.toList()));
assertEquals(totalRecords, sqlContext.sql("select * from bootstrapped where _hoodie_commit_time IN "
+ "(" + predicate + ")").count());
if (validateRecordsForCommitTime) {
assertEquals(totalRecords, sqlContext.sql("select * from bootstrapped where _hoodie_commit_time IN "
+ "(" + predicate + ")").count());
}
Dataset<Row> missingOriginal = sqlContext.sql("select a._row_key from original a where a._row_key not "
+ "in (select _hoodie_record_key from bootstrapped)");
assertEquals(0, missingOriginal.count());

View File

@@ -248,7 +248,7 @@ public class TestOrcBootstrap extends HoodieClientTestBase {
SparkRDDWriteClient client = new SparkRDDWriteClient(context, config);
client.bootstrap(Option.empty());
checkBootstrapResults(totalRecords, schema, bootstrapCommitInstantTs, checkNumRawFiles, numInstantsAfterBootstrap,
numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants);
numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants, true);
// Rollback Bootstrap
if (deltaCommit) {
@@ -278,7 +278,7 @@ public class TestOrcBootstrap extends HoodieClientTestBase {
}
checkBootstrapResults(totalRecords, schema, bootstrapCommitInstantTs, checkNumRawFiles, numInstantsAfterBootstrap,
numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants);
numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants, true);
// Upsert case
long updateTimestamp = Instant.now().toEpochMilli();
@@ -290,7 +290,7 @@ public class TestOrcBootstrap extends HoodieClientTestBase {
String newInstantTs = client.startCommit();
client.upsert(updateBatch, newInstantTs);
checkBootstrapResults(totalRecords, schema, newInstantTs, false, numInstantsAfterBootstrap + 1,
updateTimestamp, deltaCommit ? timestamp : updateTimestamp, deltaCommit);
updateTimestamp, deltaCommit ? timestamp : updateTimestamp, deltaCommit, true);
if (deltaCommit) {
Option<String> compactionInstant = client.scheduleCompaction(Option.empty());
@@ -298,7 +298,7 @@ public class TestOrcBootstrap extends HoodieClientTestBase {
client.compact(compactionInstant.get());
checkBootstrapResults(totalRecords, schema, compactionInstant.get(), checkNumRawFiles,
numInstantsAfterBootstrap + 2, 2, updateTimestamp, updateTimestamp, !deltaCommit,
Arrays.asList(compactionInstant.get()));
Arrays.asList(compactionInstant.get()), !config.isPreserveHoodieCommitMetadataForCompaction());
}
}
@@ -328,14 +328,14 @@ public class TestOrcBootstrap extends HoodieClientTestBase {
}
private void checkBootstrapResults(int totalRecords, Schema schema, String maxInstant, boolean checkNumRawFiles,
int expNumInstants, long expTimestamp, long expROTimestamp, boolean isDeltaCommit) throws Exception {
int expNumInstants, long expTimestamp, long expROTimestamp, boolean isDeltaCommit, boolean validateRecordsForCommitTime) throws Exception {
checkBootstrapResults(totalRecords, schema, maxInstant, checkNumRawFiles, expNumInstants, expNumInstants,
expTimestamp, expROTimestamp, isDeltaCommit, Arrays.asList(maxInstant));
expTimestamp, expROTimestamp, isDeltaCommit, Arrays.asList(maxInstant), validateRecordsForCommitTime);
}
private void checkBootstrapResults(int totalRecords, Schema schema, String instant, boolean checkNumRawFiles,
int expNumInstants, int numVersions, long expTimestamp, long expROTimestamp, boolean isDeltaCommit,
List<String> instantsWithValidRecords) throws Exception {
List<String> instantsWithValidRecords, boolean validateCommitRecords) throws Exception {
metaClient.reloadActiveTimeline();
assertEquals(expNumInstants, metaClient.getCommitsTimeline().filterCompletedInstants().countInstants());
assertEquals(instant, metaClient.getActiveTimeline()
@@ -355,8 +355,10 @@ public class TestOrcBootstrap extends HoodieClientTestBase {
if (!isDeltaCommit) {
String predicate = String.join(", ",
instantsWithValidRecords.stream().map(p -> "\"" + p + "\"").collect(Collectors.toList()));
assertEquals(totalRecords, sqlContext.sql("select * from bootstrapped where _hoodie_commit_time IN "
+ "(" + predicate + ")").count());
if (validateCommitRecords) {
assertEquals(totalRecords, sqlContext.sql("select * from bootstrapped where _hoodie_commit_time IN "
+ "(" + predicate + ")").count());
}
Dataset<Row> missingOriginal = sqlContext.sql("select a._row_key from original a where a._row_key not "
+ "in (select _hoodie_record_key from bootstrapped)");
assertEquals(0, missingOriginal.count());

View File

@@ -272,8 +272,9 @@ class TestMORDataSource extends HoodieClientTestBase {
.option(DataSourceReadOptions.BEGIN_INSTANTTIME.key, commit5Time)
.option(DataSourceReadOptions.END_INSTANTTIME.key, commit6Time)
.load(basePath)
// compaction updated 150 rows + inserted 2 new row
assertEquals(152, hudiIncDF6.count())
// even though compaction updated 150 rows, since preserve commit metadata is true, they won't be part of incremental query.
// inserted 2 new row
assertEquals(2, hudiIncDF6.count())
}
@Test

View File

@@ -66,7 +66,7 @@ public class DefaultSource extends BaseDefaultSource implements DataSourceV2,
String path = options.get("path").get();
String tblName = options.get(HoodieWriteConfig.TBL_NAME.key()).get();
boolean populateMetaFields = options.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS.key(),
Boolean.parseBoolean(HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()));
HoodieTableConfig.POPULATE_META_FIELDS.defaultValue());
Map<String, String> properties = options.asMap();
// Auto set the value of "hoodie.parquet.writelegacyformat.enabled"
mayBeOverwriteParquetWriteLegacyFormatProp(properties, schema);

View File

@@ -53,7 +53,7 @@ public class DefaultSource extends BaseDefaultSource implements TableProvider {
String path = properties.get("path");
String tblName = properties.get(HoodieWriteConfig.TBL_NAME.key());
boolean populateMetaFields = Boolean.parseBoolean(properties.getOrDefault(HoodieTableConfig.POPULATE_META_FIELDS.key(),
HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()));
Boolean.toString(HoodieTableConfig.POPULATE_META_FIELDS.defaultValue())));
boolean arePartitionRecordsSorted = Boolean.parseBoolean(properties.getOrDefault(HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED,
Boolean.toString(HoodieInternalConfig.DEFAULT_BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED)));
// Create a new map as the properties is an unmodifiableMap on Spark 3.2.0