1
0

[HUDI-2072] Add pre-commit validator framework (#3153)

* [HUDI-2072] Add pre-commit validator framework

* trigger Travis rebuild
This commit is contained in:
satishkotha
2021-08-03 12:07:45 -07:00
committed by GitHub
parent bec23bda50
commit 826a04d142
14 changed files with 1130 additions and 32 deletions

View File

@@ -25,6 +25,10 @@ import org.apache.hudi.avro.model.HoodieClusteringPlan;
import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.client.validator.SparkPreCommitValidator;
import org.apache.hudi.client.validator.SqlQueryEqualityPreCommitValidator;
import org.apache.hudi.client.validator.SqlQuerySingleResultPreCommitValidator;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.FileSlice;
@@ -34,6 +38,7 @@ import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.model.IOType;
import org.apache.hudi.common.model.WriteOperationType;
@@ -53,17 +58,21 @@ import org.apache.hudi.common.util.CollectionUtils;
import org.apache.hudi.common.util.BaseFileUtils;
import org.apache.hudi.common.util.FileIOUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieClusteringConfig;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodiePreCommitValidatorConfig;
import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieCommitException;
import org.apache.hudi.exception.HoodieCorruptedDataException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieInsertException;
import org.apache.hudi.exception.HoodieRollbackException;
import org.apache.hudi.exception.HoodieUpsertException;
import org.apache.hudi.exception.HoodieValidationException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.index.HoodieIndex.IndexType;
import org.apache.hudi.io.HoodieMergeHandle;
@@ -169,6 +178,8 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
private HoodieTestTable testTable;
private static final String COUNT_SQL_QUERY_FOR_VALIDATION = "select count(*) from <TABLE_NAME>";
@BeforeEach
public void setUpTestTable() {
testTable = HoodieSparkWriteableTestTable.of(metaClient);
@@ -256,6 +267,106 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
}
}
@Test
public void testPreCommitValidatorsOnInsert() throws Exception {
int numRecords = 200;
HoodiePreCommitValidatorConfig validatorConfig = HoodiePreCommitValidatorConfig.newBuilder()
.withPreCommitValidator(SqlQuerySingleResultPreCommitValidator.class.getName())
.withPrecommitValidatorSingleResultSqlQueries(COUNT_SQL_QUERY_FOR_VALIDATION + "#" + numRecords)
.build();
HoodieWriteConfig config = getConfigBuilder().withAutoCommit(true)
.withPreCommitValidatorConfig(validatorConfig).build();
try (SparkRDDWriteClient client = getHoodieWriteClient(config)) {
Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> writeFn = (writeClient, recordRDD, instantTime) ->
writeClient.bulkInsert(recordRDD, instantTime, Option.empty());
String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
JavaRDD<WriteStatus> result = insertFirstBatch(config, client, newCommitTime,
"000", numRecords, writeFn, false, false, numRecords);
assertTrue(testTable.commitExists(newCommitTime));
}
}
@Test
public void testPreCommitValidationFailureOnInsert() throws Exception {
int numRecords = 200;
HoodiePreCommitValidatorConfig validatorConfig = HoodiePreCommitValidatorConfig.newBuilder()
.withPreCommitValidator(SqlQuerySingleResultPreCommitValidator.class.getName())
//set wrong value for expected number of rows
.withPrecommitValidatorSingleResultSqlQueries(COUNT_SQL_QUERY_FOR_VALIDATION + "#" + 500)
.build();
HoodieWriteConfig config = getConfigBuilder().withPreCommitValidatorConfig(validatorConfig).build();
String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
try (SparkRDDWriteClient client = getHoodieWriteClient(config)) {
Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> writeFn = (writeClient, recordRDD, instantTime) ->
writeClient.bulkInsert(recordRDD, instantTime, Option.empty());
JavaRDD<WriteStatus> result = insertFirstBatch(config, client, newCommitTime,
"000", numRecords, writeFn, false, false, numRecords);
fail("Expected validation to fail because we only insert 200 rows. Validation is configured to expect 500 rows");
} catch (HoodieInsertException e) {
if (e.getCause() instanceof HoodieValidationException) {
// expected because wrong value passed
} else {
throw e;
}
}
assertFalse(testTable.commitExists(newCommitTime));
}
@Test
public void testPreCommitValidationWithMultipleInflights() throws Exception {
int numRecords = 200;
HoodiePreCommitValidatorConfig validatorConfig = HoodiePreCommitValidatorConfig.newBuilder()
.withPreCommitValidator(SqlQuerySingleResultPreCommitValidator.class.getName())
//set wrong value for expected number of rows
.withPrecommitValidatorSingleResultSqlQueries(COUNT_SQL_QUERY_FOR_VALIDATION + "#" + 500)
.build();
HoodieWriteConfig config = getConfigBuilder()
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.NEVER).build())
.withPreCommitValidatorConfig(validatorConfig)
.build();
String instant1 = HoodieActiveTimeline.createNewInstantTime();
try {
insertWithConfig(config, numRecords, instant1);
fail("Expected validation to fail because we only insert 200 rows. Validation is configured to expect 500 rows");
} catch (HoodieInsertException e) {
if (e.getCause() instanceof HoodieValidationException) {
// expected because wrong value passed
} else {
throw e;
}
}
assertFalse(testTable.commitExists(instant1));
assertTrue(testTable.inflightCommitExists(instant1));
numRecords = 300;
validatorConfig = HoodiePreCommitValidatorConfig.newBuilder()
.withPreCommitValidator(SqlQuerySingleResultPreCommitValidator.class.getName())
//set wrong value for expected number of rows
.withPrecommitValidatorSingleResultSqlQueries(COUNT_SQL_QUERY_FOR_VALIDATION + "#" + numRecords)
.build();
config = getConfigBuilder()
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.NEVER).build())
.withPreCommitValidatorConfig(validatorConfig)
.build();
String instant2 = HoodieActiveTimeline.createNewInstantTime();
// expect pre-commit validators to succeed. Note that validator is expected to exclude inflight instant1
insertWithConfig(config, numRecords, instant2);
assertTrue(testTable.inflightCommitExists(instant1));
assertTrue(testTable.commitExists(instant2));
}
private void insertWithConfig(HoodieWriteConfig config, int numRecords, String instant) throws Exception {
try (SparkRDDWriteClient client = getHoodieWriteClient(config)) {
Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> writeFn = (writeClient, recordRDD, instantTime) ->
writeClient.bulkInsert(recordRDD, instantTime, Option.empty());
JavaRDD<WriteStatus> result = insertFirstBatch(config, client, instant,
"000", numRecords, writeFn, false, false, numRecords);
}
}
/**
* Test De-duplication behavior for HoodieWriteClient insert API.
*/
@@ -1176,31 +1287,31 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
@ParameterizedTest
@MethodSource("populateMetaFieldsParams")
public void testSimpleClustering(boolean populateMetaFields) throws Exception {
// setup clustering config
// setup clustering config.
HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10)
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build();
testClustering(clusteringConfig, populateMetaFields);
testInsertAndClustering(clusteringConfig, populateMetaFields, true, SqlQueryEqualityPreCommitValidator.class.getName(), COUNT_SQL_QUERY_FOR_VALIDATION, "");
}
@ParameterizedTest
@MethodSource("populateMetaFieldsParams")
public void testClusteringWithSortColumns(boolean populateMetaFields) throws Exception {
// setup clustering config
// setup clustering config.
HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10)
.withClusteringSortColumns(populateMetaFields ? "_hoodie_record_key" : "_row_key")
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build();
testClustering(clusteringConfig, populateMetaFields);
testInsertAndClustering(clusteringConfig, populateMetaFields, true, SqlQueryEqualityPreCommitValidator.class.getName(), COUNT_SQL_QUERY_FOR_VALIDATION, "");
}
@ParameterizedTest
@MethodSource("populateMetaFieldsParams")
public void testPendingClusteringRollback(boolean populateMetaFields) throws Exception {
// setup clustering config
// setup clustering config.
HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10)
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build();
// start clustering, but don't commit
List<HoodieRecord> allRecords = testClustering(clusteringConfig, populateMetaFields);
List<HoodieRecord> allRecords = testInsertAndClustering(clusteringConfig, populateMetaFields, false);
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).build();
List<Pair<HoodieInstant, HoodieClusteringPlan>> pendingClusteringPlans =
ClusteringUtils.getAllPendingClusteringPlans(metaClient).collect(Collectors.toList());
@@ -1215,7 +1326,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
dataGen = new HoodieTestDataGenerator();
String commitTime = HoodieActiveTimeline.createNewInstantTime();
allRecords.addAll(dataGen.generateInserts(commitTime, 200));
writeAndVerifyBatch(client, allRecords, commitTime);
writeAndVerifyBatch(client, allRecords, commitTime, populateMetaFields);
// verify pending clustering can be rolled back (even though there is a completed commit greater than pending clustering)
client.rollback(pendingClusteringInstant.getTimestamp());
@@ -1224,50 +1335,141 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
assertEquals(0, ClusteringUtils.getAllPendingClusteringPlans(metaClient).count());
}
private List<HoodieRecord> testClustering(HoodieClusteringConfig clusteringConfig, boolean populateMetaFields) throws Exception {
return testClustering(clusteringConfig, false, populateMetaFields);
@Test
public void testClusteringWithFailingValidator() throws Exception {
// setup clustering config.
HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10)
.withClusteringSortColumns("_hoodie_record_key")
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build();
try {
testInsertAndClustering(clusteringConfig, true, true, FailingPreCommitValidator.class.getName(), COUNT_SQL_QUERY_FOR_VALIDATION, "");
fail("expected pre-commit clustering validation to fail");
} catch (HoodieValidationException e) {
// expected
}
}
@Test
public void testClusteringInvalidConfigForSqlQueryValidator() throws Exception {
// setup clustering config.
HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10)
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build();
try {
testInsertAndClustering(clusteringConfig, false, true, SqlQueryEqualityPreCommitValidator.class.getName(), "", "");
fail("expected pre-commit clustering validation to fail because sql query is not configured");
} catch (HoodieValidationException e) {
// expected
}
}
@Test
public void testClusteringInvalidConfigForSqlQuerySingleResultValidator() throws Exception {
// setup clustering config.
HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10)
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build();
testInsertAndClustering(clusteringConfig, false, true, SqlQuerySingleResultPreCommitValidator.class.getName(),
"", COUNT_SQL_QUERY_FOR_VALIDATION + "#400");
}
@Test
public void testClusteringInvalidConfigForSqlQuerySingleResultValidatorFailure() throws Exception {
// setup clustering config.
HoodieClusteringConfig clusteringConfig = HoodieClusteringConfig.newBuilder().withClusteringMaxNumGroups(10)
.withClusteringTargetPartitions(0).withInlineClusteringNumCommits(1).build();
try {
testInsertAndClustering(clusteringConfig, false, true, SqlQuerySingleResultPreCommitValidator.class.getName(),
"", COUNT_SQL_QUERY_FOR_VALIDATION + "#802");
fail("expected pre-commit clustering validation to fail because of count mismatch. expect 400 rows, not 802");
} catch (HoodieValidationException e) {
// expected
}
}
private List<HoodieRecord> testInsertAndClustering(HoodieClusteringConfig clusteringConfig, boolean populateMetaFields, boolean completeClustering) throws Exception {
return testInsertAndClustering(clusteringConfig, populateMetaFields, completeClustering, "", "", "");
}
private List<HoodieRecord> testClustering(HoodieClusteringConfig clusteringConfig, boolean completeClustering, boolean populateMetaFields) throws Exception {
private List<HoodieRecord> testInsertAndClustering(HoodieClusteringConfig clusteringConfig, boolean populateMetaFields,
boolean completeClustering, String validatorClasses,
String sqlQueryForEqualityValidation, String sqlQueryForSingleResultValidation) throws Exception {
List<HoodieRecord> allRecords = testInsertTwoBatches(populateMetaFields);
testClustering(clusteringConfig, populateMetaFields, completeClustering, validatorClasses, sqlQueryForEqualityValidation, sqlQueryForSingleResultValidation, allRecords);
return allRecords;
}
private List<HoodieRecord> testInsertTwoBatches(boolean populateMetaFields) throws IOException {
// create config to not update small files.
HoodieWriteConfig config = getSmallInsertWriteConfig(2000, TRIP_EXAMPLE_SCHEMA, 10, false, populateMetaFields,
populateMetaFields ? new Properties() : getPropertiesForKeyGen());
SparkRDDWriteClient client = getHoodieWriteClient(config);
dataGen = new HoodieTestDataGenerator();
dataGen = new HoodieTestDataGenerator(new String[] {"2015/03/16"});
String commitTime = HoodieActiveTimeline.createNewInstantTime();
List<HoodieRecord> records1 = dataGen.generateInserts(commitTime, 200);
List<WriteStatus> statuses1 = writeAndVerifyBatch(client, records1, commitTime);
List<WriteStatus> statuses1 = writeAndVerifyBatch(client, records1, commitTime, populateMetaFields);
Set<HoodieFileGroupId> fileIds1 = getFileGroupIdsFromWriteStatus(statuses1);
commitTime = HoodieActiveTimeline.createNewInstantTime();
List<HoodieRecord> records2 = dataGen.generateInserts(commitTime, 200);
List<WriteStatus> statuses2 = writeAndVerifyBatch(client, records2, commitTime);
List<WriteStatus> statuses2 = writeAndVerifyBatch(client, records2, commitTime, populateMetaFields);
Set<HoodieFileGroupId> fileIds2 = getFileGroupIdsFromWriteStatus(statuses2);
//verify new files are created for 2nd write
Set<HoodieFileGroupId> fileIdIntersection = new HashSet<>(fileIds1);
fileIdIntersection.retainAll(fileIds2);
assertEquals(0, fileIdIntersection.size());
return Stream.concat(records1.stream(), records2.stream()).collect(Collectors.toList());
}
private String testClustering(HoodieClusteringConfig clusteringConfig, boolean populateMetaFields, boolean completeClustering,
String validatorClasses, String sqlQueryForEqualityValidation, String sqlQueryForSingleResultValidation,
List<HoodieRecord> allRecords) throws IOException {
config = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY).withAutoCommit(completeClustering)
HoodieWriteConfig config = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY).withAutoCommit(false)
.withClusteringConfig(clusteringConfig)
.withProps(populateMetaFields ? new Properties() : getPropertiesForKeyGen()).build();
.withProps(getPropertiesForKeyGen()).build();
HoodieWriteMetadata<JavaRDD<WriteStatus>> clusterMetadata =
performClustering(clusteringConfig, populateMetaFields, completeClustering, validatorClasses, sqlQueryForEqualityValidation, sqlQueryForSingleResultValidation, allRecords);
if (completeClustering) {
String clusteringCommitTime = metaClient.reloadActiveTimeline().getCompletedReplaceTimeline()
.getReverseOrderedInstants().findFirst().get().getTimestamp();
verifyRecordsWritten(clusteringCommitTime, populateMetaFields, allRecords, clusterMetadata.getWriteStatuses().collect(), config);
return clusteringCommitTime;
} else {
return "";
}
}
private HoodieWriteMetadata<JavaRDD<WriteStatus>> performClustering(HoodieClusteringConfig clusteringConfig,
boolean populateMetaFields,
boolean completeClustering,
String validatorClasses,
String sqlQueryForEqualityValidation, String sqlQueryForSingleResultValidation,
List<HoodieRecord> allRecords) throws IOException {
HoodiePreCommitValidatorConfig validatorConfig = HoodiePreCommitValidatorConfig.newBuilder()
.withPreCommitValidator(StringUtils.nullToEmpty(validatorClasses))
.withPrecommitValidatorEqualitySqlQueries(sqlQueryForEqualityValidation)
.withPrecommitValidatorSingleResultSqlQueries(sqlQueryForSingleResultValidation)
.build();
HoodieWriteConfig config = getConfigBuilder().withAutoCommit(false)
.withPreCommitValidatorConfig(validatorConfig)
.withProps(populateMetaFields ? new Properties() : getPropertiesForKeyGen())
.withClusteringConfig(clusteringConfig).build();
// create client with new config.
client = getHoodieWriteClient(config);
SparkRDDWriteClient client = getHoodieWriteClient(config);
String clusteringCommitTime = client.scheduleClustering(Option.empty()).get().toString();
HoodieWriteMetadata<JavaRDD<WriteStatus>> clusterMetadata = client.cluster(clusteringCommitTime, completeClustering);
List<HoodieRecord> allRecords = Stream.concat(records1.stream(), records2.stream()).collect(Collectors.toList());
verifyRecordsWritten(clusteringCommitTime, allRecords, clusterMetadata.getWriteStatuses().collect(), config);
Set<HoodieFileGroupId> insertedFileIds = new HashSet<>();
insertedFileIds.addAll(fileIds1);
insertedFileIds.addAll(fileIds2);
verifyRecordsWritten(clusteringCommitTime, populateMetaFields, allRecords, clusterMetadata.getWriteStatuses().collect(), config);
Set<HoodieFileGroupId> replacedFileIds = new HashSet<>();
clusterMetadata.getPartitionToReplaceFileIds().entrySet().forEach(partitionFiles ->
partitionFiles.getValue().stream().forEach(file ->
replacedFileIds.add(new HoodieFileGroupId(partitionFiles.getKey(), file))));
assertEquals(insertedFileIds, replacedFileIds);
return allRecords;
return clusterMetadata;
}
private Set<HoodieFileGroupId> getFileGroupIdsFromWriteStatus(List<WriteStatus> statuses) {
@@ -1317,7 +1519,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
// Do Inserts
String commit1 = "001";
List<WriteStatus> statuses = writeAndVerifyBatch(client, dataGen.generateInserts(commit1, batch1RecordsCount), commit1);
List<WriteStatus> statuses = writeAndVerifyBatch(client, dataGen.generateInserts(commit1, batch1RecordsCount), commit1, populateMetaFields);
Set<String> batch1Buckets = getFileIdsFromWriteStatus(statuses);
// Do Insert Overwrite
@@ -1332,7 +1534,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
assertNoWriteErrors(statuses);
assertEquals(batch1Buckets, new HashSet<>(writeResult.getPartitionToReplaceFileIds().get(testPartitionPath)));
verifyRecordsWritten(commitTime2, inserts2, statuses, config);
verifyRecordsWritten(commitTime2, populateMetaFields, inserts2, statuses, config);
}
private Set<String> getFileIdsFromWriteStatus(List<WriteStatus> statuses) {
@@ -1373,7 +1575,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
List<WriteStatus> statuses = client.upsert(insertRecordsRDD1, commitTime1).collect();
assertNoWriteErrors(statuses);
Set<String> batchBuckets = statuses.stream().map(s -> s.getFileId()).collect(Collectors.toSet());
verifyRecordsWritten(commitTime1, inserts1, statuses, client.config);
verifyRecordsWritten(commitTime1, true, inserts1, statuses, client.config);
return batchBuckets;
}
@@ -1451,8 +1653,8 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
/**
* Verify data in base files matches expected records and commit time.
*/
private void verifyRecordsWritten(String commitTime, List<HoodieRecord> expectedRecords, List<WriteStatus> allStatus,
HoodieWriteConfig config) throws IOException {
private void verifyRecordsWritten(String commitTime, boolean populateMetadataField,
List<HoodieRecord> expectedRecords, List<WriteStatus> allStatus, HoodieWriteConfig config) throws IOException {
List<GenericRecord> records = new ArrayList<>();
for (WriteStatus status : allStatus) {
Path filePath = new Path(basePath, status.getStat().getPath());
@@ -1472,18 +1674,20 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
KeyGenerator keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(config.getProps()));
for (GenericRecord record : records) {
String recordKey = keyGenerator.getKey(record).getRecordKey();
assertNull(record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD));
if (!populateMetadataField) {
assertNull(record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD));
}
assertTrue(expectedKeys.contains(recordKey));
}
}
}
private List<WriteStatus> writeAndVerifyBatch(SparkRDDWriteClient client, List<HoodieRecord> inserts, String commitTime) throws IOException {
private List<WriteStatus> writeAndVerifyBatch(SparkRDDWriteClient client, List<HoodieRecord> inserts, String commitTime, boolean populateMetaFields) throws IOException {
client.startCommitWithTime(commitTime);
JavaRDD<HoodieRecord> insertRecordsRDD1 = jsc.parallelize(inserts, 2);
List<WriteStatus> statuses = client.upsert(insertRecordsRDD1, commitTime).collect();
assertNoWriteErrors(statuses);
verifyRecordsWritten(commitTime, inserts, statuses, client.config);
verifyRecordsWritten(commitTime, populateMetaFields, inserts, statuses, client.config);
return statuses;
}
@@ -2130,4 +2334,16 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
.withProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen()).build();
}
public static class FailingPreCommitValidator<T extends HoodieRecordPayload, I, K, O extends JavaRDD<WriteStatus>> extends SparkPreCommitValidator<T, I, K, O> {
public FailingPreCommitValidator(HoodieSparkTable table, HoodieEngineContext context, HoodieWriteConfig config) {
super(table, context, config);
}
@Override
protected void validateRecordsBeforeAndAfter(final Dataset<Row> before, final Dataset<Row> after, final Set<String> partitionsAffected) {
throw new HoodieValidationException("simulate failure");
}
}
}