diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java
index 56e00aa24cd7c..22f70480a3f4d 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java
@@ -217,6 +217,7 @@ public String showLogFileRecords(
.withSpillableMapBasePath(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH.defaultValue())
.withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue())
.withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue())
+ .withUseScanV2(Boolean.parseBoolean(HoodieCompactionConfig.USE_LOG_RECORD_READER_SCAN_V2.defaultValue()))
.build();
for (HoodieRecord extends HoodieRecordPayload> hoodieRecord : scanner) {
Option record = hoodieRecord.getData().getInsertValue(readerSchema);
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java
index e93ad0c8cad4e..25d54b648715f 100644
--- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java
@@ -228,6 +228,7 @@ public void testShowLogFileRecordsWithMerge() throws IOException, InterruptedExc
.withSpillableMapBasePath(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH.defaultValue())
.withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue())
.withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue())
+ .withUseScanV2(Boolean.parseBoolean(HoodieCompactionConfig.USE_LOG_RECORD_READER_SCAN_V2.defaultValue()))
.build();
Iterator> records = scanner.iterator();
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java
index d9f260e633cfb..146889c8dbfaf 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java
@@ -141,6 +141,7 @@ public abstract class BaseHoodieWriteClient> extraMetadata) {
protected void rollbackFailedBootstrap() {
LOG.info("Rolling back pending bootstrap if present");
HoodieTable table = createTable(config, hadoopConf);
- HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
+ HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingMajorAndMinorCompaction();
Option instant = Option.fromJavaOptional(
inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp).findFirst());
if (instant.isPresent() && HoodieTimeline.compareTimestamps(instant.get(), HoodieTimeline.LESSER_THAN_OR_EQUALS,
@@ -562,6 +563,15 @@ protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata me
inlineScheduleCompaction(extraMetadata);
}
+ // Do an inline log compaction if enabled
+ if (config.inlineLogCompactionEnabled()) {
+ runAnyPendingLogCompactions(table);
+ metadata.addMetadata(HoodieCompactionConfig.INLINE_LOG_COMPACT.key(), "true");
+ inlineLogCompact(extraMetadata);
+ } else {
+ metadata.addMetadata(HoodieCompactionConfig.INLINE_LOG_COMPACT.key(), "false");
+ }
+
// Do an inline clustering if enabled
if (config.inlineClusteringEnabled()) {
runAnyPendingClustering(table);
@@ -589,6 +599,14 @@ protected void runAnyPendingCompactions(HoodieTable table) {
});
}
+ protected void runAnyPendingLogCompactions(HoodieTable table) {
+ table.getActiveTimeline().getWriteTimeline().filterPendingLogCompactionTimeline().getInstants()
+ .forEach(instant -> {
+ LOG.info("Running previously failed inflight log compaction at instant " + instant);
+ logCompact(instant.getTimestamp(), true);
+ });
+ }
+
protected void runAnyPendingClustering(HoodieTable table) {
table.getActiveTimeline().filterPendingReplaceTimeline().getInstants().forEach(instant -> {
Option> instantPlan = ClusteringUtils.getClusteringPlan(table.getMetaClient(), instant);
@@ -1077,13 +1095,60 @@ public abstract void commitCompaction(String compactionInstantTime, HoodieCommit
*/
protected abstract void completeCompaction(HoodieCommitMetadata metadata, HoodieTable table, String compactionCommitTime);
+ /**
+ * Schedules a new log compaction instant.
+ * @param extraMetadata Extra Metadata to be stored
+ */
+ public Option scheduleLogCompaction(Option
*/
-public class HoodieSparkMergeOnReadTable extends HoodieSparkCopyOnWriteTable {
+public class HoodieSparkMergeOnReadTable extends HoodieSparkCopyOnWriteTable implements HoodieCompactionHandler {
HoodieSparkMergeOnReadTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) {
super(config, context, metaClient);
@@ -127,9 +132,8 @@ public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngi
@Override
public Option scheduleCompaction(HoodieEngineContext context, String instantTime, Option> extraMetadata) {
- ScheduleCompactionActionExecutor scheduleCompactionExecutor = new ScheduleCompactionActionExecutor<>(
- context, config, this, instantTime, extraMetadata,
- new HoodieSparkMergeOnReadTableCompactor<>());
+ ScheduleCompactionActionExecutor scheduleCompactionExecutor = new ScheduleCompactionActionExecutor(
+ context, config, this, instantTime, extraMetadata, WriteOperationType.COMPACT);
return scheduleCompactionExecutor.execute();
}
@@ -138,7 +142,7 @@ public HoodieWriteMetadata> compact(
HoodieEngineContext context, String compactionInstantTime) {
RunCompactionActionExecutor compactionExecutor = new RunCompactionActionExecutor<>(
context, config, this, compactionInstantTime, new HoodieSparkMergeOnReadTableCompactor<>(),
- new HoodieSparkCopyOnWriteTable<>(config, context, getMetaClient()));
+ new HoodieSparkCopyOnWriteTable<>(config, context, getMetaClient()), WriteOperationType.COMPACT);
return compactionExecutor.execute();
}
@@ -147,6 +151,21 @@ public HoodieBootstrapWriteMetadata> bootstrap(HoodieEng
return new SparkBootstrapDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, extraMetadata).execute();
}
+ @Override
+ public Option scheduleLogCompaction(HoodieEngineContext context, String instantTime, Option> extraMetadata) {
+ ScheduleCompactionActionExecutor scheduleLogCompactionExecutor = new ScheduleCompactionActionExecutor(
+ context, config, this, instantTime, extraMetadata, WriteOperationType.LOG_COMPACT);
+ return scheduleLogCompactionExecutor.execute();
+ }
+
+ @Override
+ public HoodieWriteMetadata> logCompact(
+ HoodieEngineContext context, String logCompactionInstantTime) {
+ RunCompactionActionExecutor logCompactionExecutor = new RunCompactionActionExecutor(context, config, this,
+ logCompactionInstantTime, new HoodieSparkMergeOnReadTableCompactor<>(), this, WriteOperationType.LOG_COMPACT);
+ return logCompactionExecutor.execute();
+ }
+
@Override
public void rollbackBootstrap(HoodieEngineContext context, String instantTime) {
new RestorePlanActionExecutor<>(context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute();
@@ -161,6 +180,17 @@ public Option scheduleRollback(HoodieEngineContext context,
shouldRollbackUsingMarkers).execute();
}
+ @Override
+ public Iterator> handleInsertsForLogCompaction(String instantTime, String partitionPath, String fileId,
+ Map> recordMap,
+ Map header) {
+ HoodieAppendHandle appendHandle = new HoodieAppendHandle(config, instantTime, this,
+ partitionPath, fileId, recordMap.values().iterator(), taskContextSupplier, header);
+ appendHandle.write(recordMap);
+ List writeStatuses = appendHandle.close();
+ return Collections.singletonList(writeStatuses).iterator();
+ }
+
@Override
public HoodieRollbackMetadata rollback(HoodieEngineContext context,
String rollbackInstantTime,
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java
index 61cb1ffd27bd1..af3e4960ad071 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java
@@ -23,6 +23,7 @@
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.WriteOperationType;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.config.HoodieWriteConfig;
@@ -41,11 +42,13 @@ public class HoodieSparkMergeOnReadTableCompactor
@Override
public void preCompact(
- HoodieTable table, HoodieTimeline pendingCompactionTimeline, String compactionInstantTime) {
- HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime);
- if (!pendingCompactionTimeline.containsInstant(instant)) {
+ HoodieTable table, HoodieTimeline pendingCompactionTimeline, WriteOperationType operationType, String instantTime) {
+ HoodieInstant requestedCompactionInstantTime = WriteOperationType.COMPACT.equals(operationType)
+ ? HoodieTimeline.getCompactionRequestedInstant(instantTime)
+ : HoodieTimeline.getLogCompactionRequestedInstant(instantTime);
+ if (!pendingCompactionTimeline.containsInstant(requestedCompactionInstantTime)) {
throw new IllegalStateException(
- "No Compaction request available at " + compactionInstantTime + " to run compaction");
+ "No Compaction request available at " + requestedCompactionInstantTime.getTimestamp() + " to run compaction");
}
}
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java
index 67d82578fccbf..a5cc430b6d1c7 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java
@@ -100,7 +100,7 @@ public void testUnscheduleCompactionFileId() throws Exception {
Stream.of("001", "003", "005", "007").map(instant -> {
try {
return Pair.of(instant, CompactionUtils.getCompactionPlan(metaClient, instant));
- } catch (IOException ioe) {
+ } catch (Exception ioe) {
throw new HoodieException(ioe);
}
}).map(instantWithPlan -> instantWithPlan.getRight().getOperations().stream()
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestDataValidationCheckForLogCompactionActions.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestDataValidationCheckForLogCompactionActions.java
new file mode 100644
index 0000000000000..eb9d2b462865a
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestDataValidationCheckForLogCompactionActions.java
@@ -0,0 +1,419 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.client.functional;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.fs.ConsistencyGuardConfig;
+import org.apache.hudi.common.model.HoodieAvroRecord;
+import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
+import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
+import org.apache.hudi.common.table.view.FileSystemViewStorageType;
+import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
+import org.apache.hudi.common.testutils.HoodieTestTable;
+import org.apache.hudi.common.testutils.HoodieTestUtils;
+import org.apache.hudi.common.testutils.RawTripTestPayload;
+import org.apache.hudi.common.util.CollectionUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieCleanConfig;
+import org.apache.hudi.config.HoodieCompactionConfig;
+import org.apache.hudi.config.HoodieIndexConfig;
+import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.testutils.GenericRecordValidationTestUtils;
+import org.apache.hudi.testutils.HoodieClientTestBase;
+import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
+import org.apache.hudi.testutils.MetadataMergeWriteStatus;
+import org.apache.spark.api.java.JavaRDD;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.io.TempDir;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.common.model.HoodieRecord.COMMIT_SEQNO_METADATA_FIELD;
+import static org.apache.hudi.common.model.HoodieRecord.COMMIT_TIME_METADATA_FIELD;
+import static org.apache.hudi.common.model.HoodieRecord.FILENAME_METADATA_FIELD;
+import static org.apache.hudi.common.model.HoodieRecord.OPERATION_METADATA_FIELD;
+import static org.apache.hudi.common.model.HoodieRecord.RECORD_KEY_METADATA_FIELD;
+import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
+import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
+import static org.apache.hudi.testutils.GenericRecordValidationTestUtils.assertGenericRecords;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestDataValidationCheckForLogCompactionActions extends HoodieClientTestBase {
+
+ private HoodieTestTable testTable;
+ Random random = new Random();
+ public static final String RECORD_KEY_APPEND_VALUE = "-EXP";
+
+ @TempDir
+ java.nio.file.Path secondTableBasePath;
+
+ Function3, HoodieTestDataGenerator, String, Integer> insertsGenFunction =
+ HoodieTestDataGenerator::generateInserts;
+ Function3, HoodieTestDataGenerator, String, Integer> updatesGenFunction =
+ HoodieTestDataGenerator::generateUniqueUpdates;
+ Function2, HoodieTestDataGenerator, Integer> deletesGenFunction =
+ HoodieTestDataGenerator::generateUniqueDeletes;
+
+ Function3, SparkRDDWriteClient, JavaRDD, String> insertsFunction = SparkRDDWriteClient::insert;
+ Function3, SparkRDDWriteClient, JavaRDD, String> updatesFunction = SparkRDDWriteClient::upsert;
+ Function3, SparkRDDWriteClient, JavaRDD, String> deletesFunction = SparkRDDWriteClient::delete;
+
+ @BeforeEach
+ public void setUpTestTable() {
+ testTable = HoodieSparkWriteableTestTable.of(metaClient);
+ }
+
+ //TODO: include both the table's contents.
+ /**
+ * Cleanups resource group for the subclasses of {@link HoodieClientTestBase}.
+ */
+ @AfterEach
+ public void cleanupResources() throws IOException {
+ cleanupTimelineService();
+ cleanupClients();
+ cleanupSparkContexts();
+ cleanupTestDataGenerator();
+ cleanupFileSystem();
+ cleanupDFS();
+ cleanupExecutorService();
+ System.gc();
+ }
+
+ /**
+ * Stress test logcompaction along with compaction by following approach.
+ * a. Create a random seed to do insert/upsert/deleting operations on main table and replicate same action on experiment table.
+ * b. Schedule inline major compaction to run for every 5 deltacommits on both the tables.
+ * c. After writes on both the tables configure log compaction to run on second table and keep no. of blocks threshold to 2.
+ * d. After every commit operation refresh the timeline and run a validation query for all the records.
+ */
+ @ParameterizedTest
+ @ValueSource(ints = {17})
+ public void stressTestCompactionAndLogCompactionOperations(int seed) throws Exception {
+
+ // Set seed.
+ random.setSeed(seed);
+
+ // Setup First table.
+ TestTableContents mainTable = setupTestTable1();
+
+ // Setup second table.
+ TestTableContents experimentTable = setupTestTable2();
+
+ // Total ingestion writes.
+ int totalWrites = 15;
+
+ LOG.warn("Starting trial with seed " + seed);
+
+ // Current ingestion commit.
+ int curr = 1;
+ while (curr < totalWrites) {
+ LOG.warn("Starting write No. " + curr);
+
+ // Pick an action. It can be insert/update/delete and write data to main table.
+ boolean status = writeOnMainTable(mainTable, curr);
+ if (status) {
+ // Write data into experiment table.
+ writeOnExperimentTable(mainTable, experimentTable);
+
+ // schedule and run log compaction on second table.
+ scheduleLogCompactionOnExperimentTable(experimentTable);
+
+ // Verify that no compaction plans are left on the timeline.
+ assertEquals(0, mainTable.metaClient.reloadActiveTimeline().filterPendingCompactionTimeline().getInstants().count());
+ assertEquals(0, experimentTable.metaClient.reloadActiveTimeline().filterPendingCompactionTimeline().getInstants().count());
+ assertEquals(0, experimentTable.metaClient.reloadActiveTimeline().filterPendingLogCompactionTimeline().getInstants().count());
+
+ // Verify the records in both the tables.
+ verifyRecords(mainTable, experimentTable);
+ LOG.warn("For write No." + curr + ", verification passed. Last ingestion commit timestamp is " + mainTable.commitTimeOnMainTable);
+ }
+ curr++;
+ }
+
+ }
+
+ private void verifyRecords(TestTableContents mainTable, TestTableContents experimentTable) {
+ Map mainRecordsMap =
+ GenericRecordValidationTestUtils.getRecordsMap(mainTable.config, hadoopConf, dataGen);
+ Map experimentRecordsMap =
+ GenericRecordValidationTestUtils.getRecordsMap(experimentTable.config, hadoopConf, dataGen);
+
+ // Verify row count.
+ assertEquals(mainRecordsMap.size(), experimentRecordsMap.size());
+
+ Schema readerSchema = new Schema.Parser().parse(mainTable.config.getSchema());
+ List excludeFields = CollectionUtils.createImmutableList(COMMIT_TIME_METADATA_FIELD, COMMIT_SEQNO_METADATA_FIELD,
+ FILENAME_METADATA_FIELD, OPERATION_METADATA_FIELD, RECORD_KEY_METADATA_FIELD);
+
+ // Verify every field.
+ mainRecordsMap.forEach((key, value) -> {
+ assertTrue(experimentRecordsMap.containsKey(key + RECORD_KEY_APPEND_VALUE));
+ assertGenericRecords(value, experimentRecordsMap.get(key + RECORD_KEY_APPEND_VALUE), readerSchema, excludeFields);
+ });
+ }
+
+ private void scheduleLogCompactionOnExperimentTable(TestTableContents experimentTable) {
+ Option logCompactionTimeStamp = experimentTable.logCompactionClient.scheduleLogCompaction(Option.empty());
+ if (logCompactionTimeStamp.isPresent()) {
+ experimentTable.logCompactionClient.logCompact(logCompactionTimeStamp.get());
+ }
+ }
+
+ private boolean writeOnMainTable(TestTableContents mainTable, int curr) throws IOException {
+ String commitTime = HoodieActiveTimeline.createNewInstantTime();
+ mainTable.client.startCommitWithTime(commitTime);
+
+ int actionType = pickAWriteAction();
+ JavaRDD result;
+ if (curr == 1 || actionType == 0) {
+ result = insertDataIntoMainTable(mainTable, commitTime);
+ } else {
+ try {
+ if (actionType == 1) {
+ result = updateDataIntoMainTable(mainTable, commitTime);
+ } else {
+ result = deleteDataIntoMainTable(mainTable, commitTime);
+ }
+ } catch (IllegalArgumentException e) {
+ LOG.warn(e.getMessage() + " ignoring current command.");
+ return false;
+ }
+ }
+ verifyWriteStatus(result);
+ return true;
+ }
+
+ /**
+ * This method has 50% chance to pick an insert, 30% chance to pick an update and 20% chance to pick a delete operation
+ */
+ private int pickAWriteAction() {
+ int val = random.nextInt(10);
+ if (val < 5) {
+ return 0;
+ } else if (val < 8) {
+ return 1;
+ }
+ return 2;
+ }
+
+ private void writeOnExperimentTable(TestTableContents mainTable, TestTableContents experimentTable) throws IOException {
+ String commitTime = mainTable.commitTimeOnMainTable;
+ experimentTable.client.startCommitWithTime(commitTime);
+ int actionType = mainTable.previousActionType;
+ JavaRDD result;
+ if (actionType == 0) {
+ result = insertDataIntoExperimentTable(mainTable, experimentTable);
+ } else if (actionType == 1) {
+ result = updateDataIntoExperimentTable(mainTable, experimentTable);
+ } else {
+ result = deleteDataIntoExperimentTable(mainTable, experimentTable);
+ }
+ verifyWriteStatus(result);
+ }
+
+ private JavaRDD insertDataIntoMainTable(TestTableContents mainTable, String commitTime) throws IOException {
+ int numRecords = 50 + random.nextInt(10);
+ List records = insertsGenFunction.apply(dataGen, commitTime, numRecords);
+ mainTable.updatePreviousGeneration(records, commitTime, 0);
+ JavaRDD writeRecords = jsc.parallelize(records, 1);
+ return insertsFunction.apply(mainTable.client, writeRecords, commitTime);
+ }
+
+ private JavaRDD updateDataIntoMainTable(TestTableContents mainTable, String commitTime) throws IOException {
+ int numRecords = 10 + random.nextInt(10);
+ List records = updatesGenFunction.apply(dataGen, commitTime, numRecords);
+ mainTable.updatePreviousGeneration(records, commitTime, 1);
+ JavaRDD writeRecords = jsc.parallelize(records, 1);
+ return updatesFunction.apply(mainTable.client, writeRecords, commitTime);
+ }
+
+ private JavaRDD deleteDataIntoMainTable(TestTableContents mainTable, String commitTime) throws IOException {
+ int numRecords = 5 + random.nextInt(10);
+ List keys = deletesGenFunction.apply(dataGen, numRecords);
+ mainTable.updatePreviousGenerationForDelete(keys, commitTime);
+ JavaRDD deleteKeys = jsc.parallelize(keys, 1);
+ return deletesFunction.apply(mainTable.client, deleteKeys, commitTime);
+ }
+
+ private JavaRDD insertDataIntoExperimentTable(TestTableContents mainTable, TestTableContents experimentTable) throws IOException {
+ JavaRDD writeRecords = jsc.parallelize(mainTable.generatedRecords, 1);
+ return insertsFunction.apply(experimentTable.client, writeRecords, mainTable.commitTimeOnMainTable);
+ }
+
+ private JavaRDD updateDataIntoExperimentTable(TestTableContents mainTable, TestTableContents experimentTable) throws IOException {
+ JavaRDD writeRecords = jsc.parallelize(mainTable.generatedRecords, 1);
+ return updatesFunction.apply(experimentTable.client, writeRecords, mainTable.commitTimeOnMainTable);
+ }
+
+ private JavaRDD deleteDataIntoExperimentTable(TestTableContents mainTable, TestTableContents experimentTable) throws IOException {
+ JavaRDD writeKeys = jsc.parallelize(mainTable.generatedKeysForDelete, 1);
+ return deletesFunction.apply(experimentTable.client, writeKeys, mainTable.commitTimeOnMainTable);
+ }
+
+ private void verifyWriteStatus(JavaRDD writeStatuses) {
+ List statuses = writeStatuses.collect();
+ assertNoWriteErrors(statuses);
+ }
+
+ private class TestTableContents {
+ final String basePath;
+ final String tableName;
+ final HoodieTableMetaClient metaClient;
+ final HoodieWriteConfig config;
+ final SparkRDDWriteClient client;
+ String commitTimeOnMainTable = "";
+ List generatedRecords = new ArrayList<>();
+ List generatedKeysForDelete = new ArrayList<>();
+ // 0 means insert, 1 means update, 2 means delete.
+ int previousActionType = 0;
+
+ final SparkRDDWriteClient logCompactionClient;
+
+ public TestTableContents(String basePath, String tableName, HoodieTableMetaClient metaClient,
+ HoodieWriteConfig config, SparkRDDWriteClient client) {
+ this(basePath, tableName, metaClient, config, client, null);
+ }
+
+ public TestTableContents(String basePath, String tableName, HoodieTableMetaClient metaClient, HoodieWriteConfig config,
+ SparkRDDWriteClient client, SparkRDDWriteClient logCompactionClient) {
+ this.basePath = basePath;
+ this.tableName = tableName;
+ this.metaClient = metaClient;
+ this.config = config;
+ this.client = client;
+ this.logCompactionClient = logCompactionClient;
+ }
+
+ private void updatePreviousGeneration(List generatedRecords, String commitTimeOnMainTable, int previousActionType) {
+ Schema schema = new Schema.Parser().parse(this.config.getSchema());
+ this.generatedRecords = generatedRecords.stream().map(rec -> deepCopyAndModifyRecordKey(rec)).collect(Collectors.toList());
+ this.commitTimeOnMainTable = commitTimeOnMainTable;
+ this.previousActionType = previousActionType;
+ }
+
+ private HoodieRecord deepCopyAndModifyRecordKey(HoodieRecord record) {
+ HoodieKey key = deepCopyAndModifyRecordKey(record.getKey());
+ RawTripTestPayload payload = ((RawTripTestPayload)record.getData()).clone();
+ return new HoodieAvroRecord(key, payload);
+ }
+
+ private HoodieKey deepCopyAndModifyRecordKey(HoodieKey key) {
+ return new HoodieKey(key.getRecordKey() + RECORD_KEY_APPEND_VALUE, key.getPartitionPath());
+ }
+
+ private void updatePreviousGenerationForDelete(List generatedKeysForDelete, String commitTimeOnMainTable) {
+ this.generatedKeysForDelete = generatedKeysForDelete.stream().map(this::deepCopyAndModifyRecordKey).collect(Collectors.toList());
+ this.commitTimeOnMainTable = commitTimeOnMainTable;
+ this.previousActionType = 2;
+ }
+ }
+
+ private TestTableContents setupTestTable1() {
+ Properties properties = new Properties();
+ properties.setProperty("hoodie.parquet.small.file.limit", "0");
+ HoodieWriteConfig config = getConfigBuilder(TRIP_EXAMPLE_SCHEMA, HoodieIndex.IndexType.INMEMORY)
+ .withCompactionConfig(HoodieCompactionConfig.newBuilder().withInlineCompaction(true).build())
+ .withAutoCommit(true)
+ .withProperties(properties)
+ .build();
+ SparkRDDWriteClient client = new SparkRDDWriteClient(context, config);
+ return new TestTableContents(basePath, tableName, metaClient, config, client);
+ }
+
+ private TestTableContents setupTestTable2() throws IOException {
+ String tableName2 = "test-trip-table2";
+ String basePath2 = createBasePathForSecondTable(secondTableBasePath);
+ Properties properties = new Properties();
+ properties.put(HoodieTableConfig.NAME.key(), tableName2);
+
+ // Create metaclient
+ HoodieTableMetaClient metaClient2 = HoodieTestUtils.init(hadoopConf, basePath2,
+ HoodieTableType.MERGE_ON_READ, properties);
+ HoodieWriteConfig config2 = getConfigBuilderForSecondTable(tableName2, basePath2,
+ TRIP_EXAMPLE_SCHEMA, HoodieIndex.IndexType.INMEMORY)
+ .withCompactionConfig(HoodieCompactionConfig.newBuilder().withInlineCompaction(true).build())
+ .withAutoCommit(true).build();
+
+ // Create writeClient
+ SparkRDDWriteClient client2 = new SparkRDDWriteClient(context, config2);
+
+ // Create logcompaction client.
+ HoodieWriteConfig logCompactionConfig = HoodieWriteConfig.newBuilder().withProps(config2.getProps())
+ .withCompactionConfig(HoodieCompactionConfig.newBuilder()
+ .withLogCompactionBlocksThreshold("2").build())
+ .build();
+ SparkRDDWriteClient logCompactionClient = new SparkRDDWriteClient(context, logCompactionConfig);
+
+ return new TestTableContents(basePath2, tableName2, metaClient2, config2, client2, logCompactionClient);
+ }
+
+ private String createBasePathForSecondTable(java.nio.file.Path secondTableBasePath) throws IOException {
+ java.nio.file.Path basePath = secondTableBasePath.resolve("dataset2");
+ java.nio.file.Files.createDirectories(basePath);
+ return basePath.toString();
+ }
+
+ private HoodieWriteConfig.Builder getConfigBuilderForSecondTable(String tableName, String basePath, String schemaStr, HoodieIndex.IndexType indexType) {
+ Properties properties = new Properties();
+ properties.setProperty("hoodie.parquet.small.file.limit", "0");
+ return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr)
+ .withParallelism(2, 2).withBulkInsertParallelism(2).withFinalizeWriteParallelism(2).withDeleteParallelism(2)
+ .withTimelineLayoutVersion(TimelineLayoutVersion.CURR_VERSION)
+ .withWriteStatusClass(MetadataMergeWriteStatus.class)
+ .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
+ .withCompactionConfig(HoodieCompactionConfig.newBuilder()
+ .compactionSmallFileSize(1024 * 1024).build())
+ .withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).orcMaxFileSize(1024 * 1024).build())
+ .withCleanConfig(HoodieCleanConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER).build())
+ .forTable(tableName)
+ .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build())
+ .withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
+ .withEnableBackupForRemoteFileSystemView(false) // Fail test if problem connecting to timeline-server
+ .withRemoteServerPort(timelineServicePort)
+ .withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build())
+ .withProperties(properties);
+ }
+
+ @Override
+ protected HoodieTableType getTableType() {
+ return HoodieTableType.MERGE_ON_READ;
+ }
+}
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnMergeOnReadStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnMergeOnReadStorage.java
new file mode 100644
index 0000000000000..e54ca074da742
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnMergeOnReadStorage.java
@@ -0,0 +1,544 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.client.functional;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.transaction.lock.InProcessLockProvider;
+import org.apache.hudi.common.config.HoodieMetadataConfig;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
+import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.model.WriteConcurrencyMode;
+import org.apache.hudi.common.table.log.HoodieUnMergedLogRecordScanner;
+import org.apache.hudi.common.table.marker.MarkerType;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.table.view.SyncableFileSystemView;
+import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
+import org.apache.hudi.common.testutils.HoodieTestTable;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieArchivalConfig;
+import org.apache.hudi.config.HoodieCleanConfig;
+import org.apache.hudi.config.HoodieCompactionConfig;
+import org.apache.hudi.config.HoodieLockConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.hudi.testutils.GenericRecordValidationTestUtils;
+import org.apache.hudi.testutils.HoodieClientTestBase;
+import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.testutils.GenericRecordValidationTestUtils.assertDataInMORTable;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestHoodieClientOnMergeOnReadStorage extends HoodieClientTestBase {
+
+ private HoodieTestTable testTable;
+
+ @BeforeEach
+ public void setUpTestTable() {
+ testTable = HoodieSparkWriteableTestTable.of(metaClient);
+ }
+
+ @Test
+ public void testReadingMORTableWithoutBaseFile() throws Exception {
+ HoodieWriteConfig config = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA,
+ HoodieIndex.IndexType.INMEMORY).withAutoCommit(true)
+ .withCompactionConfig(HoodieCompactionConfig.newBuilder().withMaxNumDeltaCommitsBeforeCompaction(2).build())
+ .build();
+ SparkRDDWriteClient client = getHoodieWriteClient(config);
+
+ // Do insert and updates thrice one after the other.
+ // Insert
+ String commitTime = HoodieActiveTimeline.createNewInstantTime();
+ insertBatch(config, client, commitTime, "000", 10, SparkRDDWriteClient::insert,
+ false, false, 10, 10, 1, Option.empty());
+
+ // Update
+ String commitTimeBetweenPrevAndNew = commitTime;
+ commitTime = HoodieActiveTimeline.createNewInstantTime();
+ updateBatch(config, client, commitTime, commitTimeBetweenPrevAndNew,
+ Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", 5, SparkRDDWriteClient::upsert,
+ false, false, 5, 10, 2, config.populateMetaFields());
+
+ // Delete 5 records
+ String prevCommitTime = commitTime;
+ commitTime = HoodieActiveTimeline.createNewInstantTime();
+ deleteBatch(config, client, commitTime, prevCommitTime,
+ "000", 2, SparkRDDWriteClient::delete, false, false,
+ 0, 150);
+
+ // Verify all the records.
+ metaClient.reloadActiveTimeline();
+ Map recordMap = GenericRecordValidationTestUtils.getRecordsMap(config, hadoopConf, dataGen);
+ assertEquals(8, recordMap.size());
+ }
+
+ @Test
+ public void testCompactionOnMORTable() throws Exception {
+ HoodieWriteConfig config = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA,
+ HoodieIndex.IndexType.INMEMORY).withAutoCommit(true)
+ .withCompactionConfig(HoodieCompactionConfig.newBuilder().withMaxNumDeltaCommitsBeforeCompaction(2).build())
+ .build();
+ SparkRDDWriteClient client = getHoodieWriteClient(config);
+
+ // Do insert and updates thrice one after the other.
+ // Insert
+ String commitTime = HoodieActiveTimeline.createNewInstantTime();
+ insertBatch(config, client, commitTime, "000", 10, SparkRDDWriteClient::insert,
+ false, false, 10, 10, 1, Option.empty());
+
+ // Update
+ String commitTimeBetweenPrevAndNew = commitTime;
+ commitTime = HoodieActiveTimeline.createNewInstantTime();
+ updateBatch(config, client, commitTime, commitTimeBetweenPrevAndNew,
+ Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", 5, SparkRDDWriteClient::upsert,
+ false, false, 5, 10, 2, config.populateMetaFields());
+
+ // Schedule and execute compaction.
+ Option timeStamp = client.scheduleCompaction(Option.empty());
+ assertTrue(timeStamp.isPresent());
+ client.compact(timeStamp.get());
+
+ // Verify all the records.
+ metaClient.reloadActiveTimeline();
+ assertDataInMORTable(config, commitTime, timeStamp.get(), hadoopConf, Arrays.asList(dataGen.getPartitionPaths()));
+ }
+
+ @Test
+ public void testLogCompactionOnMORTable() throws Exception {
+ HoodieCompactionConfig compactionConfig = HoodieCompactionConfig.newBuilder()
+ .withLogCompactionBlocksThreshold("1")
+ .build();
+ HoodieWriteConfig config = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA,
+ HoodieIndex.IndexType.INMEMORY).withAutoCommit(true).withCompactionConfig(compactionConfig).build();
+ SparkRDDWriteClient client = getHoodieWriteClient(config);
+
+ // First insert
+ String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
+ insertBatch(config, client, newCommitTime, "000", 10,
+ SparkRDDWriteClient::insert, false, false, 10, 100,
+ 1, Option.empty());
+
+ String prevCommitTime = newCommitTime;
+ for (int i = 0; i < 5; i++) {
+ // Upsert
+ newCommitTime = HoodieActiveTimeline.createNewInstantTime();
+ updateBatch(config, client, newCommitTime, prevCommitTime,
+ Option.of(Arrays.asList(prevCommitTime)), "000", 2, SparkRDDWriteClient::upsert,
+ false, false, 50, 10, i + 2, config.populateMetaFields());
+ prevCommitTime = newCommitTime;
+ }
+
+ // Schedule and execute compaction.
+ Option compactionTimeStamp = client.scheduleCompaction(Option.empty());
+ assertTrue(compactionTimeStamp.isPresent());
+ client.compact(compactionTimeStamp.get());
+
+ prevCommitTime = compactionTimeStamp.get();
+ //TODO: Below commits are creating duplicates when all the tests are run together. but individually they are passing.
+ for (int i = 0; i < 2; i++) {
+ // Upsert
+ newCommitTime = HoodieActiveTimeline.createNewInstantTime();
+ updateBatch(config, client, newCommitTime, prevCommitTime,
+ Option.of(Arrays.asList(prevCommitTime)), "000", 2, SparkRDDWriteClient::upsert,
+ false, false, 50, 10, i + 8, config.populateMetaFields());
+ prevCommitTime = newCommitTime;
+ }
+ String lastCommitBeforeLogCompaction = prevCommitTime;
+
+ // Schedule and execute compaction.
+ Option logCompactionTimeStamp = client.scheduleLogCompaction(Option.empty());
+ assertTrue(logCompactionTimeStamp.isPresent());
+ client.logCompact(logCompactionTimeStamp.get());
+
+ // Verify all the records.
+ assertDataInMORTable(config, lastCommitBeforeLogCompaction, logCompactionTimeStamp.get(),
+ hadoopConf, Arrays.asList(dataGen.getPartitionPaths()));
+ }
+
+ /**
+ * Test logcompaction before any compaction is scheduled. Here base file is not yet created.
+ */
+ @Test
+ public void testLogCompactionOnMORTableWithoutBaseFile() throws Exception {
+ HoodieCompactionConfig compactionConfig = HoodieCompactionConfig.newBuilder()
+ .withLogCompactionBlocksThreshold("1")
+ .withLogRecordReaderScanV2("true")
+ .build();
+ HoodieWriteConfig config = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA,
+ HoodieIndex.IndexType.INMEMORY).withAutoCommit(true).withCompactionConfig(compactionConfig).build();
+ SparkRDDWriteClient client = getHoodieWriteClient(config);
+
+ // First insert 10 records
+ String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
+ insertBatch(config, client, newCommitTime, "000", 10,
+ SparkRDDWriteClient::insert, false, false, 10, 10,
+ 1, Option.of(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH));
+
+ // Upsert 5 records
+ String prevCommitTime = newCommitTime;
+ newCommitTime = HoodieActiveTimeline.createNewInstantTime();
+ updateBatch(config, client, newCommitTime, prevCommitTime,
+ Option.of(Arrays.asList(prevCommitTime)), "000", 5, SparkRDDWriteClient::upsert,
+ false, false, 5, 10, 2, config.populateMetaFields());
+ prevCommitTime = newCommitTime;
+
+ // Delete 3 records
+ newCommitTime = HoodieActiveTimeline.createNewInstantTime();
+ deleteBatch(config, client, newCommitTime, prevCommitTime,
+ "000", 3, SparkRDDWriteClient::delete, false, false,
+ 0, 10);
+
+ String lastCommitBeforeLogCompaction = newCommitTime;
+ // Schedule and execute compaction.
+ Option timeStamp = client.scheduleLogCompaction(Option.empty());
+ assertTrue(timeStamp.isPresent());
+ client.logCompact(timeStamp.get());
+ // Verify all the records.
+ assertDataInMORTable(config, lastCommitBeforeLogCompaction, timeStamp.get(),
+ hadoopConf, Arrays.asList(dataGen.getPartitionPaths()));
+ }
+
+ /**
+ * Test scheduling logcompaction right after scheduling compaction. This should fail.
+ */
+ @Test
+ public void testSchedulingLogCompactionAfterSchedulingCompaction() throws Exception {
+ HoodieCompactionConfig compactionConfig = HoodieCompactionConfig.newBuilder()
+ .withMaxNumDeltaCommitsBeforeCompaction(1)
+ .withLogCompactionBlocksThreshold("1")
+ .build();
+ HoodieWriteConfig config = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA,
+ HoodieIndex.IndexType.INMEMORY).withAutoCommit(true).withCompactionConfig(compactionConfig).build();
+ SparkRDDWriteClient client = getHoodieWriteClient(config);
+
+ // First insert
+ String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
+ insertBatch(config, client, newCommitTime, "000", 100,
+ SparkRDDWriteClient::insert, false, false, 10, 100,
+ 1, Option.empty());
+
+ String prevCommitTime = newCommitTime;
+ // Upsert
+ newCommitTime = HoodieActiveTimeline.createNewInstantTime();
+ updateBatch(config, client, newCommitTime, prevCommitTime,
+ Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert,
+ false, false, 50, 10, 2, config.populateMetaFields());
+
+ // Schedule compaction
+ Option compactionTimeStamp = client.scheduleCompaction(Option.empty());
+ assertTrue(compactionTimeStamp.isPresent());
+
+ // Try scheduing log compaction, it wont succeed
+ Option logCompactionTimeStamp = client.scheduleLogCompaction(Option.empty());
+ assertFalse(logCompactionTimeStamp.isPresent());
+ }
+
+ /**
+ * Test scheduling compaction right after scheduling logcompaction. This should fail.
+ */
+ @Test
+ public void testSchedulingCompactionAfterSchedulingLogCompaction() throws Exception {
+ HoodieCompactionConfig compactionConfig = HoodieCompactionConfig.newBuilder()
+ .withMaxNumDeltaCommitsBeforeCompaction(1)
+ .withLogCompactionBlocksThreshold("1")
+ .build();
+ HoodieWriteConfig config = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA,
+ HoodieIndex.IndexType.INMEMORY).withAutoCommit(true)
+ .withCompactionConfig(compactionConfig)
+ .withCleanConfig(HoodieCleanConfig.newBuilder()
+ .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY)
+ .withAutoClean(false).build())
+ .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL)
+ // Timeline-server-based markers are not used for multi-writer tests
+ .withMarkersType(MarkerType.DIRECT.name())
+ .withLockConfig(HoodieLockConfig.newBuilder().withLockProvider(InProcessLockProvider.class)
+ .build())
+ .build();
+ SparkRDDWriteClient client = getHoodieWriteClient(config);
+
+ // First insert
+ String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
+ insertBatch(config, client, newCommitTime, "000", 100,
+ SparkRDDWriteClient::insert, false, false, 10, 100,
+ 1, Option.empty());
+
+ String prevCommitTime = newCommitTime;
+ // Upsert
+ newCommitTime = HoodieActiveTimeline.createNewInstantTime();
+ updateBatch(config, client, newCommitTime, prevCommitTime,
+ Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert,
+ false, false, 50, 10, 2, config.populateMetaFields());
+
+ // Schedule log compaction
+ Option logCompactionTimeStamp = client.scheduleLogCompaction(Option.empty());
+ assertTrue(logCompactionTimeStamp.isPresent());
+
+ // Try scheduling compaction, it wont succeed
+ Option compactionTimeStamp = client.scheduleCompaction(Option.empty());
+ assertTrue(compactionTimeStamp.isPresent());
+ client.compact(compactionTimeStamp.get());
+ assertThrows(Exception.class, () -> client.logCompact(logCompactionTimeStamp.get()));
+ }
+
+ @Test
+ public void testCleanFunctionalityWhenCompactionRequestedInstantIsPresent() throws Exception {
+ HoodieWriteConfig config = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA,
+ HoodieIndex.IndexType.INMEMORY).withAutoCommit(true)
+ .withCompactionConfig(HoodieCompactionConfig.newBuilder()
+ .withMaxNumDeltaCommitsBeforeCompaction(1)
+ .build())
+ .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(4).build())
+ .build();
+ SparkRDDWriteClient client = getHoodieWriteClient(config);
+
+ // First insert. Here First file slice gets added to file group.
+ String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
+ insertBatch(config, client, newCommitTime, "000", 100,
+ SparkRDDWriteClient::insert, false, false, 10, 100,
+ 1, Option.empty());
+
+ // Schedule and execute compaction. Here, second file slice gets added.
+ Option compactionTimeStamp = client.scheduleCompaction(Option.empty());
+ assertTrue(compactionTimeStamp.isPresent());
+ client.compact(compactionTimeStamp.get());
+ String prevCommitTime = compactionTimeStamp.get();
+
+ // First upsert on second file slice.
+ newCommitTime = HoodieActiveTimeline.createNewInstantTime();
+ updateBatch(config, client, newCommitTime, prevCommitTime,
+ Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert,
+ false, false, 50, 10, 2, config.populateMetaFields());
+ prevCommitTime = newCommitTime;
+
+ // Schedule compaction. Third file slice gets added, compaction is not complete so base file is not created yet.
+ compactionTimeStamp = client.scheduleCompaction(Option.empty());
+ assertTrue(compactionTimeStamp.isPresent());
+ prevCommitTime = compactionTimeStamp.get();
+
+ for (int i = 0; i < 6; i++) {
+ // First upsert on third file slice.
+ newCommitTime = HoodieActiveTimeline.createNewInstantTime();
+ updateBatch(config, client, newCommitTime, prevCommitTime,
+ Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert,
+ false, false, 50, 10, 2, config.populateMetaFields());
+ prevCommitTime = newCommitTime;
+ if (i == 2) {
+ // Since retain commits is 4 exactly after 6th completed commit there will be some files to be cleaned,
+ // since a version older than the earliest commit is also retained.
+ HoodieInstant cleanInstant = metaClient.reloadActiveTimeline().lastInstant().get();
+ assertEquals(HoodieTimeline.CLEAN_ACTION, cleanInstant.getAction());
+ } else {
+ // Make sure clean is never triggered for other commits. The cleaner is blocked due to pending compaction instant.
+ assertEquals(HoodieTimeline.DELTA_COMMIT_ACTION, metaClient.reloadActiveTimeline().lastInstant().get().getAction());
+ }
+ }
+ }
+
+ @Test
+ public void testRollbackOnLogCompaction() throws Exception {
+ HoodieCompactionConfig compactionConfig = HoodieCompactionConfig.newBuilder()
+ .withLogCompactionBlocksThreshold("1")
+ .build();
+ HoodieWriteConfig lcConfig = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA, HoodieIndex.IndexType.INMEMORY)
+ .withAutoCommit(false).withCompactionConfig(compactionConfig).build();
+ SparkRDDWriteClient lcClient = new SparkRDDWriteClient(context, lcConfig);
+ HoodieWriteConfig config = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA, HoodieIndex.IndexType.INMEMORY)
+ .withAutoCommit(true).build();
+ SparkRDDWriteClient client = new SparkRDDWriteClient(context, config);
+
+ // First insert
+ String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
+ insertBatch(config, client, newCommitTime, "000", 100,
+ SparkRDDWriteClient::insert, false, false, 10, 100,
+ 1, Option.empty());
+ String prevCommitTime = newCommitTime;
+
+ // Upsert
+ newCommitTime = HoodieActiveTimeline.createNewInstantTime();
+ updateBatch(config, client, newCommitTime, prevCommitTime,
+ Option.of(Arrays.asList(prevCommitTime)), "000", 10, SparkRDDWriteClient::upsert,
+ false, false, 10, 10, 4, config.populateMetaFields());
+ prevCommitTime = newCommitTime;
+
+ // Schedule and execute logcompaction but do not commit.
+ Option logCompactionTimeStamp = lcClient.scheduleLogCompaction(Option.empty());
+ assertTrue(logCompactionTimeStamp.isPresent());
+ lcClient.logCompact(logCompactionTimeStamp.get());
+
+ // Rollback the log compaction commit.
+ HoodieInstant instant = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.LOG_COMPACTION_ACTION, logCompactionTimeStamp.get());
+ getHoodieTable(metaClient, config).rollbackInflightLogCompaction(instant);
+
+ // Validate timeline.
+ HoodieTimeline activeTimeline = metaClient.reloadActiveTimeline();
+ HoodieInstant rollbackInstant = activeTimeline.lastInstant().get();
+ assertEquals(3, activeTimeline.countInstants());
+ assertEquals(HoodieTimeline.ROLLBACK_ACTION, rollbackInstant.getAction());
+
+ // Validate block instant times.
+ validateBlockInstantsBeforeAndAfterRollback(config, prevCommitTime, rollbackInstant.getTimestamp());
+ prevCommitTime = rollbackInstant.getTimestamp();
+
+ // Do one more upsert
+ newCommitTime = HoodieActiveTimeline.createNewInstantTime();
+ updateBatch(config, client, newCommitTime, prevCommitTime,
+ Option.of(Arrays.asList(prevCommitTime)), "000", 10, SparkRDDWriteClient::upsert,
+ false, false, 10, 10, 4, config.populateMetaFields());
+ prevCommitTime = newCommitTime;
+
+ // Complete logcompaction now.
+ logCompactionTimeStamp = lcClient.scheduleLogCompaction(Option.empty());
+ assertTrue(logCompactionTimeStamp.isPresent());
+ HoodieWriteMetadata metadata = lcClient.logCompact(logCompactionTimeStamp.get());
+ lcClient.commitLogCompaction(logCompactionTimeStamp.get(), (HoodieCommitMetadata) metadata.getCommitMetadata().get(), Option.empty());
+ assertDataInMORTable(config, prevCommitTime, logCompactionTimeStamp.get(), hadoopConf, Arrays.asList(dataGen.getPartitionPaths()));
+ }
+
+ private void validateBlockInstantsBeforeAndAfterRollback(HoodieWriteConfig config, String instant, String currentInstant) {
+ HoodieTable table = getHoodieTable(metaClient, config);
+ SyncableFileSystemView fileSystemView = (SyncableFileSystemView) table.getSliceView();
+ List partitionPaths = Stream.of(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS).collect(Collectors.toList());
+ for (String partitionPath: partitionPaths) {
+ fileSystemView.getLatestFileSlices(partitionPath).forEach(slice -> {
+ HoodieUnMergedLogRecordScanner scanner = HoodieUnMergedLogRecordScanner.newBuilder()
+ .withFileSystem(metaClient.getFs())
+ .withBasePath(table.getMetaClient().getBasePath())
+ .withLogFilePaths(slice.getLogFiles()
+ .sorted(HoodieLogFile.getLogFileComparator())
+ .map(file -> file.getPath().toString())
+ .collect(Collectors.toList()))
+ .withLatestInstantTime(instant)
+ .withBufferSize(config.getMaxDFSStreamBufferSize())
+ .withUseScanV2(true)
+ .build();
+ scanner.scanInternal(Option.empty(), true);
+ List prevInstants = scanner.getValidBlockInstants();
+ HoodieUnMergedLogRecordScanner scanner2 = HoodieUnMergedLogRecordScanner.newBuilder()
+ .withFileSystem(metaClient.getFs())
+ .withBasePath(table.getMetaClient().getBasePath())
+ .withLogFilePaths(slice.getLogFiles()
+ .sorted(HoodieLogFile.getLogFileComparator())
+ .map(file -> file.getPath().toString())
+ .collect(Collectors.toList()))
+ .withLatestInstantTime(currentInstant)
+ .withBufferSize(config.getMaxDFSStreamBufferSize())
+ .withUseScanV2(true)
+ .build();
+ scanner2.scanInternal(Option.empty(), true);
+ List currentInstants = scanner2.getValidBlockInstants();
+ assertEquals(prevInstants, currentInstants);
+ });
+ }
+ }
+
+ @Test
+ public void testArchivalOnLogCompaction() throws Exception {
+ HoodieCompactionConfig logCompactionConfig = HoodieCompactionConfig.newBuilder()
+ .withLogCompactionBlocksThreshold("2")
+ .build();
+ HoodieWriteConfig lcWriteConfig = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA,
+ HoodieIndex.IndexType.INMEMORY).withAutoCommit(true).withCompactionConfig(logCompactionConfig).build();
+ SparkRDDWriteClient lcWriteClient = new SparkRDDWriteClient(context, lcWriteConfig);
+
+ HoodieCompactionConfig compactionConfig = HoodieCompactionConfig.newBuilder()
+ .withMaxNumDeltaCommitsBeforeCompaction(1)
+ .build();
+ HoodieWriteConfig config = getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA, HoodieIndex.IndexType.INMEMORY)
+ .withAutoCommit(true).withCompactionConfig(compactionConfig)
+ .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(2).build())
+ .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(3, 4).build())
+ .withMetadataConfig(HoodieMetadataConfig.newBuilder().withMaxNumDeltaCommitsBeforeCompaction(2).build())
+ .build();
+ SparkRDDWriteClient client = new SparkRDDWriteClient(context, config);
+
+ // First insert
+ String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
+ insertBatch(config, client, newCommitTime, "000", 100,
+ SparkRDDWriteClient::insert, false, false, 10, 100,
+ 1, Option.empty());
+ String prevCommitTime = newCommitTime;
+ List logCompactionInstantTimes = new ArrayList<>();
+
+ for (int i = 0; i < 6; i++) {
+ if (i % 4 == 0) {
+ // Schedule compaction.
+ Option compactionTimeStamp = client.scheduleCompaction(Option.empty());
+ assertTrue(compactionTimeStamp.isPresent());
+ client.compact(compactionTimeStamp.get());
+ prevCommitTime = compactionTimeStamp.get();
+ }
+
+ // Upsert
+ newCommitTime = HoodieActiveTimeline.createNewInstantTime();
+ updateBatch(config, client, newCommitTime, prevCommitTime,
+ Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert,
+ false, false, 50, 10, 0, config.populateMetaFields());
+ // Schedule log compaction.
+ Option logCompactionTimeStamp = lcWriteClient.scheduleLogCompaction(Option.empty());
+ if (logCompactionTimeStamp.isPresent()) {
+ logCompactionInstantTimes.add(logCompactionTimeStamp.get());
+ lcWriteClient.logCompact(logCompactionTimeStamp.get());
+ prevCommitTime = logCompactionTimeStamp.get();
+ }
+ }
+ boolean logCompactionInstantArchived = false;
+ Map> instantsMap = metaClient.getArchivedTimeline().getInstants()
+ .collect(Collectors.groupingBy(HoodieInstant::getTimestamp));
+ for (String logCompactionTimeStamp: logCompactionInstantTimes) {
+ List instants = instantsMap.get(logCompactionTimeStamp);
+ if (instants == null) {
+ continue;
+ }
+ assertEquals(3, instants.size());
+ for (HoodieInstant instant: instants) {
+ if (instant.isCompleted()) {
+ assertEquals(HoodieTimeline.DELTA_COMMIT_ACTION, instant.getAction());
+ } else {
+ assertEquals(HoodieTimeline.LOG_COMPACTION_ACTION, instant.getAction());
+ }
+ }
+ logCompactionInstantArchived = true;
+ }
+ assertTrue(logCompactionInstantArchived);
+ }
+
+ @Override
+ protected HoodieTableType getTableType() {
+ return HoodieTableType.MERGE_ON_READ;
+ }
+
+}
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java
index 18f764c1fa25f..5ba0f33f9bc53 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java
@@ -18,9 +18,11 @@
package org.apache.hudi.table;
+import org.apache.hudi.client.HoodieReadClient;
import org.apache.hudi.client.SparkRDDReadClient;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieCommitMetadata;
@@ -39,6 +41,7 @@
import org.apache.hudi.common.util.CollectionUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieClusteringConfig;
+import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.data.HoodieJavaRDD;
import org.apache.hudi.index.HoodieIndex;
@@ -80,6 +83,7 @@
import static org.apache.hudi.testutils.HoodieClientTestHarness.buildProfile;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
public class TestHoodieMergeOnReadTable extends SparkClientFunctionalTestHarness {
@@ -300,6 +304,98 @@ public void testLogFileCountsAfterCompaction(boolean preserveCommitMeta) throws
}
}
+ @ParameterizedTest
+ @ValueSource(booleans = {true})
+ public void testLogBlocksCountsAfterLogCompaction(boolean populateMetaFields) throws Exception {
+
+ HoodieCompactionConfig compactionConfig = HoodieCompactionConfig.newBuilder()
+ .withInlineCompaction(false)
+ .withLogCompactionBlocksThreshold("1")
+ .build();
+ // insert 100 recordsx
+ HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(true)
+ .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build())
+ .withCompactionConfig(compactionConfig);
+ addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
+ HoodieWriteConfig config = cfgBuilder.build();
+ setUp(config.getProps());
+
+ try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config)) {
+ String newCommitTime = "100";
+ writeClient.startCommitWithTime(newCommitTime);
+
+ List records = dataGen.generateInserts(newCommitTime, 100);
+ JavaRDD recordsRDD = jsc().parallelize(records, 1);
+ writeClient.insert(recordsRDD, newCommitTime).collect();
+
+ // Update all the 100 records
+ newCommitTime = "101";
+ List updatedRecords = dataGen.generateUpdates(newCommitTime, records);
+ JavaRDD updatedRecordsRDD = jsc().parallelize(updatedRecords, 1);
+
+ HoodieReadClient readClient = new HoodieReadClient(context(), config);
+ JavaRDD updatedTaggedRecordsRDD = readClient.tagLocation(updatedRecordsRDD);
+
+ writeClient.startCommitWithTime(newCommitTime);
+ writeClient.upsertPreppedRecords(updatedTaggedRecordsRDD, newCommitTime).collect();
+
+
+ newCommitTime = "102";
+ writeClient.startCommitWithTime(newCommitTime);
+ writeClient.upsertPreppedRecords(updatedTaggedRecordsRDD, newCommitTime).collect();
+
+
+ // Write them to corresponding avro logfiles
+ metaClient = HoodieTableMetaClient.reload(metaClient);
+
+ HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(
+ writeClient.getEngineContext().getHadoopConf().get(), config, writeClient.getEngineContext());
+ HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable
+ .of(metaClient, HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS, metadataWriter);
+
+ Set allPartitions = updatedRecords.stream()
+ .map(record -> record.getPartitionPath())
+ .collect(Collectors.groupingBy(partitionPath -> partitionPath))
+ .keySet();
+ assertEquals(allPartitions.size(), testTable.listAllBaseFiles().length);
+
+ // Verify that all data file has one log file
+ HoodieTable table = HoodieSparkTable.create(config, context(), metaClient);
+ for (String partitionPath : dataGen.getPartitionPaths()) {
+ List groupedLogFiles =
+ table.getSliceView().getLatestFileSlices(partitionPath).collect(Collectors.toList());
+ for (FileSlice fileSlice : groupedLogFiles) {
+ assertEquals(2, fileSlice.getLogFiles().count(),
+ "There should be 1 log file written for the latest data file - " + fileSlice);
+ }
+ }
+
+ // Do a log compaction
+ String logCompactionInstantTime = writeClient.scheduleLogCompaction(Option.empty()).get().toString();
+ HoodieWriteMetadata> result = writeClient.logCompact(logCompactionInstantTime);
+
+ // Verify that recently written compacted data file has no log file
+ metaClient = HoodieTableMetaClient.reload(metaClient);
+ table = HoodieSparkTable.create(config, context(), metaClient);
+ HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
+
+ assertTrue(HoodieTimeline
+ .compareTimestamps(timeline.lastInstant().get().getTimestamp(), HoodieTimeline.GREATER_THAN, newCommitTime),
+ "Compaction commit should be > than last insert");
+
+ for (String partitionPath : dataGen.getPartitionPaths()) {
+ List fileSlices =
+ table.getSliceView().getLatestFileSlices(partitionPath).collect(Collectors.toList());
+ assertEquals(1, fileSlices.size());
+ for (FileSlice slice : fileSlices) {
+ assertEquals(3, slice.getLogFiles().count(), "After compaction there will still be one log file.");
+ assertNotNull(slice.getBaseFile(), "Base file is not created by log compaction operation.");
+ }
+ assertTrue(result.getCommitMetadata().get().getWritePartitionPaths().stream().anyMatch(part -> part.contentEquals(partitionPath)));
+ }
+ }
+ }
+
/**
* Test to ensure metadata stats are correctly written to metadata file.
*/
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/GenericRecordValidationTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/GenericRecordValidationTestUtils.java
new file mode 100644
index 0000000000000..36ef74b60558a
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/GenericRecordValidationTestUtils.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.testutils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.ArrayWritable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
+import org.apache.hudi.common.util.CollectionUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieValidationException;
+import org.apache.hudi.hadoop.config.HoodieRealtimeConfig;
+import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils;
+
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.common.model.HoodieRecord.COMMIT_SEQNO_METADATA_FIELD;
+import static org.apache.hudi.common.model.HoodieRecord.COMMIT_TIME_METADATA_FIELD;
+import static org.apache.hudi.common.model.HoodieRecord.FILENAME_METADATA_FIELD;
+import static org.apache.hudi.common.model.HoodieRecord.OPERATION_METADATA_FIELD;
+import static org.apache.hudi.common.model.HoodieRecord.RECORD_KEY_METADATA_FIELD;
+import static org.apache.hudi.hadoop.utils.HoodieHiveUtils.HOODIE_CONSUME_COMMIT;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class GenericRecordValidationTestUtils {
+
+ public static void assertGenericRecords(GenericRecord record1, GenericRecord record2,
+ Schema schema, List excludeFields) {
+ for (Schema.Field f: schema.getFields()) {
+ String fieldName = f.name();
+ if (excludeFields.contains(fieldName)) {
+ continue;
+ }
+ Object value1 = record1.get(fieldName);
+ Object value2 = record2.get(fieldName);
+ if (value1 != null && value2 != null) {
+ if (value1 instanceof ArrayWritable) {
+ assertEquals(HoodieRealtimeRecordReaderUtils.arrayWritableToString((ArrayWritable) value1),
+ HoodieRealtimeRecordReaderUtils.arrayWritableToString((ArrayWritable) value2));
+ } else {
+ assertEquals(value1, value2, "Field name " + fieldName + " is not same."
+ + " Val1: " + value1 + ", Val2:" + value2);
+ }
+ } else if (value1 != null || value2 != null) {
+ throw new HoodieValidationException("Field name " + fieldName + " is not same."
+ + " Val1: " + value1 + ", Val2:" + value2);
+ }
+ }
+ }
+
+ public static void assertDataInMORTable(HoodieWriteConfig config, String instant1, String instant2,
+ Configuration hadoopConf, List partitionPaths) {
+ List excludeFields = CollectionUtils.createImmutableList(COMMIT_TIME_METADATA_FIELD, COMMIT_SEQNO_METADATA_FIELD,
+ FILENAME_METADATA_FIELD, OPERATION_METADATA_FIELD);
+ assertDataInMORTable(config, instant1, instant2, hadoopConf, partitionPaths, excludeFields);
+ }
+
+ public static void assertDataInMORTable(HoodieWriteConfig config, String instant1, String instant2,
+ Configuration hadoopConf, List partitionPaths, List excludeFields) {
+ JobConf jobConf = new JobConf(hadoopConf);
+ List fullPartitionPaths = partitionPaths.stream()
+ .map(partitionPath -> Paths.get(config.getBasePath(), partitionPath).toString())
+ .collect(Collectors.toList());
+
+ jobConf.set(String.format(HOODIE_CONSUME_COMMIT, config.getTableName()), instant1);
+ jobConf.set(HoodieRealtimeConfig.USE_LOG_RECORD_READER_SCAN_V2, "true");
+ List records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
+ hadoopConf, fullPartitionPaths, config.getBasePath(), jobConf, true);
+ Map prevRecordsMap = records.stream()
+ .collect(Collectors.toMap(rec -> rec.get(RECORD_KEY_METADATA_FIELD).toString(), Function.identity()));
+
+ jobConf.set(String.format(HOODIE_CONSUME_COMMIT, config.getTableName()), instant2);
+ List records1 = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
+ hadoopConf, fullPartitionPaths, config.getBasePath(), jobConf, true);
+ Map newRecordsMap = records1.stream()
+ .collect(Collectors.toMap(rec -> rec.get(RECORD_KEY_METADATA_FIELD).toString(), Function.identity()));
+
+ // Verify row count.
+ assertEquals(prevRecordsMap.size(), newRecordsMap.size());
+
+ Schema readerSchema = HoodieAvroUtils.addMetadataFields(
+ new Schema.Parser().parse(config.getSchema()), config.allowOperationMetadataField());
+
+ // Verify every field.
+ prevRecordsMap.forEach((key, value) -> {
+ assertTrue(newRecordsMap.containsKey(key));
+ assertGenericRecords(value, newRecordsMap.get(key), readerSchema, excludeFields);
+ });
+ }
+
+ public static Map getRecordsMap(HoodieWriteConfig config, Configuration hadoopConf,
+ HoodieTestDataGenerator dataGen) {
+ JobConf jobConf = new JobConf(hadoopConf);
+ List fullPartitionPaths = Arrays.stream(dataGen.getPartitionPaths())
+ .map(partitionPath -> Paths.get(config.getBasePath(), partitionPath).toString())
+ .collect(Collectors.toList());
+ return HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
+ hadoopConf, fullPartitionPaths, config.getBasePath(), jobConf, true).stream()
+ .collect(Collectors.toMap(rec -> rec.get(RECORD_KEY_METADATA_FIELD).toString(), Function.identity()));
+ }
+
+}
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java
index 900674a677588..424bb6c53e37e 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java
@@ -65,6 +65,7 @@
import java.util.function.Function;
import java.util.stream.Collectors;
+import static org.apache.hudi.common.testutils.HoodieTestUtils.RAW_TRIPS_TEST_NAME;
import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
@@ -150,7 +151,7 @@ public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr, IndexType in
.withCleanConfig(HoodieCleanConfig.newBuilder().withFailedWritesCleaningPolicy(cleaningPolicy).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).orcMaxFileSize(1024 * 1024).build())
- .forTable("test-trip-table")
+ .forTable(RAW_TRIPS_TEST_NAME)
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build())
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
.withEnableBackupForRemoteFileSystemView(false) // Fail test if problem connecting to timeline-server
diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml
index ae5303cc70726..87b8e5e0beea3 100644
--- a/hudi-common/pom.xml
+++ b/hudi-common/pom.xml
@@ -68,6 +68,7 @@
${basedir}/src/main/avro/HoodieCommitMetadata.avsc
+ ${basedir}/src/main/avro/HoodieCompactionStrategy.avsc
${basedir}/src/main/avro/HoodieCompactionOperation.avsc
${basedir}/src/main/avro/HoodieSavePointMetadata.avsc
${basedir}/src/main/avro/HoodieCompactionMetadata.avsc
diff --git a/hudi-common/src/main/avro/HoodieCompactionOperation.avsc b/hudi-common/src/main/avro/HoodieCompactionOperation.avsc
index 2095a9518c53e..bab7321f29cae 100644
--- a/hudi-common/src/main/avro/HoodieCompactionOperation.avsc
+++ b/hudi-common/src/main/avro/HoodieCompactionOperation.avsc
@@ -84,6 +84,18 @@
"name":"version",
"type":["int", "null"],
"default": 1
+ },
+ {
+ "name":"strategy",
+ "type":[
+ "null", "HoodieCompactionStrategy"
+ ],
+ "default": null
+ },
+ {
+ "name":"preserveHoodieMetadata",
+ "type":["boolean", "null"],
+ "default": false
}
]
}
diff --git a/hudi-common/src/main/avro/HoodieCompactionStrategy.avsc b/hudi-common/src/main/avro/HoodieCompactionStrategy.avsc
new file mode 100644
index 0000000000000..eff500d1f5f18
--- /dev/null
+++ b/hudi-common/src/main/avro/HoodieCompactionStrategy.avsc
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+{
+ "namespace": "org.apache.hudi.avro.model",
+ "name": "HoodieCompactionStrategy",
+ "type": "record",
+ "fields": [
+ {
+ "name":"compactorClassName",
+ "doc": "The class name that is provided here should extend CompactionExecutionStrategy abstract class, idea is that the plan generator will also provide the execution strategy to use.",
+ "type":["null","string"],
+ "default": null
+ },
+ {
+ "name":"strategyParams",
+ "doc": "These parameters play a key role in determining which action type it is i.e. compaction or logcompaction and it will also include configs that determine how compaction needs to be executed.",
+ "type":["null", {
+ "type":"map",
+ "values":"string"
+ }],
+ "default": null
+ },
+ {
+ "name":"version",
+ "type":["int", "null"],
+ "default": 1
+ }
+ ]
+}
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java
index b16373ef83436..793d79e256aff 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java
@@ -234,6 +234,13 @@ public final class HoodieMetadataConfig extends HoodieConfig {
+ "metadata table which are never added before. This config determines how to handle "
+ "such spurious deletes");
+ public static final ConfigProperty USE_LOG_RECORD_READER_SCAN_V2 = ConfigProperty
+ .key(METADATA_PREFIX + ".log.record.reader.use.scanV2")
+ .defaultValue(false)
+ .sinceVersion("0.13.0")
+ .withDocumentation("ScanV2 logic address all the multiwriter challenges while appending to log files. "
+ + "It also differentiates original blocks written by ingestion writers and compacted blocks written log compaction.");
+
private HoodieMetadataConfig() {
super();
}
@@ -318,6 +325,10 @@ public boolean ignoreSpuriousDeletes() {
return getBoolean(IGNORE_SPURIOUS_DELETES);
}
+ public boolean getUseLogRecordReaderScanV2() {
+ return getBoolean(USE_LOG_RECORD_READER_SCAN_V2);
+ }
+
public static class Builder {
private EngineType engineType = EngineType.SPARK;
@@ -461,6 +472,11 @@ public Builder withProperties(Properties properties) {
return this;
}
+ public Builder withLogRecordReaderScanV2(boolean useLogRecordReaderScanV2) {
+ metadataConfig.setValue(USE_LOG_RECORD_READER_SCAN_V2, String.valueOf(useLogRecordReaderScanV2));
+ return this;
+ }
+
public HoodieMetadataConfig build() {
metadataConfig.setDefaultValue(ENABLE, getDefaultMetadataEnable(engineType));
metadataConfig.setDefaults(HoodieMetadataConfig.class.getName());
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/ActionType.java b/hudi-common/src/main/java/org/apache/hudi/common/model/ActionType.java
index c10c99d8dc8ee..eb7e578522ee8 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/model/ActionType.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/model/ActionType.java
@@ -22,5 +22,5 @@
* The supported action types.
*/
public enum ActionType {
- commit, savepoint, compaction, clean, rollback, replacecommit, deltacommit
+ commit, savepoint, compaction, clean, rollback, replacecommit, deltacommit, logcompaction
}
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/TableServiceType.java b/hudi-common/src/main/java/org/apache/hudi/common/model/TableServiceType.java
index 69dd30782ff77..2aa5d08f06a8e 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/model/TableServiceType.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/model/TableServiceType.java
@@ -24,7 +24,7 @@
* Supported runtime table services.
*/
public enum TableServiceType {
- ARCHIVE, COMPACT, CLUSTER, CLEAN;
+ ARCHIVE, COMPACT, CLUSTER, CLEAN, LOG_COMPACT;
public String getAction() {
switch (this) {
@@ -37,6 +37,8 @@ public String getAction() {
return HoodieTimeline.CLEAN_ACTION;
case CLUSTER:
return HoodieTimeline.REPLACE_COMMIT_ACTION;
+ case LOG_COMPACT:
+ return HoodieTimeline.LOG_COMPACTION_ACTION;
default:
throw new IllegalArgumentException("Unknown table service " + this);
}
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java
index f2f3809cf5c3a..69867f71ebd8f 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java
@@ -53,7 +53,8 @@ public enum WriteOperationType {
// alter schema
ALTER_SCHEMA("alter_schema"),
-
+ // log compact
+ LOG_COMPACT("logcompact"),
// used for old version
UNKNOWN("unknown");
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java
index 5bfb395dbc54c..88da6aa1f0669 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java
@@ -57,17 +57,23 @@
import java.io.IOException;
import java.util.ArrayDeque;
+import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Deque;
+import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Properties;
+import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;
+import static org.apache.hudi.common.table.log.block.HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_BLOCK;
+import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType.COMPACTED_BLOCK_TIMES;
import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME;
+import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME;
import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.COMMAND_BLOCK;
import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK;
@@ -140,6 +146,10 @@ public abstract class AbstractHoodieLogRecordReader {
private Option partitionName;
// Populate meta fields for the records
private boolean populateMetaFields = true;
+ // Collect all the block instants after scanning all the log files.
+ private List validBlockInstants = new ArrayList<>();
+ // Use scanV2 method.
+ private boolean useScanV2;
protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List logFilePaths,
Schema readerSchema,
@@ -147,14 +157,14 @@ protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List instantRange,
boolean withOperationField) {
this(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize,
- instantRange, withOperationField, true, Option.empty(), InternalSchema.getEmptyInternalSchema());
+ instantRange, withOperationField, true, Option.empty(), InternalSchema.getEmptyInternalSchema(), false);
}
protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List logFilePaths,
Schema readerSchema, String latestInstantTime, boolean readBlocksLazily,
boolean reverseReader, int bufferSize, Option instantRange,
boolean withOperationField, boolean forceFullScan,
- Option partitionName, InternalSchema internalSchema) {
+ Option partitionName, InternalSchema internalSchema, boolean useScanV2) {
this.readerSchema = readerSchema;
this.latestInstantTime = latestInstantTime;
this.hoodieTableMetaClient = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(basePath).build();
@@ -176,6 +186,7 @@ protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List keys) {
- scanInternal(Option.of(new KeySpec(keys, true)));
+ scanInternal(Option.of(new KeySpec(keys, true)), false);
}
- protected synchronized void scanInternal(Option keySpecOpt) {
+ public synchronized void scanInternal(Option keySpecOpt, boolean skipProcessingBlocks) {
+ if (useScanV2) {
+ scanInternalV2(keySpecOpt, skipProcessingBlocks);
+ } else {
+ scanInternal(keySpecOpt);
+ }
+ }
+
+ private synchronized void scanInternal(Option keySpecOpt) {
currentInstantLogBlocks = new ArrayDeque<>();
progress = 0.0f;
totalLogFiles = new AtomicLong(0);
@@ -295,7 +314,7 @@ protected synchronized void scanInternal(Option keySpecOpt) {
String targetInstantForCommandBlock =
logBlock.getLogBlockHeader().get(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME);
switch (commandBlock.getType()) { // there can be different types of command blocks
- case ROLLBACK_PREVIOUS_BLOCK:
+ case ROLLBACK_BLOCK:
// Rollback the last read log block
// Get commit time from last record block, compare with targetCommitTime,
// rollback only if equal, this is required in scenarios of invalid/extra
@@ -368,6 +387,232 @@ protected synchronized void scanInternal(Option keySpecOpt) {
}
}
+ private synchronized void scanInternalV2(Option keySpecOption, boolean skipProcessingBlocks) {
+ currentInstantLogBlocks = new ArrayDeque<>();
+ progress = 0.0f;
+ totalLogFiles = new AtomicLong(0);
+ totalRollbacks = new AtomicLong(0);
+ totalCorruptBlocks = new AtomicLong(0);
+ totalLogBlocks = new AtomicLong(0);
+ totalLogRecords = new AtomicLong(0);
+ HoodieLogFormatReader logFormatReaderWrapper = null;
+ HoodieTimeline commitsTimeline = this.hoodieTableMetaClient.getCommitsTimeline();
+ HoodieTimeline completedInstantsTimeline = commitsTimeline.filterCompletedInstants();
+ HoodieTimeline inflightInstantsTimeline = commitsTimeline.filterInflights();
+ try {
+
+ // Get the key field based on populate meta fields config
+ // and the table type
+ final String keyField = getKeyField();
+
+ boolean enableRecordLookups = !forceFullScan;
+ // Iterate over the paths
+ logFormatReaderWrapper = new HoodieLogFormatReader(fs,
+ logFilePaths.stream().map(logFile -> new HoodieLogFile(new Path(logFile))).collect(Collectors.toList()),
+ readerSchema, readBlocksLazily, reverseReader, bufferSize, enableRecordLookups, keyField, internalSchema);
+
+ /**
+ * Scanning log blocks and placing the compacted blocks at the right place require two traversals.
+ * First traversal to identify the rollback blocks and valid data and compacted blocks.
+ *
+ * Scanning blocks is easy to do in single writer mode, where the rollback block is right after the effected data blocks.
+ * With multiwriter mode the blocks can be out of sync. An example scenario.
+ * B1, B2, B3, B4, R1(B3), B5
+ * In this case, rollback block R1 is invalidating the B3 which is not the previous block.
+ * This becomes more complicated if we have compacted blocks, which are data blocks created using log compaction.
+ *
+ * To solve this, run a single traversal, collect all the valid blocks that are not corrupted
+ * along with the block instant times and rollback block's target instant times.
+ *
+ * As part of second traversal iterate block instant times in reverse order.
+ * While iterating in reverse order keep a track of final compacted instant times for each block.
+ * In doing so, when a data block is seen include the final compacted block if it is not already added.
+ *
+ * find the final compacted block which contains the merged contents.
+ * For example B1 and B2 are merged and created a compacted block called M1 and now M1, B3 and B4 are merged and
+ * created another compacted block called M2. So, now M2 is the final block which contains all the changes of B1,B2,B3,B4.
+ * So, blockTimeToCompactionBlockTimeMap will look like
+ * (B1 -> M2), (B2 -> M2), (B3 -> M2), (B4 -> M2), (M1 -> M2)
+ * This map is updated while iterating and is used to place the compacted blocks in the correct position.
+ * This way we can have multiple layers of merge blocks and still be able to find the correct positions of merged blocks.
+ */
+
+ // Collect targetRollbackInstants, using which we can determine which blocks are invalid.
+ Set targetRollbackInstants = new HashSet<>();
+
+ // This holds block instant time to list of blocks. Note here the log blocks can be normal data blocks or compacted log blocks.
+ Map> instantToBlocksMap = new HashMap<>();
+
+ // Order of Instants.
+ List orderedInstantsList = new ArrayList<>();
+
+ Set scannedLogFiles = new HashSet<>();
+
+ /*
+ * 1. First step to traverse in forward direction. While traversing the log blocks collect following,
+ * a. instant times
+ * b. instant to logblocks map.
+ * c. targetRollbackInstants.
+ */
+ while (logFormatReaderWrapper.hasNext()) {
+ HoodieLogFile logFile = logFormatReaderWrapper.getLogFile();
+ LOG.info("Scanning log file " + logFile);
+ scannedLogFiles.add(logFile);
+ totalLogFiles.set(scannedLogFiles.size());
+ // Use the HoodieLogFileReader to iterate through the blocks in the log file
+ HoodieLogBlock logBlock = logFormatReaderWrapper.next();
+ final String instantTime = logBlock.getLogBlockHeader().get(INSTANT_TIME);
+ totalLogBlocks.incrementAndGet();
+ // Ignore the corrupt blocks. No further handling is required for them.
+ if (logBlock.getBlockType().equals(CORRUPT_BLOCK)) {
+ LOG.info("Found a corrupt block in " + logFile.getPath());
+ totalCorruptBlocks.incrementAndGet();
+ continue;
+ }
+ if (!HoodieTimeline.compareTimestamps(logBlock.getLogBlockHeader().get(INSTANT_TIME),
+ HoodieTimeline.LESSER_THAN_OR_EQUALS, this.latestInstantTime)) {
+ // hit a block with instant time greater than should be processed, stop processing further
+ break;
+ }
+ if (logBlock.getBlockType() != COMMAND_BLOCK) {
+ if (!completedInstantsTimeline.containsOrBeforeTimelineStarts(instantTime)
+ || inflightInstantsTimeline.containsInstant(instantTime)) {
+ // hit an uncommitted block possibly from a failed write, move to the next one and skip processing this one
+ continue;
+ }
+ if (instantRange.isPresent() && !instantRange.get().isInRange(instantTime)) {
+ // filter the log block by instant range
+ continue;
+ }
+ }
+
+ switch (logBlock.getBlockType()) {
+ case HFILE_DATA_BLOCK:
+ case AVRO_DATA_BLOCK:
+ case DELETE_BLOCK:
+ List logBlocksList = instantToBlocksMap.getOrDefault(instantTime, new ArrayList<>());
+ if (logBlocksList.size() == 0) {
+ // Keep a track of instant Times in the order of arrival.
+ orderedInstantsList.add(instantTime);
+ }
+ logBlocksList.add(logBlock);
+ instantToBlocksMap.put(instantTime, logBlocksList);
+ break;
+ case COMMAND_BLOCK:
+ LOG.info("Reading a command block from file " + logFile.getPath());
+ // This is a command block - take appropriate action based on the command
+ HoodieCommandBlock commandBlock = (HoodieCommandBlock) logBlock;
+
+ // Rollback blocks contain information of instants that are failed, collect them in a set..
+ if (commandBlock.getType().equals(ROLLBACK_BLOCK)) {
+ totalRollbacks.incrementAndGet();
+ String targetInstantForCommandBlock =
+ logBlock.getLogBlockHeader().get(TARGET_INSTANT_TIME);
+ targetRollbackInstants.add(targetInstantForCommandBlock);
+ } else {
+ throw new UnsupportedOperationException("Command type not yet supported.");
+ }
+ break;
+ default:
+ throw new UnsupportedOperationException("Block type not yet supported.");
+ }
+ }
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Ordered instant times seen " + orderedInstantsList);
+ }
+
+ int numBlocksRolledBack = 0;
+
+ // All the block's instants time that are added to the queue are collected in this set.
+ Set instantTimesIncluded = new HashSet<>();
+
+ // Key will have details related to instant time and value will be empty if that instant is not compacted.
+ // Ex: B1(i1), B2(i2), CB(i3,[i1,i2]) entries will be like i1 -> i3, i2 -> i3.
+ Map blockTimeToCompactionBlockTimeMap = new HashMap<>();
+
+ /*
+ * 2. Iterate the instants list in reverse order to get the latest instants first.
+ * While iterating update the blockTimeToCompactionBlockTimesMap and include the compacted blocks in right position.
+ */
+ for (int i = orderedInstantsList.size() - 1; i >= 0; i--) {
+ String instantTime = orderedInstantsList.get(i);
+
+ // Exclude the blocks which are included in targetRollbackInstants set.
+ // Here, rollback can include instants affiliated to deltacommits or log compaction commits.
+ if (targetRollbackInstants.contains(instantTime)) {
+ numBlocksRolledBack += instantToBlocksMap.get(instantTime).size();
+ continue;
+ }
+ List instantsBlocks = instantToBlocksMap.get(instantTime);
+ if (instantsBlocks.size() == 0) {
+ throw new HoodieException("Data corrupted while writing. Found zero blocks for an instant " + instantTime);
+ }
+ HoodieLogBlock firstBlock = instantsBlocks.get(0);
+
+ // For compacted blocks COMPACTED_BLOCK_TIMES entry is present under its headers.
+ if (firstBlock.getLogBlockHeader().containsKey(COMPACTED_BLOCK_TIMES)) {
+ // When compacted blocks are seen update the blockTimeToCompactionBlockTimeMap.
+ Arrays.stream(firstBlock.getLogBlockHeader().get(COMPACTED_BLOCK_TIMES).split(","))
+ .forEach(originalInstant -> {
+ String finalInstant = blockTimeToCompactionBlockTimeMap.getOrDefault(instantTime, instantTime);
+ blockTimeToCompactionBlockTimeMap.put(originalInstant, finalInstant);
+ });
+ } else {
+ // When a data block is found check if it is already compacted.
+ String compactedFinalInstantTime = blockTimeToCompactionBlockTimeMap.get(instantTime);
+ if (compactedFinalInstantTime == null) {
+ // If it is not compacted then add the blocks related to the instant time at the end of the queue and continue.
+ List logBlocks = instantToBlocksMap.get(instantTime);
+ Collections.reverse(logBlocks);
+ logBlocks.forEach(block -> currentInstantLogBlocks.addLast(block));
+ instantTimesIncluded.add(instantTime);
+ validBlockInstants.add(instantTime);
+ continue;
+ }
+ // If the compacted block exists and it is already included in the dequeue then ignore and continue.
+ if (instantTimesIncluded.contains(compactedFinalInstantTime)) {
+ continue;
+ }
+ // If the compacted block exists and it is not already added then add all the blocks related to that instant time.
+ List logBlocks = instantToBlocksMap.get(compactedFinalInstantTime);
+ Collections.reverse(logBlocks);
+ logBlocks.forEach(block -> currentInstantLogBlocks.addLast(block));
+ instantTimesIncluded.add(compactedFinalInstantTime);
+ validBlockInstants.add(compactedFinalInstantTime);
+ }
+ }
+ LOG.info("Number of applied rollback blocks " + numBlocksRolledBack);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.info("Final view of the Block time to compactionBlockMap " + blockTimeToCompactionBlockTimeMap);
+ }
+
+ // merge the last read block when all the blocks are done reading
+ if (!currentInstantLogBlocks.isEmpty() && !skipProcessingBlocks) {
+ LOG.info("Merging the final data blocks");
+ processQueuedBlocksForInstant(currentInstantLogBlocks, scannedLogFiles.size(), keySpecOption);
+ }
+ // Done
+ progress = 1.0f;
+ } catch (IOException e) {
+ LOG.error("Got IOException when reading log file", e);
+ throw new HoodieIOException("IOException when reading log file ", e);
+ } catch (Exception e) {
+ LOG.error("Got exception when reading log file", e);
+ throw new HoodieException("Exception when reading log file ", e);
+ } finally {
+ try {
+ if (null != logFormatReaderWrapper) {
+ logFormatReaderWrapper.close();
+ }
+ } catch (IOException ioe) {
+ // Eat exception as we do not want to mask the original exception that can happen
+ LOG.error("Unable to close log format reader", ioe);
+ }
+ }
+ }
+
/**
* Checks if the current logblock belongs to a later instant.
*/
@@ -551,6 +796,14 @@ public KeySpec(List keys, boolean fullKey) {
}
}
+ public Deque getCurrentInstantLogBlocks() {
+ return currentInstantLogBlocks;
+ }
+
+ public List getValidBlockInstants() {
+ return validBlockInstants;
+ }
+
/**
* Builder used to build {@code AbstractHoodieLogRecordScanner}.
*/
@@ -584,6 +837,10 @@ public Builder withOperationField(boolean withOperationField) {
throw new UnsupportedOperationException();
}
+ public Builder withUseScanV2(boolean useScanV2) {
+ throw new UnsupportedOperationException();
+ }
+
public abstract AbstractHoodieLogRecordReader build();
}
}
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java
index d7e725544aa65..756e031cea6c7 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java
@@ -88,10 +88,11 @@ protected HoodieMergedLogRecordScanner(FileSystem fs, String basePath, List partitionName, InternalSchema internalSchema) {
+ Option partitionName, InternalSchema internalSchema,
+ boolean useScanV2) {
super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize,
instantRange, withOperationField,
- forceFullScan, partitionName, internalSchema);
+ forceFullScan, partitionName, internalSchema, useScanV2);
try {
// Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize
this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, new DefaultSizeEstimator(),
@@ -156,7 +157,11 @@ protected void processNextRecord(HoodieRecord extends HoodieRecordPayload> hoo
// If combinedValue is oldValue, no need rePut oldRecord
if (combinedValue != oldValue) {
HoodieOperation operation = hoodieRecord.getOperation();
- records.put(key, new HoodieAvroRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue, operation));
+ HoodieRecord latestHoodieRecord = new HoodieAvroRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue, operation);
+ latestHoodieRecord.unseal();
+ latestHoodieRecord.setCurrentLocation(hoodieRecord.getCurrentLocation());
+ latestHoodieRecord.seal();
+ records.put(key, latestHoodieRecord);
}
} else {
// Put the record as is
@@ -221,8 +226,12 @@ public static class Builder extends AbstractHoodieLogRecordReader.Builder {
// incremental filtering
protected Option instantRange = Option.empty();
protected String partitionName;
+ // auto scan default true
+ private boolean autoScan = true;
// operation field default false
private boolean withOperationField = false;
+ // Use scanV2 method.
+ private boolean useScanV2 = false;
@Override
public Builder withFileSystem(FileSystem fs) {
@@ -316,6 +325,12 @@ public Builder withPartition(String partitionName) {
return this;
}
+ @Override
+ public Builder withUseScanV2(boolean useScanV2) {
+ this.useScanV2 = useScanV2;
+ return this;
+ }
+
@Override
public HoodieMergedLogRecordScanner build() {
if (this.partitionName == null && CollectionUtils.nonEmpty(this.logFilePaths)) {
@@ -325,7 +340,7 @@ public HoodieMergedLogRecordScanner build() {
latestInstantTime, maxMemorySizeInBytes, readBlocksLazily, reverseReader,
bufferSize, spillableMapBasePath, instantRange,
diskMapType, isBitCaskDiskMapCompressionEnabled, withOperationField, true,
- Option.ofNullable(partitionName), internalSchema);
+ Option.ofNullable(partitionName), internalSchema, useScanV2);
}
}
}
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java
index 8ea34d6f2fa0d..7eb87e8e7fdb4 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java
@@ -25,6 +25,7 @@
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
+import org.apache.hudi.internal.schema.InternalSchema;
import java.util.List;
@@ -37,8 +38,9 @@ public class HoodieUnMergedLogRecordScanner extends AbstractHoodieLogRecordReade
private HoodieUnMergedLogRecordScanner(FileSystem fs, String basePath, List logFilePaths, Schema readerSchema,
String latestInstantTime, boolean readBlocksLazily, boolean reverseReader, int bufferSize,
- LogRecordScannerCallback callback, Option instantRange) {
- super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, instantRange, false);
+ LogRecordScannerCallback callback, Option instantRange, boolean useScanV2) {
+ super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, instantRange,
+ false, true, Option.empty(), InternalSchema.getEmptyInternalSchema(), useScanV2);
this.callback = callback;
}
@@ -84,6 +86,7 @@ public static class Builder extends AbstractHoodieLogRecordReader.Builder {
private Option instantRange = Option.empty();
// specific configurations
private LogRecordScannerCallback callback;
+ private boolean useScanV2;
public Builder withFileSystem(FileSystem fs) {
this.fs = fs;
@@ -135,10 +138,16 @@ public Builder withLogRecordScannerCallback(LogRecordScannerCallback callback) {
return this;
}
+ @Override
+ public Builder withUseScanV2(boolean useScanV2) {
+ this.useScanV2 = useScanV2;
+ return this;
+ }
+
@Override
public HoodieUnMergedLogRecordScanner build() {
return new HoodieUnMergedLogRecordScanner(fs, basePath, logFilePaths, readerSchema,
- latestInstantTime, readBlocksLazily, reverseReader, bufferSize, callback, instantRange);
+ latestInstantTime, readBlocksLazily, reverseReader, bufferSize, callback, instantRange, useScanV2);
}
}
}
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieCommandBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieCommandBlock.java
index 0ff3a77b5007b..c44f1950144b5 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieCommandBlock.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieCommandBlock.java
@@ -36,7 +36,7 @@ public class HoodieCommandBlock extends HoodieLogBlock {
* Hoodie command block type enum.
*/
public enum HoodieCommandBlockTypeEnum {
- ROLLBACK_PREVIOUS_BLOCK
+ ROLLBACK_BLOCK
}
public HoodieCommandBlock(Map header) {
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieLogBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieLogBlock.java
index 1718e7dd02457..f8307333d257c 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieLogBlock.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieLogBlock.java
@@ -113,6 +113,14 @@ public Option getContent() {
return content;
}
+ /**
+ * Compacted blocks are created using log compaction which basically merges the consecutive blocks together and create
+ * huge block with all the changes.
+ */
+ public boolean isCompactedLogBlock() {
+ return logBlockHeader.containsKey(HeaderMetadataType.COMPACTED_BLOCK_TIMES);
+ }
+
/**
* Type of the log block WARNING: This enum is serialized as the ordinal. Only add new enums at the end.
*/
@@ -144,7 +152,7 @@ public static HoodieLogBlockType fromId(String id) {
* new enums at the end.
*/
public enum HeaderMetadataType {
- INSTANT_TIME, TARGET_INSTANT_TIME, SCHEMA, COMMAND_BLOCK_TYPE
+ INSTANT_TIME, TARGET_INSTANT_TIME, SCHEMA, COMMAND_BLOCK_TYPE, COMPACTED_BLOCK_TIMES
}
/**
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java
index 2b27d3ab5e568..be2febdff3f57 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java
@@ -70,6 +70,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
CLEAN_EXTENSION, REQUESTED_CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION,
INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION,
REQUESTED_RESTORE_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION,
+ INFLIGHT_LOG_COMPACTION_EXTENSION, REQUESTED_LOG_COMPACTION_EXTENSION,
ROLLBACK_EXTENSION, REQUESTED_ROLLBACK_EXTENSION, INFLIGHT_ROLLBACK_EXTENSION,
REQUESTED_REPLACE_COMMIT_EXTENSION, INFLIGHT_REPLACE_COMMIT_EXTENSION, REPLACE_COMMIT_EXTENSION,
REQUESTED_INDEX_COMMIT_EXTENSION, INFLIGHT_INDEX_COMMIT_EXTENSION, INDEX_COMMIT_EXTENSION,
@@ -225,7 +226,7 @@ public void saveAsComplete(HoodieInstant instant, Option data) {
public HoodieInstant revertToInflight(HoodieInstant instant) {
LOG.info("Reverting instant to inflight " + instant);
- HoodieInstant inflight = HoodieTimeline.getInflightInstant(instant, metaClient.getTableType());
+ HoodieInstant inflight = HoodieTimeline.getInflightInstant(instant, metaClient);
revertCompleteToInflight(instant, inflight);
LOG.info("Reverted " + instant + " to inflight " + inflight);
return inflight;
@@ -406,6 +407,27 @@ public HoodieInstant revertInstantFromInflightToRequested(HoodieInstant inflight
return requestedInstant;
}
+ /**
+ * TODO: This method is not needed, since log compaction plan is not a immutable plan.
+ * Revert logcompaction State from inflight to requested.
+ *
+ * @param inflightInstant Inflight Instant
+ * @return requested instant
+ */
+ public HoodieInstant revertLogCompactionInflightToRequested(HoodieInstant inflightInstant) {
+ ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION));
+ ValidationUtils.checkArgument(inflightInstant.isInflight());
+ HoodieInstant requestedInstant =
+ new HoodieInstant(State.REQUESTED, LOG_COMPACTION_ACTION, inflightInstant.getTimestamp());
+ if (metaClient.getTimelineLayoutVersion().isNullVersion()) {
+ // Pass empty data since it is read from the corresponding .aux/.compaction instant file
+ transitionState(inflightInstant, requestedInstant, Option.empty());
+ } else {
+ deleteInflight(inflightInstant);
+ }
+ return requestedInstant;
+ }
+
/**
* Transition Compaction State from requested to inflight.
*
@@ -421,6 +443,21 @@ public HoodieInstant transitionCompactionRequestedToInflight(HoodieInstant reque
return inflightInstant;
}
+ /**
+ * Transition LogCompaction State from requested to inflight.
+ *
+ * @param requestedInstant Requested instant
+ * @return inflight instant
+ */
+ public HoodieInstant transitionLogCompactionRequestedToInflight(HoodieInstant requestedInstant) {
+ ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION));
+ ValidationUtils.checkArgument(requestedInstant.isRequested());
+ HoodieInstant inflightInstant =
+ new HoodieInstant(State.INFLIGHT, LOG_COMPACTION_ACTION, requestedInstant.getTimestamp());
+ transitionState(requestedInstant, inflightInstant, Option.empty());
+ return inflightInstant;
+ }
+
/**
* Transition Compaction State from inflight to Committed.
*
@@ -436,6 +473,21 @@ public HoodieInstant transitionCompactionInflightToComplete(HoodieInstant inflig
return commitInstant;
}
+ /**
+ * Transition Log Compaction State from inflight to Committed.
+ *
+ * @param inflightInstant Inflight instant
+ * @param data Extra Metadata
+ * @return commit instant
+ */
+ public HoodieInstant transitionLogCompactionInflightToComplete(HoodieInstant inflightInstant, Option data) {
+ ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION));
+ ValidationUtils.checkArgument(inflightInstant.isInflight());
+ HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, DELTA_COMMIT_ACTION, inflightInstant.getTimestamp());
+ transitionState(inflightInstant, commitInstant, data);
+ return commitInstant;
+ }
+
private void createFileInAuxiliaryFolder(HoodieInstant instant, Option data) {
// This will be removed in future release. See HUDI-546
Path fullPath = new Path(metaClient.getMetaAuxiliaryPath(), instant.getFileName());
@@ -653,6 +705,17 @@ public void saveToCompactionRequested(HoodieInstant instant, Option cont
createFileInMetaPath(instant.getFileName(), content, overwrite);
}
+ public void saveToLogCompactionRequested(HoodieInstant instant, Option content) {
+ saveToLogCompactionRequested(instant, content, false);
+ }
+
+ public void saveToLogCompactionRequested(HoodieInstant instant, Option content, boolean overwrite) {
+ ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION));
+ // Write workload to auxiliary folder
+ createFileInAuxiliaryFolder(instant, content);
+ createFileInMetaPath(instant.getFileName(), content, overwrite);
+ }
+
/**
* Saves content for requested REPLACE instant.
*/
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java
index 4df30b115e0ea..2f0ebffbefba3 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java
@@ -205,6 +205,7 @@ private Option getMetadataKey(String action) {
case HoodieTimeline.SAVEPOINT_ACTION:
return Option.of("hoodieSavePointMetadata");
case HoodieTimeline.COMPACTION_ACTION:
+ case HoodieTimeline.LOG_COMPACTION_ACTION:
return Option.of("hoodieCompactionPlan");
case HoodieTimeline.REPLACE_COMMIT_ACTION:
return Option.of("hoodieReplaceCommitMetadata");
@@ -363,7 +364,7 @@ private int getArchivedFileSuffix(FileStatus f) {
@Override
public HoodieDefaultTimeline getWriteTimeline() {
// filter in-memory instants
- Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION);
+ Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, LOG_COMPACTION_ACTION, REPLACE_COMMIT_ACTION);
return new HoodieDefaultTimeline(getInstants().filter(i ->
readCommits.containsKey(i.getTimestamp()))
.filter(s -> validActions.contains(s.getAction())), details);
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java
index 7324421894c0d..0803faeab27cd 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java
@@ -97,6 +97,20 @@ public HoodieTimeline filterPendingExcludingCompaction() {
&& (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION))), details);
}
+ @Override
+ public HoodieTimeline filterPendingExcludingLogCompaction() {
+ return new HoodieDefaultTimeline(instants.stream().filter(instant -> (!instant.isCompleted())
+ && (!instant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION))), details);
+ }
+
+ //TODO: Use a better naming convention for this.
+ @Override
+ public HoodieTimeline filterPendingExcludingMajorAndMinorCompaction() {
+ return new HoodieDefaultTimeline(instants.stream().filter(instant -> (!instant.isCompleted())
+ && (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)
+ || !instant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION))), details);
+ }
+
@Override
public HoodieTimeline filterCompletedInstants() {
return new HoodieDefaultTimeline(instants.stream().filter(HoodieInstant::isCompleted), details);
@@ -108,9 +122,21 @@ public HoodieTimeline filterCompletedAndCompactionInstants() {
|| s.getAction().equals(HoodieTimeline.COMPACTION_ACTION)), details);
}
+ @Override
+ public HoodieTimeline filterCompletedOrMajorOrMinorCompactionInstants() {
+ return new HoodieDefaultTimeline(instants.stream().filter(s -> s.isCompleted()
+ || s.getAction().equals(HoodieTimeline.COMPACTION_ACTION) || s.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)), details);
+ }
+
+ @Override
+ public HoodieDefaultTimeline filterCompletedInstantsOrRewriteTimeline() {
+ Set validActions = CollectionUtils.createSet(COMPACTION_ACTION, LOG_COMPACTION_ACTION, REPLACE_COMMIT_ACTION);
+ return new HoodieDefaultTimeline(instants.stream().filter(s -> s.isCompleted() || validActions.contains(s.getAction())), details);
+ }
+
@Override
public HoodieDefaultTimeline getWriteTimeline() {
- Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION);
+ Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, LOG_COMPACTION_ACTION, REPLACE_COMMIT_ACTION);
return new HoodieDefaultTimeline(instants.stream().filter(s -> validActions.contains(s.getAction())), details);
}
@@ -148,6 +174,23 @@ public HoodieTimeline filterPendingCompactionTimeline() {
instants.stream().filter(s -> s.getAction().equals(HoodieTimeline.COMPACTION_ACTION) && !s.isCompleted()), details);
}
+ @Override
+ public HoodieTimeline filterPendingLogCompactionTimeline() {
+ return new HoodieDefaultTimeline(
+ instants.stream().filter(s -> s.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION) && !s.isCompleted()), details);
+ }
+
+ /**
+ * Compaction and logcompaction operation on MOR table is called major and minor compaction respectively.
+ */
+ @Override
+ public HoodieTimeline filterPendingMajorOrMinorCompactionTimeline() {
+ return new HoodieDefaultTimeline(
+ instants.stream().filter(s -> s.getAction().equals(HoodieTimeline.COMPACTION_ACTION)
+ || s.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)
+ && !s.isCompleted()), details);
+ }
+
@Override
public HoodieDefaultTimeline findInstantsInRange(String startTs, String endTs) {
return new HoodieDefaultTimeline(
@@ -216,13 +259,15 @@ public HoodieTimeline getCommitsTimeline() {
*/
public HoodieTimeline getAllCommitsTimeline() {
return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION,
- CLEAN_ACTION, COMPACTION_ACTION, SAVEPOINT_ACTION, ROLLBACK_ACTION, REPLACE_COMMIT_ACTION, INDEXING_ACTION));
+ CLEAN_ACTION, COMPACTION_ACTION, SAVEPOINT_ACTION, ROLLBACK_ACTION, REPLACE_COMMIT_ACTION, INDEXING_ACTION,
+ LOG_COMPACTION_ACTION));
}
/**
* Get only pure commits (inflight and completed) in the active timeline.
*/
public HoodieTimeline getCommitTimeline() {
+ //TODO: Make sure this change does not break existing functionality.
return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, REPLACE_COMMIT_ACTION));
}
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java
index 8b1cb875c09f6..0115742e07a08 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java
@@ -18,11 +18,11 @@
package org.apache.hudi.common.table.timeline;
-import org.apache.hudi.common.util.CollectionUtils;
import org.apache.hadoop.fs.FileStatus;
import java.io.Serializable;
import java.util.Comparator;
+import java.util.HashMap;
import java.util.Map;
import java.util.Objects;
@@ -38,8 +38,7 @@ public class HoodieInstant implements Serializable, Comparable {
* A COMPACTION action eventually becomes COMMIT when completed. So, when grouping instants
* for state transitions, this needs to be taken into account
*/
- private static final Map COMPARABLE_ACTIONS =
- CollectionUtils.createImmutableMap(HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.COMMIT_ACTION);
+ private static final Map COMPARABLE_ACTIONS = createComparableActionsMap();
public static final Comparator ACTION_COMPARATOR =
Comparator.comparing(instant -> getComparableAction(instant.getAction()));
@@ -164,6 +163,14 @@ public String getFileName() {
} else {
return HoodieTimeline.makeCommitFileName(timestamp);
}
+ } else if (HoodieTimeline.LOG_COMPACTION_ACTION.equals(action)) {
+ if (isInflight()) {
+ return HoodieTimeline.makeInflightLogCompactionFileName(timestamp);
+ } else if (isRequested()) {
+ return HoodieTimeline.makeRequestedLogCompactionFileName(timestamp);
+ } else {
+ return HoodieTimeline.makeDeltaFileName(timestamp);
+ }
} else if (HoodieTimeline.RESTORE_ACTION.equals(action)) {
return isInflight() ? HoodieTimeline.makeInflightRestoreFileName(timestamp)
: isRequested() ? HoodieTimeline.makeRequestedRestoreFileName(timestamp)
@@ -184,6 +191,13 @@ public String getFileName() {
throw new IllegalArgumentException("Cannot get file name for unknown action " + action);
}
+ private static final Map createComparableActionsMap() {
+ Map comparableMap = new HashMap<>();
+ comparableMap.put(HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.COMMIT_ACTION);
+ comparableMap.put(HoodieTimeline.LOG_COMPACTION_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION);
+ return comparableMap;
+ }
+
@Override
public boolean equals(Object o) {
if (this == o) {
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java
index e52a2795969ab..a210783095019 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java
@@ -53,6 +53,7 @@ public interface HoodieTimeline extends Serializable {
// With Async Compaction, compaction instant can be in 3 states :
// (compaction-requested), (compaction-inflight), (completed)
String COMPACTION_ACTION = "compaction";
+ String LOG_COMPACTION_ACTION = "logcompaction";
String REQUESTED_EXTENSION = ".requested";
String RESTORE_ACTION = "restore";
String INDEXING_ACTION = "indexing";
@@ -93,6 +94,10 @@ public interface HoodieTimeline extends Serializable {
String SAVE_SCHEMA_ACTION_EXTENSION = "." + SCHEMA_COMMIT_ACTION;
String INFLIGHT_SAVE_SCHEMA_ACTION_EXTENSION = "." + SCHEMA_COMMIT_ACTION + INFLIGHT_EXTENSION;
String REQUESTED_SAVE_SCHEMA_ACTION_EXTENSION = "." + SCHEMA_COMMIT_ACTION + REQUESTED_EXTENSION;
+ // Log compaction action
+ String REQUESTED_LOG_COMPACTION_SUFFIX = StringUtils.join(LOG_COMPACTION_ACTION, REQUESTED_EXTENSION);
+ String REQUESTED_LOG_COMPACTION_EXTENSION = StringUtils.join(".", REQUESTED_LOG_COMPACTION_SUFFIX);
+ String INFLIGHT_LOG_COMPACTION_EXTENSION = StringUtils.join(".", LOG_COMPACTION_ACTION, INFLIGHT_EXTENSION);
String INVALID_INSTANT_TS = "0";
@@ -124,6 +129,20 @@ public interface HoodieTimeline extends Serializable {
*/
HoodieTimeline filterPendingExcludingCompaction();
+ /**
+ * Filter this timeline to just include the in-flights excluding logcompaction instants.
+ *
+ * @return New instance of HoodieTimeline with just in-flights excluding compaction instants
+ */
+ HoodieTimeline filterPendingExcludingLogCompaction();
+
+ /**
+ * Filter this timeline to just include the in-flights excluding major and minor compaction instants.
+ *
+ * @return New instance of HoodieTimeline with just in-flights excluding majoe and minor compaction instants
+ */
+ HoodieTimeline filterPendingExcludingMajorAndMinorCompaction();
+
/**
* Filter this timeline to just include the completed instants.
*
@@ -131,6 +150,7 @@ public interface HoodieTimeline extends Serializable {
*/
HoodieTimeline filterCompletedInstants();
+ // TODO: Check if logcompaction also needs to be included in this API.
/**
* Filter this timeline to just include the completed + compaction (inflight + requested) instants A RT filesystem
* view is constructed with this timeline so that file-slice after pending compaction-requested instant-time is also
@@ -141,6 +161,15 @@ public interface HoodieTimeline extends Serializable {
*/
HoodieTimeline filterCompletedAndCompactionInstants();
+ HoodieTimeline filterCompletedOrMajorOrMinorCompactionInstants();
+
+ /**
+ * Timeline to just include completed commits or all rewrites like compaction, logcompaction and replace actions.
+ *
+ * @return
+ */
+ HoodieTimeline filterCompletedInstantsOrRewriteTimeline();
+
/**
* Timeline to just include commits (commit/deltacommit), compaction and replace actions.
*
@@ -171,6 +200,20 @@ public interface HoodieTimeline extends Serializable {
*/
HoodieTimeline filterPendingCompactionTimeline();
+ /**
+ * Filter this timeline to just include requested and inflight log compaction instants.
+ *
+ * @return
+ */
+ HoodieTimeline filterPendingLogCompactionTimeline();
+
+ /**
+ * Filter this timeline to just include requested and inflight from both major and minor compaction instants.
+ *
+ * @return
+ */
+ HoodieTimeline filterPendingMajorOrMinorCompactionTimeline();
+
/**
* Filter this timeline to just include requested and inflight replacecommit instants.
*/
@@ -366,6 +409,16 @@ static HoodieInstant getCompactionInflightInstant(final String timestamp) {
return new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, timestamp);
}
+ // Returns Log compaction requested instant
+ static HoodieInstant getLogCompactionRequestedInstant(final String timestamp) {
+ return new HoodieInstant(State.REQUESTED, LOG_COMPACTION_ACTION, timestamp);
+ }
+
+ // Returns Log compaction inflight instant
+ static HoodieInstant getLogCompactionInflightInstant(final String timestamp) {
+ return new HoodieInstant(State.INFLIGHT, LOG_COMPACTION_ACTION, timestamp);
+ }
+
static HoodieInstant getReplaceCommitRequestedInstant(final String timestamp) {
return new HoodieInstant(State.REQUESTED, REPLACE_COMMIT_ACTION, timestamp);
}
@@ -388,14 +441,26 @@ static HoodieInstant getIndexInflightInstant(final String timestamp) {
/**
* Returns the inflight instant corresponding to the instant being passed. Takes care of changes in action names
- * between inflight and completed instants (compaction <=> commit).
+ * between inflight and completed instants (compaction <=> commit) and (logcompaction <==> deltacommit).
* @param instant Hoodie Instant
- * @param tableType Hoodie Table Type
+ * @param metaClient Hoodie metaClient to fetch tableType and fileSystem.
* @return Inflight Hoodie Instant
*/
- static HoodieInstant getInflightInstant(final HoodieInstant instant, final HoodieTableType tableType) {
- if ((tableType == HoodieTableType.MERGE_ON_READ) && instant.getAction().equals(COMMIT_ACTION)) {
- return new HoodieInstant(true, COMPACTION_ACTION, instant.getTimestamp());
+ static HoodieInstant getInflightInstant(final HoodieInstant instant, final HoodieTableMetaClient metaClient) {
+ if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) {
+ if (instant.getAction().equals(COMMIT_ACTION)) {
+ return new HoodieInstant(true, COMPACTION_ACTION, instant.getTimestamp());
+ } else if (instant.getAction().equals(DELTA_COMMIT_ACTION)) {
+ // Deltacommit is used by both ingestion and logcompaction.
+ // So, distinguish both of them check for the inflight file being present.
+ HoodieActiveTimeline rawActiveTimeline = new HoodieActiveTimeline(metaClient, false);
+ Option logCompactionInstant = Option.fromJavaOptional(rawActiveTimeline.getInstants()
+ .filter(hoodieInstant -> hoodieInstant.getTimestamp().equals(instant.getTimestamp())
+ && LOG_COMPACTION_ACTION.equals(hoodieInstant.getAction())).findFirst());
+ if (logCompactionInstant.isPresent()) {
+ return new HoodieInstant(true, LOG_COMPACTION_ACTION, instant.getTimestamp());
+ }
+ }
}
return new HoodieInstant(true, instant.getAction(), instant.getTimestamp());
}
@@ -464,6 +529,15 @@ static String makeRequestedCompactionFileName(String instantTime) {
return StringUtils.join(instantTime, HoodieTimeline.REQUESTED_COMPACTION_EXTENSION);
}
+ // Log comaction action
+ static String makeInflightLogCompactionFileName(String instantTime) {
+ return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_LOG_COMPACTION_EXTENSION);
+ }
+
+ static String makeRequestedLogCompactionFileName(String instantTime) {
+ return StringUtils.join(instantTime, HoodieTimeline.REQUESTED_LOG_COMPACTION_EXTENSION);
+ }
+
static String makeRestoreFileName(String instant) {
return StringUtils.join(instant, HoodieTimeline.RESTORE_EXTENSION);
}
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineDiffHelper.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineDiffHelper.java
index 5659dcbdf7fae..29399705f00db 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineDiffHelper.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineDiffHelper.java
@@ -72,8 +72,14 @@ public static TimelineDiffResult getNewInstantsForIncrementalSync(HoodieTimeline
&& instantPair.getValue().isCompleted())
.map(Pair::getKey).collect(Collectors.toList());
- newT.getInstants().filter(instant -> !oldTimelineInstants.contains(instant)).forEach(newInstants::add);
- return new TimelineDiffResult(newInstants, finishedCompactionInstants, true);
+ newTimeline.getInstants().filter(instant -> !oldTimelineInstants.contains(instant)).forEach(newInstants::add);
+
+ List> logCompactionInstants = getPendingLogCompactionTransitions(oldTimeline, newTimeline);
+ List finishedOrRemovedLogCompactionInstants = logCompactionInstants.stream()
+ .filter(instantPair -> !instantPair.getKey().isCompleted()
+ && (instantPair.getValue() == null || instantPair.getValue().isCompleted()))
+ .map(Pair::getKey).collect(Collectors.toList());
+ return new TimelineDiffResult(newInstants, finishedCompactionInstants, finishedOrRemovedLogCompactionInstants, true);
} else {
// One or more timelines is empty
LOG.warn("One or more timelines is empty");
@@ -81,6 +87,35 @@ public static TimelineDiffResult getNewInstantsForIncrementalSync(HoodieTimeline
}
}
+ /**
+ * Getting pending log compaction transitions.
+ */
+ private static List> getPendingLogCompactionTransitions(HoodieTimeline oldTimeline,
+ HoodieTimeline newTimeline) {
+ Set newTimelineInstants = newTimeline.getInstants().collect(Collectors.toSet());
+
+ return oldTimeline.filterPendingLogCompactionTimeline().getInstants().map(instant -> {
+ if (newTimelineInstants.contains(instant)) {
+ return Pair.of(instant, instant);
+ } else {
+ HoodieInstant logCompacted =
+ new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instant.getTimestamp());
+ if (newTimelineInstants.contains(logCompacted)) {
+ return Pair.of(instant, logCompacted);
+ }
+ HoodieInstant inflightLogCompacted =
+ new HoodieInstant(State.INFLIGHT, HoodieTimeline.LOG_COMPACTION_ACTION, instant.getTimestamp());
+ if (newTimelineInstants.contains(inflightLogCompacted)) {
+ return Pair.of(instant, inflightLogCompacted);
+ }
+ return Pair.of(instant, null);
+ }
+ }).collect(Collectors.toList());
+ }
+
+ /**
+ * Getting pending compaction transitions.
+ */
private static List> getPendingCompactionTransitions(HoodieTimeline oldTimeline,
HoodieTimeline newTimeline) {
Set newTimelineInstants = newTimeline.getInstants().collect(Collectors.toSet());
@@ -94,6 +129,11 @@ private static List> getPendingCompactionTran
if (newTimelineInstants.contains(compacted)) {
return Pair.of(instant, compacted);
}
+ HoodieInstant inflightCompacted =
+ new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, instant.getTimestamp());
+ if (newTimelineInstants.contains(inflightCompacted)) {
+ return Pair.of(instant, inflightCompacted);
+ }
return Pair.of(instant, null);
}
}).collect(Collectors.toList());
@@ -106,14 +146,17 @@ public static class TimelineDiffResult {
private final List newlySeenInstants;
private final List finishedCompactionInstants;
+ private final List finishedOrRemovedLogCompactionInstants;
private final boolean canSyncIncrementally;
- public static final TimelineDiffResult UNSAFE_SYNC_RESULT = new TimelineDiffResult(null, null, false);
+ public static final TimelineDiffResult UNSAFE_SYNC_RESULT =
+ new TimelineDiffResult(null, null, null, false);
public TimelineDiffResult(List newlySeenInstants, List finishedCompactionInstants,
- boolean canSyncIncrementally) {
+ List finishedOrRemovedLogCompactionInstants, boolean canSyncIncrementally) {
this.newlySeenInstants = newlySeenInstants;
this.finishedCompactionInstants = finishedCompactionInstants;
+ this.finishedOrRemovedLogCompactionInstants = finishedOrRemovedLogCompactionInstants;
this.canSyncIncrementally = canSyncIncrementally;
}
@@ -125,14 +168,22 @@ public List getFinishedCompactionInstants() {
return finishedCompactionInstants;
}
+ public List getFinishedOrRemovedLogCompactionInstants() {
+ return finishedOrRemovedLogCompactionInstants;
+ }
+
public boolean canSyncIncrementally() {
return canSyncIncrementally;
}
@Override
public String toString() {
- return "TimelineDiffResult{newlySeenInstants=" + newlySeenInstants + ", finishedCompactionInstants="
- + finishedCompactionInstants + ", canSyncIncrementally=" + canSyncIncrementally + '}';
+ return "TimelineDiffResult{"
+ + "newlySeenInstants=" + newlySeenInstants
+ + ", finishedCompactionInstants=" + finishedCompactionInstants
+ + ", finishedOrRemovedLogCompactionInstants=" + finishedOrRemovedLogCompactionInstants
+ + ", canSyncIncrementally=" + canSyncIncrementally
+ + '}';
}
}
}
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java
index ed4bfd760161f..6dbbf85fa578b 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java
@@ -82,6 +82,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
protected HoodieTableMetaClient metaClient;
// This is the commits timeline that will be visible for all views extending this view
+ // This is nothing but the write timeline, which contains both ingestion and compaction(major and minor) writers.
private HoodieTimeline visibleCommitsAndCompactionTimeline;
// Used to concurrently load and populate partition views
@@ -110,6 +111,10 @@ protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActi
// Load Pending Compaction Operations
resetPendingCompactionOperations(CompactionUtils.getAllPendingCompactionOperations(metaClient).values().stream()
.map(e -> Pair.of(e.getKey(), CompactionOperation.convertFromAvroRecordInstance(e.getValue()))));
+ // Load Pending LogCompaction Operations.
+ resetPendingLogCompactionOperations(CompactionUtils.getAllPendingLogCompactionOperations(metaClient).values().stream()
+ .map(e -> Pair.of(e.getKey(), CompactionOperation.convertFromAvroRecordInstance(e.getValue()))));
+
resetBootstrapBaseFileMapping(Stream.empty());
resetFileGroupsInPendingClustering(ClusteringUtils.getAllFileGroupsInPendingClusteringPlans(metaClient));
}
@@ -489,6 +494,16 @@ public final List getPartitionPaths() {
}
}
+ @Override
+ public final Stream> getPendingLogCompactionOperations() {
+ try {
+ readLock.lock();
+ return fetchPendingLogCompactionOperations();
+ } finally {
+ readLock.unlock();
+ }
+ }
+
@Override
public final Stream getLatestBaseFiles(String partitionStr) {
try {
@@ -819,6 +834,35 @@ public final Stream> getFileGroupsInPendi
*/
abstract void removePendingCompactionOperations(Stream> operations);
+ /**
+ * Check if there is an outstanding log compaction scheduled for this file.
+ *
+ * @param fgId File-Group Id
+ * @return true if there is a pending log compaction, false otherwise
+ */
+ protected abstract boolean isPendingLogCompactionScheduledForFileId(HoodieFileGroupId fgId);
+
+ /**
+ * resets the pending Log compaction operation and overwrite with the new list.
+ *
+ * @param operations Pending Log Compaction Operations
+ */
+ abstract void resetPendingLogCompactionOperations(Stream> operations);
+
+ /**
+ * Add pending Log compaction operations to store.
+ *
+ * @param operations Pending Log compaction operations to be added
+ */
+ abstract void addPendingLogCompactionOperations(Stream> operations);
+
+ /**
+ * Remove pending Log compaction operations from store.
+ *
+ * @param operations Pending Log compaction operations to be removed
+ */
+ abstract void removePendingLogCompactionOperations(Stream> operations);
+
/**
* Check if there is an outstanding clustering operation (requested/inflight) scheduled for this file.
*
@@ -861,11 +905,24 @@ public final Stream> getFileGroupsInPendi
protected abstract Option> getPendingCompactionOperationWithInstant(
HoodieFileGroupId fileGroupId);
+ /**
+ * Return pending Log compaction operation for a file-group.
+ *
+ * @param fileGroupId File-Group Id
+ */
+ protected abstract Option> getPendingLogCompactionOperationWithInstant(
+ HoodieFileGroupId fileGroupId);
+
/**
* Fetch all pending compaction operations.
*/
abstract Stream> fetchPendingCompactionOperations();
+ /**
+ * Fetch all pending log compaction operations.
+ */
+ abstract Stream> fetchPendingLogCompactionOperations();
+
/**
* Check if there is an bootstrap base file present for this file.
*
@@ -1147,7 +1204,7 @@ public HoodieTimeline getTimeline() {
@Override
public void sync() {
HoodieTimeline oldTimeline = getTimeline();
- HoodieTimeline newTimeline = metaClient.reloadActiveTimeline().filterCompletedAndCompactionInstants();
+ HoodieTimeline newTimeline = metaClient.reloadActiveTimeline().filterCompletedOrMajorOrMinorCompactionInstants();
try {
writeLock.lock();
runSync(oldTimeline, newTimeline);
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java
index 92937f61e2c2c..2f43fb9b4f3b6 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java
@@ -85,6 +85,11 @@ public class FileSystemViewStorageConfig extends HoodieConfig {
.defaultValue(0.8)
.withDocumentation("Fraction of the file system view memory, to be used for holding compaction related metadata.");
+ public static final ConfigProperty SPILLABLE_LOG_COMPACTION_MEM_FRACTION = ConfigProperty
+ .key("hoodie.filesystem.view.spillable.log.compaction.mem.fraction")
+ .defaultValue(0.8)
+ .withDocumentation("Fraction of the file system view memory, to be used for holding log compaction related metadata.");
+
public static final ConfigProperty BOOTSTRAP_BASE_FILE_MEM_FRACTION = ConfigProperty
.key("hoodie.filesystem.view.spillable.bootstrap.base.file.mem.fraction")
.defaultValue(0.05)
@@ -206,6 +211,12 @@ public long getMaxMemoryForPendingCompaction() {
.longValue();
}
+ public long getMaxMemoryForPendingLogCompaction() {
+ long totalMemory = getLong(SPILLABLE_MEMORY);
+ return new Double(totalMemory * getDouble(SPILLABLE_LOG_COMPACTION_MEM_FRACTION))
+ .longValue();
+ }
+
public long getMaxMemoryForBootstrapBaseFile() {
long totalMemory = getLong(SPILLABLE_MEMORY);
long reservedForExternalDataFile =
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java
index 9dac36081384b..ea72f305b0738 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java
@@ -62,6 +62,11 @@ public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystem
*/
protected Map> fgIdToPendingCompaction;
+ /**
+ * PartitionPath + File-Id to pending compaction instant time.
+ */
+ protected Map> fgIdToPendingLogCompaction;
+
/**
* PartitionPath + File-Id to bootstrap base File (Index Only bootstrapped).
*/
@@ -143,6 +148,11 @@ protected Map> createFileId
return fileIdToPendingCompaction;
}
+ protected Map> createFileIdToPendingLogCompactionMap(
+ Map> fileIdToPendingLogCompaction) {
+ return fileIdToPendingLogCompaction;
+ }
+
protected Map createFileIdToBootstrapBaseFileMap(
Map fileGroupIdBootstrapBaseFileMap) {
return fileGroupIdBootstrapBaseFileMap;
@@ -213,6 +223,39 @@ protected void removePendingCompactionOperations(Stream> operations) {
+ // Build fileId to Pending Log Compaction Instants
+ this.fgIdToPendingLogCompaction = createFileIdToPendingLogCompactionMap(operations.map(entry ->
+ Pair.of(entry.getValue().getFileGroupId(), Pair.of(entry.getKey(), entry.getValue()))).collect(Collectors.toMap(Pair::getKey, Pair::getValue)));
+ }
+
+ @Override
+ protected void addPendingLogCompactionOperations(Stream> operations) {
+ operations.forEach(opInstantPair -> {
+ ValidationUtils.checkArgument(!fgIdToPendingLogCompaction.containsKey(opInstantPair.getValue().getFileGroupId()),
+ "Duplicate FileGroupId found in pending log compaction operations. FgId :"
+ + opInstantPair.getValue().getFileGroupId());
+ fgIdToPendingLogCompaction.put(opInstantPair.getValue().getFileGroupId(),
+ Pair.of(opInstantPair.getKey(), opInstantPair.getValue()));
+ });
+ }
+
+ @Override
+ protected void removePendingLogCompactionOperations(Stream> operations) {
+ operations.forEach(opInstantPair -> {
+ ValidationUtils.checkArgument(fgIdToPendingLogCompaction.containsKey(opInstantPair.getValue().getFileGroupId()),
+ "Trying to remove a FileGroupId which is not found in pending log compaction operations. FgId :"
+ + opInstantPair.getValue().getFileGroupId());
+ fgIdToPendingLogCompaction.remove(opInstantPair.getValue().getFileGroupId());
+ });
+ }
+
@Override
protected boolean isPendingClusteringScheduledForFileId(HoodieFileGroupId fgId) {
return fgIdToPendingClustering.containsKey(fgId);
@@ -273,6 +316,11 @@ public Stream getAllFileGroups() {
@Override
Stream> fetchPendingCompactionOperations() {
return fgIdToPendingCompaction.values().stream();
+ }
+
+ @Override
+ Stream> fetchPendingLogCompactionOperations() {
+ return fgIdToPendingLogCompaction.values().stream();
}
@@ -323,6 +371,11 @@ protected Option> getPendingCompactionOperatio
return Option.ofNullable(fgIdToPendingCompaction.get(fgId));
}
+ @Override
+ protected Option> getPendingLogCompactionOperationWithInstant(HoodieFileGroupId fgId) {
+ return Option.ofNullable(fgIdToPendingLogCompaction.get(fgId));
+ }
+
@Override
protected boolean isPartitionAvailableInStore(String partitionPath) {
return partitionToFileGroupsMap.containsKey(partitionPath);
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java
index 9f9fd1f7ce601..4c2aa79f27c6d 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java
@@ -116,9 +116,20 @@ private void runIncrementalSync(HoodieTimeline timeline, TimelineDiffResult diff
}
});
- // Add new completed instants found in the latest timeline
+ // Now remove pending log compaction instants which were completed or removed
+ diffResult.getFinishedOrRemovedLogCompactionInstants().stream().forEach(instant -> {
+ try {
+ removePendingLogCompactionInstant(timeline, instant);
+ } catch (IOException e) {
+ throw new HoodieException(e);
+ }
+ });
+
+ // Add new completed instants found in the latest timeline, this also contains inflight instants.
diffResult.getNewlySeenInstants().stream()
- .filter(instant -> instant.isCompleted() || instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION))
+ .filter(instant -> instant.isCompleted()
+ || instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)
+ || instant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION))
.forEach(instant -> {
try {
if (instant.getAction().equals(HoodieTimeline.COMMIT_ACTION)
@@ -130,6 +141,8 @@ private void runIncrementalSync(HoodieTimeline timeline, TimelineDiffResult diff
addCleanInstant(timeline, instant);
} else if (instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)) {
addPendingCompactionInstant(timeline, instant);
+ } else if (instant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)) {
+ addPendingLogCompactionInstant(instant);
} else if (instant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)) {
addRollbackInstant(timeline, instant);
} else if (instant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) {
@@ -155,6 +168,21 @@ private void removePendingCompactionInstant(HoodieTimeline timeline, HoodieInsta
CompactionOperation.convertFromAvroRecordInstance(instantPair.getValue().getValue()))));
}
+ /**
+ * Remove Pending compaction instant. This is called when logcompaction is converted to delta commit,
+ * so you no longer need to track them as pending.
+ *
+ * @param timeline New Hoodie Timeline
+ * @param instant Log Compaction Instant to be removed
+ */
+ private void removePendingLogCompactionInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
+ LOG.info("Removing completed log compaction instant (" + instant + ")");
+ HoodieCompactionPlan plan = CompactionUtils.getLogCompactionPlan(metaClient, instant.getTimestamp());
+ removePendingLogCompactionOperations(CompactionUtils.getPendingCompactionOperations(instant, plan)
+ .map(instantPair -> Pair.of(instantPair.getValue().getKey(),
+ CompactionOperation.convertFromAvroRecordInstance(instantPair.getValue().getValue()))));
+ }
+
/**
* Add newly found compaction instant.
*
@@ -186,6 +214,24 @@ private void addPendingCompactionInstant(HoodieTimeline timeline, HoodieInstant
});
}
+ /**
+ * Add newly found compaction instant.
+ *
+ * @param instant Compaction Instant
+ */
+ private void addPendingLogCompactionInstant(HoodieInstant instant) throws IOException {
+ LOG.info("Syncing pending log compaction instant (" + instant + ")");
+ HoodieCompactionPlan compactionPlan = CompactionUtils.getLogCompactionPlan(metaClient, instant.getTimestamp());
+ List> pendingOps =
+ CompactionUtils.getPendingCompactionOperations(instant, compactionPlan)
+ .map(p -> Pair.of(p.getValue().getKey(),
+ CompactionOperation.convertFromAvroRecordInstance(p.getValue().getValue())))
+ .collect(Collectors.toList());
+ // Update Pending log compaction instants.
+ // Since logcompaction works similar to a deltacommit. Updating the partition view is not required.
+ addPendingLogCompactionOperations(pendingOps.stream());
+ }
+
/**
* Add newly found commit/delta-commit instant.
*
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java
index ff44c7cef017b..62edc4daa33e0 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java
@@ -224,6 +224,11 @@ public Stream> getPendingCompactionOperations(
return execute(preferredView::getPendingCompactionOperations, secondaryView::getPendingCompactionOperations);
}
+ @Override
+ public Stream> getPendingLogCompactionOperations() {
+ return execute(preferredView::getPendingLogCompactionOperations, secondaryView::getPendingLogCompactionOperations);
+ }
+
@Override
public Stream> getFileGroupsInPendingClustering() {
return execute(preferredView::getFileGroupsInPendingClustering, secondaryView::getFileGroupsInPendingClustering);
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java
index bd18ba22a25d6..759ef70c6d80f 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java
@@ -74,6 +74,7 @@ public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView,
String.format("%s/%s", BASE_URL, "slices/beforeoron/latest/");
public static final String PENDING_COMPACTION_OPS = String.format("%s/%s", BASE_URL, "compactions/pending/");
+ public static final String PENDING_LOG_COMPACTION_OPS = String.format("%s/%s", BASE_URL, "logcompactions/pending/");
public static final String LATEST_PARTITION_DATA_FILES_URL =
String.format("%s/%s", BASE_URL, "datafiles/latest/partition");
@@ -436,6 +437,18 @@ public Stream> getPendingCompactionOperations(
}
}
+ @Override
+ public Stream> getPendingLogCompactionOperations() {
+ Map paramsMap = getParams();
+ try {
+ List dtos = executeRequest(PENDING_LOG_COMPACTION_OPS, paramsMap,
+ new TypeReference>() {}, RequestMethod.GET);
+ return dtos.stream().map(CompactionOpDTO::toCompactionOperation);
+ } catch (IOException e) {
+ throw new HoodieRemoteException(e);
+ }
+ }
+
@Override
public Stream> getFileGroupsInPendingClustering() {
Map paramsMap = getParams();
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java
index ebd759f88060b..2389f267818ec 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java
@@ -135,6 +135,49 @@ void removePendingCompactionOperations(Stream>
);
}
+ @Override
+ protected boolean isPendingLogCompactionScheduledForFileId(HoodieFileGroupId fgId) {
+ return getPendingLogCompactionOperationWithInstant(fgId).isPresent();
+ }
+
+ @Override
+ protected void resetPendingLogCompactionOperations(Stream> operations) {
+ rocksDB.writeBatch(batch -> {
+ operations.forEach(opPair ->
+ rocksDB.putInBatch(batch, schemaHelper.getColFamilyForPendingLogCompaction(),
+ schemaHelper.getKeyForPendingLogCompactionLookup(opPair.getValue().getFileGroupId()), opPair)
+ );
+ LOG.info("Initializing pending Log compaction operations. Count=" + batch.count());
+ });
+ }
+
+ @Override
+ protected void addPendingLogCompactionOperations(Stream> operations) {
+ rocksDB.writeBatch(batch ->
+ operations.forEach(opInstantPair -> {
+ ValidationUtils.checkArgument(!isPendingLogCompactionScheduledForFileId(opInstantPair.getValue().getFileGroupId()),
+ "Duplicate FileGroupId found in pending log compaction operations. FgId :"
+ + opInstantPair.getValue().getFileGroupId());
+ rocksDB.putInBatch(batch, schemaHelper.getColFamilyForPendingLogCompaction(),
+ schemaHelper.getKeyForPendingLogCompactionLookup(opInstantPair.getValue().getFileGroupId()), opInstantPair);
+ })
+ );
+ }
+
+ @Override
+ void removePendingLogCompactionOperations(Stream> operations) {
+ rocksDB.writeBatch(batch ->
+ operations.forEach(opInstantPair -> {
+ ValidationUtils.checkArgument(
+ getPendingLogCompactionOperationWithInstant(opInstantPair.getValue().getFileGroupId()) != null,
+ "Trying to remove a FileGroupId which is not found in pending Log compaction operations. FgId :"
+ + opInstantPair.getValue().getFileGroupId());
+ rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForPendingLogCompaction(),
+ schemaHelper.getKeyForPendingLogCompactionLookup(opInstantPair.getValue().getFileGroupId()));
+ })
+ );
+ }
+
@Override
protected boolean isPendingClusteringScheduledForFileId(HoodieFileGroupId fgId) {
return getPendingClusteringInstant(fgId).isPresent();
@@ -171,7 +214,7 @@ void addFileGroupsInPendingClustering(Stream
fileGroups.forEach(fgIdToClusterInstant -> {
ValidationUtils.checkArgument(!isPendingClusteringScheduledForFileId(fgIdToClusterInstant.getLeft()),
- "Duplicate FileGroupId found in pending compaction operations. FgId :"
+ "Duplicate FileGroupId found in pending clustering operations. FgId :"
+ fgIdToClusterInstant.getLeft());
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForFileGroupsInPendingClustering(),
@@ -186,7 +229,7 @@ void removeFileGroupsInPendingClustering(Stream {
ValidationUtils.checkArgument(
!isPendingClusteringScheduledForFileId(fgToPendingClusteringInstant.getLeft()),
- "Trying to remove a FileGroupId which is not found in pending compaction operations. FgId :"
+ "Trying to remove a FileGroupId which is not found in pending clustering operations. FgId :"
+ fgToPendingClusteringInstant.getLeft());
rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForFileGroupsInPendingClustering(),
schemaHelper.getKeyForFileGroupsInPendingClustering(fgToPendingClusteringInstant.getLeft()));
@@ -210,6 +253,14 @@ protected Option> getPendingCompactionOperatio
return Option.ofNullable(instantOperationPair);
}
+ @Override
+ protected Option> getPendingLogCompactionOperationWithInstant(HoodieFileGroupId fgId) {
+ String lookupKey = schemaHelper.getKeyForPendingLogCompactionLookup(fgId);
+ Pair instantOperationPair =
+ rocksDB.get(schemaHelper.getColFamilyForPendingLogCompaction(), lookupKey);
+ return Option.ofNullable(instantOperationPair);
+ }
+
@Override
protected boolean isPartitionAvailableInStore(String partitionPath) {
String lookupKey = schemaHelper.getKeyForPartitionLookup(partitionPath);
@@ -323,6 +374,12 @@ Stream> fetchPendingCompactionOperations() {
.map(Pair::getValue);
}
+ @Override
+ Stream> fetchPendingLogCompactionOperations() {
+ return rocksDB.>prefixSearch(schemaHelper.getColFamilyForPendingLogCompaction(), "")
+ .map(Pair::getValue);
+ }
+
@Override
Stream fetchAllBaseFiles(String partitionPath) {
return rocksDB.prefixSearch(schemaHelper.getColFamilyForView(),
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java
index e4144420831c7..26d36d9c97bc0 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java
@@ -51,6 +51,7 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
private final long maxMemoryForFileGroupMap;
private final long maxMemoryForPendingCompaction;
+ private final long maxMemoryForPendingLogCompaction;
private final long maxMemoryForBootstrapBaseFile;
private final long maxMemoryForReplaceFileGroups;
private final long maxMemoryForClusteringFileGroups;
@@ -63,6 +64,7 @@ public SpillableMapBasedFileSystemView(HoodieTableMetaClient metaClient, HoodieT
super(config.isIncrementalTimelineSyncEnabled());
this.maxMemoryForFileGroupMap = config.getMaxMemoryForFileGroupMap();
this.maxMemoryForPendingCompaction = config.getMaxMemoryForPendingCompaction();
+ this.maxMemoryForPendingLogCompaction = config.getMaxMemoryForPendingLogCompaction();
this.maxMemoryForBootstrapBaseFile = config.getMaxMemoryForBootstrapBaseFile();
this.maxMemoryForReplaceFileGroups = config.getMaxMemoryForReplacedFileGroups();
this.maxMemoryForClusteringFileGroups = config.getMaxMemoryForPendingClusteringFileGroups();
@@ -109,6 +111,23 @@ maxMemoryForPendingCompaction, baseStoreDir, new DefaultSizeEstimator(), new Def
}
}
+ @Override
+ protected Map> createFileIdToPendingLogCompactionMap(
+ Map> fgIdToPendingLogCompaction) {
+ try {
+ LOG.info("Creating Pending Log Compaction map using external spillable Map. Max Mem=" + maxMemoryForPendingLogCompaction
+ + ", BaseDir=" + baseStoreDir);
+ new File(baseStoreDir).mkdirs();
+ Map> pendingMap = new ExternalSpillableMap<>(
+ maxMemoryForPendingLogCompaction, baseStoreDir, new DefaultSizeEstimator(), new DefaultSizeEstimator<>(),
+ diskMapType, isBitCaskDiskMapCompressionEnabled);
+ pendingMap.putAll(fgIdToPendingLogCompaction);
+ return pendingMap;
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
@Override
protected Map createFileIdToBootstrapBaseFileMap(
Map fileGroupIdBootstrapBaseFileMap) {
@@ -169,6 +188,11 @@ Stream> fetchPendingCompactionOperations() {
return ((ExternalSpillableMap) fgIdToPendingCompaction).valueStream();
}
+ @Override
+ Stream> fetchPendingLogCompactionOperations() {
+ return ((ExternalSpillableMap) fgIdToPendingLogCompaction).valueStream();
+ }
+
@Override
Stream fetchBootstrapBaseFiles() {
return ((ExternalSpillableMap) fgIdToBootstrapBaseFile).valueStream();
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java
index c32e2cabb1012..18c9a9af99817 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java
@@ -156,6 +156,13 @@ interface SliceView extends SliceViewWithLatestSlice {
*/
Stream> getPendingCompactionOperations();
+ /**
+ * Return Pending Compaction Operations.
+ *
+ * @return Pair>
+ */
+ Stream> getPendingLogCompactionOperations();
+
/**
* Last Known Instant on which the view is built.
*/
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java
index 90d6e6ae90fb0..3b94948707f5c 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CollectionUtils.java
@@ -261,4 +261,5 @@ private static Object[] checkElementsNotNull(Object[] array, int length) {
private static Object checkElementNotNull(Object element, int index) {
return Objects.requireNonNull(element, "Element is null at index " + index);
}
+
}
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java
index cf9b5fb3ced8c..d909d209de256 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CompactionUtils.java
@@ -35,9 +35,6 @@
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
-import org.apache.log4j.LogManager;
-import org.apache.log4j.Logger;
-
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
@@ -51,8 +48,6 @@
*/
public class CompactionUtils {
- private static final Logger LOG = LogManager.getLogger(CompactionUtils.class);
-
public static final Integer COMPACTION_METADATA_VERSION_1 = CompactionV1MigrationHandler.VERSION;
public static final Integer COMPACTION_METADATA_VERSION_2 = CompactionV2MigrationHandler.VERSION;
public static final Integer LATEST_COMPACTION_METADATA_VERSION = COMPACTION_METADATA_VERSION_2;
@@ -126,29 +121,79 @@ public static CompactionOperation buildCompactionOperation(HoodieCompactionOpera
/**
* Get all pending compaction plans along with their instants.
- *
* @param metaClient Hoodie Meta Client
*/
public static List> getAllPendingCompactionPlans(
HoodieTableMetaClient metaClient) {
- List pendingCompactionInstants =
- metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants().collect(Collectors.toList());
- return pendingCompactionInstants.stream().map(instant -> {
- try {
- return Pair.of(instant, getCompactionPlan(metaClient, instant.getTimestamp()));
- } catch (IOException e) {
- throw new HoodieException(e);
- }
- }).collect(Collectors.toList());
+ // This function returns pending compaction timeline.
+ Function getFilteredTimelineByActionType =
+ (hoodieTableMetaClient) -> hoodieTableMetaClient.getActiveTimeline().filterPendingCompactionTimeline();
+ // Hoodie requested instant supplier
+ Function requestedInstantSupplier = HoodieTimeline::getCompactionRequestedInstant;
+ return getCompactionPlansByTimeline(metaClient, getFilteredTimelineByActionType, requestedInstantSupplier);
+ }
+
+ /**
+ * Get all pending logcompaction plans along with their instants.
+ * @param metaClient Hoodie Meta Client
+ */
+ public static List> getAllPendingLogCompactionPlans(
+ HoodieTableMetaClient metaClient) {
+ // This function returns pending logcompaction timeline.
+ Function filteredTimelineSupplier =
+ (hoodieTableMetaClient) -> hoodieTableMetaClient.getActiveTimeline().filterPendingLogCompactionTimeline();
+ // Hoodie requested instant supplier
+ Function requestedInstantSupplier = HoodieTimeline::getLogCompactionRequestedInstant;
+ return getCompactionPlansByTimeline(metaClient, filteredTimelineSupplier, requestedInstantSupplier);
+ }
+
+ /**
+ * Util method to get compaction plans by action_type(COMPACT or LOG_COMPACT)
+ * @param metaClient HoodieTable's metaclient
+ * @param filteredTimelineSupplier gives a timeline object, this can be either filtered to return pending compactions or log compaction instants.
+ * @param requestedInstantWrapper function that gives a requested Hoodie instant.
+ * @return List of pair of HoodieInstant and it's corresponding compaction plan.
+ * Note here the compaction plan can be related to a compaction instant or log compaction instant.
+ */
+ private static List> getCompactionPlansByTimeline(
+ HoodieTableMetaClient metaClient, Function filteredTimelineSupplier,
+ Function requestedInstantWrapper) {
+ List filteredInstants = filteredTimelineSupplier.apply(metaClient).getInstants().collect(Collectors.toList());
+ return filteredInstants.stream()
+ .map(instant -> Pair.of(instant, getCompactionPlan(metaClient, requestedInstantWrapper.apply(instant.getTimestamp()))))
+ .collect(Collectors.toList());
+ }
+
+ /**
+ * This method will serve only Compaction instants
+ * because we use same HoodieCompactionPlan for both the operations.
+ */
+ public static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaClient, String compactionInstant) {
+ HoodieInstant compactionRequestedInstant = HoodieTimeline.getCompactionRequestedInstant(compactionInstant);
+ return getCompactionPlan(metaClient, compactionRequestedInstant);
}
- public static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaClient, String compactionInstant)
- throws IOException {
+ /**
+ * This method will serve only log compaction instants,
+ * because we use same HoodieCompactionPlan for both the operations.
+ */
+ public static HoodieCompactionPlan getLogCompactionPlan(HoodieTableMetaClient metaClient, String logCompactionInstant) {
+ HoodieInstant logCompactionRequestedInstant = HoodieTimeline.getLogCompactionRequestedInstant(logCompactionInstant);
+ return getCompactionPlan(metaClient, logCompactionRequestedInstant);
+ }
+
+ /**
+ * Util method to fetch both compaction and log compaction plan from requestedInstant.
+ */
+ private static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaClient, HoodieInstant requestedInstant) {
CompactionPlanMigrator migrator = new CompactionPlanMigrator(metaClient);
- HoodieCompactionPlan compactionPlan = TimelineMetadataUtils.deserializeCompactionPlan(
- metaClient.getActiveTimeline().readCompactionPlanAsBytes(
- HoodieTimeline.getCompactionRequestedInstant(compactionInstant)).get());
- return migrator.upgradeToLatest(compactionPlan, compactionPlan.getVersion());
+ try {
+ HoodieCompactionPlan compactionPlan = TimelineMetadataUtils.deserializeCompactionPlan(
+ metaClient.getActiveTimeline().readCompactionPlanAsBytes(requestedInstant).get());
+ return migrator.upgradeToLatest(compactionPlan, compactionPlan.getVersion());
+ } catch (IOException e) {
+ throw new HoodieException(e);
+ }
}
/**
@@ -160,29 +205,49 @@ public static Map> ge
HoodieTableMetaClient metaClient) {
List> pendingCompactionPlanWithInstants =
getAllPendingCompactionPlans(metaClient);
+ return getAllPendingCompactionOperationsInPendingCompactionPlans(pendingCompactionPlanWithInstants);
+ }
+
+ /**
+ * Get all partition + file Ids with pending Log Compaction operations and their target log compaction instant time.
+ */
+ public static Map> getAllPendingLogCompactionOperations(
+ HoodieTableMetaClient metaClient) {
+ List> pendingLogCompactionPlanWithInstants =
+ getAllPendingLogCompactionPlans(metaClient);
+ return getAllPendingCompactionOperationsInPendingCompactionPlans(pendingLogCompactionPlanWithInstants);
+ }
- Map> fgIdToPendingCompactionWithInstantMap =
- new HashMap<>();
- pendingCompactionPlanWithInstants.stream().flatMap(instantPlanPair ->
+ /**
+ * Get all partition + file Ids with pending Log Compaction operations and their target log compaction instant time.
+ */
+ public static Map