|
|
|
|
@@ -45,6 +45,7 @@ import org.junit.jupiter.api.Tag;
|
|
|
|
|
import org.junit.jupiter.params.ParameterizedTest;
|
|
|
|
|
import org.junit.jupiter.params.provider.Arguments;
|
|
|
|
|
import org.junit.jupiter.params.provider.MethodSource;
|
|
|
|
|
import org.junit.jupiter.params.provider.ValueSource;
|
|
|
|
|
|
|
|
|
|
import java.util.Arrays;
|
|
|
|
|
import java.util.List;
|
|
|
|
|
@@ -52,6 +53,7 @@ import java.util.stream.Stream;
|
|
|
|
|
|
|
|
|
|
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
|
|
|
|
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
|
|
|
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
|
|
|
|
|
|
|
|
|
@Tag("functional")
|
|
|
|
|
class TestHoodieSparkMergeOnReadTableClustering extends SparkClientFunctionalTestHarness {
|
|
|
|
|
@@ -111,7 +113,8 @@ class TestHoodieSparkMergeOnReadTableClustering extends SparkClientFunctionalTes
|
|
|
|
|
client.startCommitWithTime(newCommitTime);
|
|
|
|
|
|
|
|
|
|
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 400);
|
|
|
|
|
insertRecords(metaClient, records.subList(0, 200), client, cfg, newCommitTime);
|
|
|
|
|
Stream<HoodieBaseFile> dataFiles = insertRecords(metaClient, records.subList(0, 200), client, cfg, newCommitTime);
|
|
|
|
|
assertTrue(dataFiles.findAny().isPresent(), "should list the base files we wrote in the delta commit");
|
|
|
|
|
|
|
|
|
|
/*
|
|
|
|
|
* Write 2 (more inserts to create new files)
|
|
|
|
|
@@ -119,7 +122,8 @@ class TestHoodieSparkMergeOnReadTableClustering extends SparkClientFunctionalTes
|
|
|
|
|
// we already set small file size to small number to force inserts to go into new file.
|
|
|
|
|
newCommitTime = "002";
|
|
|
|
|
client.startCommitWithTime(newCommitTime);
|
|
|
|
|
insertRecords(metaClient, records.subList(200, 400), client, cfg, newCommitTime);
|
|
|
|
|
dataFiles = insertRecords(metaClient, records.subList(200, 400), client, cfg, newCommitTime);
|
|
|
|
|
assertTrue(dataFiles.findAny().isPresent(), "should list the base files we wrote in the delta commit");
|
|
|
|
|
|
|
|
|
|
if (doUpdates) {
|
|
|
|
|
/*
|
|
|
|
|
@@ -144,28 +148,101 @@ class TestHoodieSparkMergeOnReadTableClustering extends SparkClientFunctionalTes
|
|
|
|
|
assertEquals(allFiles.length, hoodieTable.getFileSystemView().getFileGroupsInPendingClustering().map(Pair::getLeft).count());
|
|
|
|
|
|
|
|
|
|
// Do the clustering and validate
|
|
|
|
|
client.cluster(clusteringCommitTime, true);
|
|
|
|
|
|
|
|
|
|
metaClient = HoodieTableMetaClient.reload(metaClient);
|
|
|
|
|
final HoodieTable clusteredTable = HoodieSparkTable.create(cfg, context(), metaClient);
|
|
|
|
|
clusteredTable.getHoodieView().sync();
|
|
|
|
|
Stream<HoodieBaseFile> dataFilesToRead = Arrays.stream(dataGen.getPartitionPaths())
|
|
|
|
|
.flatMap(p -> clusteredTable.getBaseFileOnlyView().getLatestBaseFiles(p));
|
|
|
|
|
// verify there should be only one base file per partition after clustering.
|
|
|
|
|
assertEquals(dataGen.getPartitionPaths().length, dataFilesToRead.count());
|
|
|
|
|
|
|
|
|
|
HoodieTimeline timeline = metaClient.getCommitTimeline().filterCompletedInstants();
|
|
|
|
|
assertEquals(1, timeline.findInstantsAfter("003", Integer.MAX_VALUE).countInstants(),
|
|
|
|
|
"Expecting a single commit.");
|
|
|
|
|
assertEquals(clusteringCommitTime, timeline.lastInstant().get().getTimestamp());
|
|
|
|
|
assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, timeline.lastInstant().get().getAction());
|
|
|
|
|
if (cfg.populateMetaFields()) {
|
|
|
|
|
assertEquals(400, HoodieClientTestUtils.countRecordsOptionallySince(jsc(), basePath(), sqlContext(), timeline, Option.of("000")),
|
|
|
|
|
"Must contain 200 records");
|
|
|
|
|
} else {
|
|
|
|
|
assertEquals(400, HoodieClientTestUtils.countRecordsOptionallySince(jsc(), basePath(), sqlContext(), timeline, Option.empty()));
|
|
|
|
|
}
|
|
|
|
|
doClusteringAndValidate(client, clusteringCommitTime, metaClient, cfg, dataGen);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@ParameterizedTest
|
|
|
|
|
@ValueSource(booleans = {true, false})
|
|
|
|
|
void testClusteringWithNoBaseFiles(boolean doUpdates) throws Exception {
|
|
|
|
|
// set low compaction small File Size to generate more file groups.
|
|
|
|
|
HoodieWriteConfig.Builder cfgBuilder = HoodieWriteConfig.newBuilder()
|
|
|
|
|
.forTable("test-trip-table")
|
|
|
|
|
.withPath(basePath())
|
|
|
|
|
.withSchema(TRIP_EXAMPLE_SCHEMA)
|
|
|
|
|
.withParallelism(2, 2)
|
|
|
|
|
.withDeleteParallelism(2)
|
|
|
|
|
.withAutoCommit(true)
|
|
|
|
|
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
|
|
|
|
.compactionSmallFileSize(10L)
|
|
|
|
|
.withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build())
|
|
|
|
|
.withStorageConfig(HoodieStorageConfig.newBuilder()
|
|
|
|
|
.hfileMaxFileSize(1024 * 1024 * 1024)
|
|
|
|
|
.parquetMaxFileSize(1024 * 1024 * 1024).build())
|
|
|
|
|
.withEmbeddedTimelineServerEnabled(true)
|
|
|
|
|
.withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder()
|
|
|
|
|
.withEnableBackupForRemoteFileSystemView(false).build())
|
|
|
|
|
// set index type to INMEMORY so that log files can be indexed, and it is safe to send
|
|
|
|
|
// inserts straight to the log to produce file slices with only log files and no data files
|
|
|
|
|
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build())
|
|
|
|
|
.withClusteringConfig(HoodieClusteringConfig.newBuilder()
|
|
|
|
|
.withClusteringMaxNumGroups(10)
|
|
|
|
|
.withClusteringTargetPartitions(0)
|
|
|
|
|
.withInlineClustering(true)
|
|
|
|
|
.withInlineClusteringNumCommits(1).build())
|
|
|
|
|
.withRollbackUsingMarkers(false);
|
|
|
|
|
HoodieWriteConfig cfg = cfgBuilder.build();
|
|
|
|
|
HoodieTableMetaClient metaClient = getHoodieMetaClient(HoodieTableType.MERGE_ON_READ, cfg.getProps());
|
|
|
|
|
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
|
|
|
|
|
|
|
|
|
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg)) {
|
|
|
|
|
// test 2 inserts
|
|
|
|
|
String newCommitTime = "001";
|
|
|
|
|
client.startCommitWithTime(newCommitTime);
|
|
|
|
|
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 400);
|
|
|
|
|
Stream<HoodieBaseFile> dataFiles = insertRecords(metaClient, records.subList(0, 200), client, cfg, newCommitTime);
|
|
|
|
|
assertTrue(!dataFiles.findAny().isPresent(), "should not have any base files");
|
|
|
|
|
newCommitTime = "002";
|
|
|
|
|
client.startCommitWithTime(newCommitTime);
|
|
|
|
|
dataFiles = insertRecords(metaClient, records.subList(200, 400), client, cfg, newCommitTime);
|
|
|
|
|
assertTrue(!dataFiles.findAny().isPresent(), "should not have any base files");
|
|
|
|
|
// run updates
|
|
|
|
|
if (doUpdates) {
|
|
|
|
|
newCommitTime = "003";
|
|
|
|
|
client.startCommitWithTime(newCommitTime);
|
|
|
|
|
records = dataGen.generateUpdates(newCommitTime, 100);
|
|
|
|
|
updateRecords(metaClient, records, client, cfg, newCommitTime);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context(), metaClient);
|
|
|
|
|
hoodieTable.getHoodieView().sync();
|
|
|
|
|
FileStatus[] allBaseFiles = listAllBaseFilesInPath(hoodieTable);
|
|
|
|
|
// expect 0 base files for each partition
|
|
|
|
|
assertEquals(0, allBaseFiles.length);
|
|
|
|
|
|
|
|
|
|
String clusteringCommitTime = client.scheduleClustering(Option.empty()).get().toString();
|
|
|
|
|
metaClient = HoodieTableMetaClient.reload(metaClient);
|
|
|
|
|
hoodieTable = HoodieSparkTable.create(cfg, context(), metaClient);
|
|
|
|
|
// verify log files are included in clustering plan for each partition.
|
|
|
|
|
assertEquals(dataGen.getPartitionPaths().length, hoodieTable.getFileSystemView().getFileGroupsInPendingClustering().map(Pair::getLeft).count());
|
|
|
|
|
|
|
|
|
|
// do the clustering and validate
|
|
|
|
|
doClusteringAndValidate(client, clusteringCommitTime, metaClient, cfg, dataGen);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private void doClusteringAndValidate(SparkRDDWriteClient client,
|
|
|
|
|
String clusteringCommitTime,
|
|
|
|
|
HoodieTableMetaClient metaClient,
|
|
|
|
|
HoodieWriteConfig cfg,
|
|
|
|
|
HoodieTestDataGenerator dataGen) {
|
|
|
|
|
client.cluster(clusteringCommitTime, true);
|
|
|
|
|
metaClient = HoodieTableMetaClient.reload(metaClient);
|
|
|
|
|
final HoodieTable clusteredTable = HoodieSparkTable.create(cfg, context(), metaClient);
|
|
|
|
|
clusteredTable.getHoodieView().sync();
|
|
|
|
|
Stream<HoodieBaseFile> dataFilesToRead = Arrays.stream(dataGen.getPartitionPaths())
|
|
|
|
|
.flatMap(p -> clusteredTable.getBaseFileOnlyView().getLatestBaseFiles(p));
|
|
|
|
|
assertEquals(dataGen.getPartitionPaths().length, dataFilesToRead.count());
|
|
|
|
|
HoodieTimeline timeline = metaClient.getCommitTimeline().filterCompletedInstants();
|
|
|
|
|
assertEquals(1, timeline.findInstantsAfter("003", Integer.MAX_VALUE).countInstants(),
|
|
|
|
|
"Expecting a single commit.");
|
|
|
|
|
assertEquals(clusteringCommitTime, timeline.lastInstant().get().getTimestamp());
|
|
|
|
|
assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, timeline.lastInstant().get().getAction());
|
|
|
|
|
if (cfg.populateMetaFields()) {
|
|
|
|
|
assertEquals(400, HoodieClientTestUtils.countRecordsOptionallySince(jsc(), basePath(), sqlContext(), timeline, Option.of("000")),
|
|
|
|
|
"Must contain 200 records");
|
|
|
|
|
} else {
|
|
|
|
|
assertEquals(400, HoodieClientTestUtils.countRecordsOptionallySince(jsc(), basePath(), sqlContext(), timeline, Option.empty()));
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|