1
0

[HUDI-1357] Added a check to validate records are not lost during merges. (#2216)

- Turned off by default
This commit is contained in:
Prashant Wason
2020-12-01 13:44:57 -08:00
committed by GitHub
parent b826c53e33
commit ac23d2587f
6 changed files with 122 additions and 2 deletions

View File

@@ -47,10 +47,12 @@ import org.apache.hudi.config.HoodieIndexConfig;
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.HoodieRollbackException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.index.HoodieIndex.IndexType;
import org.apache.hudi.io.HoodieMergeHandle;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.MarkerFiles;
@@ -376,6 +378,53 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
instants.get(3));
assertEquals(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "006"),
instants.get(4));
final HoodieWriteConfig cfg = hoodieWriteConfig;
final String instantTime = "007";
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
String basePathStr = basePath;
HoodieTable table = getHoodieTable(metaClient, cfg);
jsc.parallelize(Arrays.asList(1)).map(e -> {
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(metaClient.getActiveTimeline().getInstantDetails(
metaClient.getCommitsTimeline().filterCompletedInstants().lastInstant().get()).get(),
HoodieCommitMetadata.class);
String filePath = commitMetadata.getPartitionToWriteStats().values().stream()
.flatMap(w -> w.stream()).filter(s -> s.getPath().endsWith(".parquet")).findAny()
.map(ee -> ee.getPath()).orElse(null);
String partitionPath = commitMetadata.getPartitionToWriteStats().values().stream()
.flatMap(w -> w.stream()).filter(s -> s.getPath().endsWith(".parquet")).findAny()
.map(ee -> ee.getPartitionPath()).orElse(null);
Path parquetFilePath = new Path(basePathStr, filePath);
HoodieBaseFile baseFile = new HoodieBaseFile(parquetFilePath.toString());
try {
HoodieMergeHandle handle = new HoodieMergeHandle(cfg, instantTime, table, new HashMap<>(),
partitionPath, FSUtils.getFileId(parquetFilePath.getName()), baseFile, new SparkTaskContextSupplier());
WriteStatus writeStatus = new WriteStatus(false, 0.0);
writeStatus.setStat(new HoodieWriteStat());
writeStatus.getStat().setNumWrites(0);
handle.performMergeDataValidationCheck(writeStatus);
} catch (HoodieCorruptedDataException e1) {
fail("Exception not expected because merge validation check is disabled");
}
try {
final String newInstantTime = "006";
cfg.getProps().setProperty("hoodie.merge.data.validation.enabled", "true");
HoodieWriteConfig cfg2 = HoodieWriteConfig.newBuilder().withProps(cfg.getProps()).build();
HoodieMergeHandle handle = new HoodieMergeHandle(cfg2, newInstantTime, table, new HashMap<>(),
partitionPath, FSUtils.getFileId(parquetFilePath.getName()), baseFile, new SparkTaskContextSupplier());
WriteStatus writeStatus = new WriteStatus(false, 0.0);
writeStatus.setStat(new HoodieWriteStat());
writeStatus.getStat().setNumWrites(0);
handle.performMergeDataValidationCheck(writeStatus);
fail("The above line should have thrown an exception");
} catch (HoodieCorruptedDataException e2) {
// expected
}
return true;
}).collect();
}
/**