codope commented on code in PR #8900:
URL: https://github.com/apache/hudi/pull/8900#discussion_r1223850963
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java:
##########
@@ -61,6 +61,12 @@ public class HoodieCompactionConfig extends HoodieConfig {
+ "but users are expected to trigger async job for execution. If
`hoodie.compact.inline` is set to true, regular writers will do both scheduling
and "
+ "execution inline for compaction");
+ public static final ConfigProperty<String> ENABLE_LOG_COMPACTION =
ConfigProperty
+ .key("hoodie.log.compaction.enable")
+ .defaultValue("false")
+ .sinceVersion("0.14")
+ .withDocumentation("By enabling log compaction through this config, log
compaction will also gets enabled to metadata table.");
Review Comment:
```suggestion
.withDocumentation("By enabling log compaction through this config,
log compaction will also get enabled for the metadata table.");
```
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandleFactory.java:
##########
@@ -47,6 +50,7 @@ public static <T, I, K, O> HoodieMergeHandle<T, I, K, O>
create(
String fileId,
TaskContextSupplier taskContextSupplier,
Option<BaseKeyGenerator> keyGeneratorOpt) {
+ LOG.info("Get updateHandle for fileId " + fileId + " and partitionPath " +
partitionPath + " at commit " + instantTime);
Review Comment:
Are these logs really necessary? If so, please consider logging in debug
mode. Same for all logs.
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##########
@@ -1021,17 +1023,46 @@ private void
runPendingTableServicesOperations(BaseHoodieWriteClient writeClient
* deltacommit.
*/
protected void compactIfNecessary(BaseHoodieWriteClient writeClient, String
latestDeltacommitTime) {
+
+ // Check if there are any pending compaction or log compaction instants in
the timeline.
+ // If pending compact/logcompaction operations are found abort scheduling
new compaction/logcompaction operations.
+ Option<HoodieInstant> pendingLogCompactionInstant =
+
metadataMetaClient.getActiveTimeline().filterPendingLogCompactionTimeline().firstInstant();
+ Option<HoodieInstant> pendingCompactionInstant =
+
metadataMetaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant();
+ if (pendingLogCompactionInstant.isPresent() ||
pendingCompactionInstant.isPresent()) {
Review Comment:
this validation can be moved inside
"validateTimelineBeforeSchedulingCompaction"
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##########
@@ -1021,17 +1023,46 @@ private void
runPendingTableServicesOperations(BaseHoodieWriteClient writeClient
* deltacommit.
*/
protected void compactIfNecessary(BaseHoodieWriteClient writeClient, String
latestDeltacommitTime) {
+
Review Comment:
nit: remove newline
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java:
##########
@@ -109,4 +115,5 @@ public interface HoodieTableMetadataWriter extends
Serializable, AutoCloseable {
* deciding if optimizations can be
performed.
*/
void performTableServices(Option<String> inFlightInstantTimestamp);
+
Review Comment:
nit: remove newline
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/BaseHoodieCompactionPlanGenerator.java:
##########
@@ -82,6 +84,7 @@ public HoodieCompactionPlan generateCompactionPlan() throws
IOException {
// filter the partition paths if needed to reduce list status
partitionPaths = filterPartitionPathsByStrategy(writeConfig,
partitionPaths);
+ LOG.info("Filtered partition paths are " + partitionPaths);
Review Comment:
```suggestion
LOG.debug("Filtered partition paths are " + partitionPaths);
```
##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestDataValidationCheckForLogCompactionActions.java:
##########
@@ -377,7 +377,7 @@ private TestTableContents setupTestTable2() throws
IOException {
// Create logcompaction client.
HoodieWriteConfig logCompactionConfig =
HoodieWriteConfig.newBuilder().withProps(config2.getProps())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
- .withLogCompactionBlocksThreshold("2").build())
+ .withLogCompactionBlocksThreshold(2).build())
Review Comment:
good catch!
##########
hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestStreamWriteOperatorCoordinator.java:
##########
@@ -233,48 +233,49 @@ void testSyncMetadataTable() throws Exception {
assertThat(completedTimeline.lastInstant().get().getTimestamp(),
startsWith(HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP));
// test metadata table compaction
- // write another 4 commits
- for (int i = 1; i < 5; i++) {
+ // write another 9 commits to trigger compaction twice. Since default
clean version to retain is 2.
Review Comment:
would rather prefer to set the clean version to retain as 1. Doing 9 commits
is going to add to test time which is already quite high.
##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##########
@@ -1325,6 +1332,39 @@ public static Set<String>
getInflightAndCompletedMetadataPartitions(HoodieTableC
return inflightAndCompletedPartitions;
}
+ public static Set<String> getValidInstantTimestamps(HoodieTableMetaClient
dataMetaClient,
+ HoodieTableMetaClient
metadataMetaClient) {
+ // Only those log files which have a corresponding completed instant on
the dataset should be read
+ // This is because the metadata table is updated before the dataset
instants are committed.
+ HoodieActiveTimeline datasetTimeline = dataMetaClient.getActiveTimeline();
+ Set<String> validInstantTimestamps =
datasetTimeline.filterCompletedInstants().getInstantsAsStream()
+ .map(HoodieInstant::getTimestamp).collect(Collectors.toSet());
+
+ // We should also add completed indexing delta commits in the metadata
table, as they do not
+ // have corresponding completed instant in the data table
+ validInstantTimestamps.addAll(
+ metadataMetaClient.getActiveTimeline()
+ .filter(instant -> instant.isCompleted()
+ && (isIndexingCommit(instant.getTimestamp()) ||
isLogCompactionInstant(instant)))
+ .getInstantsAsStream()
+ .map(HoodieInstant::getTimestamp)
+ .collect(Collectors.toList()));
+
+ // For any rollbacks and restores, we cannot neglect the instants that
they are rolling back.
+ // The rollback instant should be more recent than the start of the
timeline for it to have rolled back any
+ // instant which we have a log block for.
+ final String earliestInstantTime = validInstantTimestamps.isEmpty() ?
SOLO_COMMIT_TIMESTAMP : Collections.min(validInstantTimestamps);
+
datasetTimeline.getRollbackAndRestoreTimeline().filterCompletedInstants().getInstantsAsStream()
+ .filter(instant ->
HoodieTimeline.compareTimestamps(instant.getTimestamp(),
HoodieTimeline.GREATER_THAN, earliestInstantTime))
+ .forEach(instant -> {
+ validInstantTimestamps.addAll(getRollbackedCommits(instant,
datasetTimeline));
+ });
+
+ // SOLO_COMMIT_TIMESTAMP is used during bootstrap so it is a valid
timestamp
+ validInstantTimestamps.add(SOLO_COMMIT_TIMESTAMP);
Review Comment:
But any MDT partition initialization instant timestamp is of the format
SOLO_COMMIT_TIMESTAMP + 3-digit suffix, e.g. `00000000000000010` for FILES
partition. How to account for the suffix?
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandleFactory.java:
##########
@@ -79,6 +83,7 @@ public static <T, I, K, O> HoodieMergeHandle<T, I, K, O>
create(
HoodieBaseFile dataFileToBeMerged,
TaskContextSupplier taskContextSupplier,
Option<BaseKeyGenerator> keyGeneratorOpt) {
+ LOG.info("Get updateHandle for fileId " + fileId + " and partitionPath " +
partitionPath + " at commit " + instantTime);
Review Comment:
tip: with slf4j, you can aso log as
`LOG.info("Get updateHandle for fileId {} and partitionPath {} at commit
{}", fileId, partitionPath, instantTime);`
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java:
##########
@@ -97,6 +98,11 @@ public interface HoodieTableMetadataWriter extends
Serializable, AutoCloseable {
*/
BaseHoodieWriteClient getWriteClient();
+ /**
+ * It returns write client for metadata table.
+ */
+ HoodieTableMetaClient getMetadataMetaClient();
Review Comment:
not used anywhere.. let's remove it?
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java:
##########
@@ -96,4 +101,20 @@ public void completeInflightLogCompaction(HoodieTable
table, String logCompactio
"Failed to commit " + table.getMetaClient().getBasePath() + " at
time " + logCompactionCommitTime, e);
}
}
+
+ public Option<InstantRange> getInstantRange(HoodieTableMetaClient
metaClient) {
+ return
HoodieTableMetadata.isMetadataTable(metaClient.getBasePathV2().toString())
+ ? Option.of(getMetadataLogReaderInstantRange(metaClient)) :
Option.empty();
+ }
+
+ private InstantRange getMetadataLogReaderInstantRange(HoodieTableMetaClient
metadataMetaClient) {
+ HoodieTableMetaClient dataMetaClient = HoodieTableMetaClient.builder()
+ .setConf(metadataMetaClient.getHadoopConf())
+
.setBasePath(HoodieTableMetadata.getDatasetBasePath(metadataMetaClient.getBasePath()))
Review Comment:
```suggestion
.setBasePath(HoodieTableMetadata.getDatasetBasePath(metadataMetaClient.getBasePathV2().toString()))
```
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##########
@@ -1021,17 +1023,46 @@ private void
runPendingTableServicesOperations(BaseHoodieWriteClient writeClient
* deltacommit.
*/
protected void compactIfNecessary(BaseHoodieWriteClient writeClient, String
latestDeltacommitTime) {
+
+ // Check if there are any pending compaction or log compaction instants in
the timeline.
+ // If pending compact/logcompaction operations are found abort scheduling
new compaction/logcompaction operations.
+ Option<HoodieInstant> pendingLogCompactionInstant =
+
metadataMetaClient.getActiveTimeline().filterPendingLogCompactionTimeline().firstInstant();
+ Option<HoodieInstant> pendingCompactionInstant =
+
metadataMetaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant();
+ if (pendingLogCompactionInstant.isPresent() ||
pendingCompactionInstant.isPresent()) {
+ LOG.info(String.format("Not scheduling compaction or logcompaction,
since a pending compaction instant %s or logcompaction %s instant is present",
Review Comment:
change log to `warn` level?
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataWriteUtils.java:
##########
@@ -111,6 +112,10 @@ public static HoodieWriteConfig createMetadataWriteConfig(
// deltacommits having corresponding completed commits. Therefore,
we need to compact all fileslices of all
// partitions together requiring UnBoundedCompactionStrategy.
.withCompactionStrategy(new UnBoundedCompactionStrategy())
+ // Check if log compaction is enabled, this is needed for tables
with lot of records.
+ .withLogCompactionEnabled(writeConfig.isLogCompactionEnabled())
+ // This config is only used if enableLogCompactionForMetadata is
set.
Review Comment:
i think it's a typo. i didn't find any method for
`enableLogCompactionForMetadata`
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]