prashantwason commented on a change in pull request #3590:
URL: https://github.com/apache/hudi/pull/3590#discussion_r727634907
##########
File path:
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java
##########
@@ -206,6 +209,19 @@ private HoodieCleanMetadata runClean(HoodieTable<T, I, K,
O> table, HoodieInstan
}
}
+ /**
+ * Update metadata table if available. Any update to metadata table happens
within data table lock.
+ * @param cleanMetadata intance of {@link HoodieCleanMetadata} to be applied
to metadata.
+ */
+ private void writeMetadata(HoodieCleanMetadata cleanMetadata) {
Review comment:
Can we move the txnManager to the base class to save on code duplication
and having to write two levels of these functions?
writeMetadata
writeCleanMetadata
##########
File path:
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java
##########
@@ -99,83 +94,93 @@ protected void initialize(HoodieEngineContext
engineContext, HoodieTableMetaClie
@Override
protected void commit(List<HoodieRecord> records, String partitionName,
String instantTime) {
ValidationUtils.checkState(enabled, "Metadata table cannot be committed to
as it is not enabled");
- JavaRDD<HoodieRecord> recordRDD = prepRecords(records, partitionName);
+ JavaRDD<HoodieRecord> recordRDD = prepRecords(records, partitionName, 1);
try (SparkRDDWriteClient writeClient = new
SparkRDDWriteClient(engineContext, metadataWriteConfig, true)) {
- writeClient.startCommitWithTime(instantTime);
+ if
(!metadataMetaClient.getActiveTimeline().filterCompletedInstants().containsInstant(instantTime))
{
+ // if this is a new commit being applied to metadata for the first time
+ writeClient.startCommitWithTime(instantTime);
+ } else {
+ // this code path refers to a re-attempted commit that got committed
to metadata table, but failed in datatable.
+ // for eg, lets say compaction c1 on 1st attempt succeeded in metadata
table and failed before committing to datatable.
+ // when retried again, data table will first rollback pending
compaction. these will be applied to metadata table, but all changes
+ // are upserts to metadata table and so only a new delta commit will
be created.
+ // once rollback is complete, compaction will be retried again, which
will eventually hit this code block where the respective commit is
+ // already part of completed commit. So, we have to manually remove
the completed instant and proceed.
+ // and it is for the same reason we enabled
withAllowMultiWriteOnSameInstant for metadata table.
+ HoodieInstant alreadyCompletedInstant =
metadataMetaClient.getActiveTimeline().filterCompletedInstants().filter(entry
-> entry.getTimestamp().equals(instantTime)).lastInstant().get();
+ HoodieActiveTimeline.deleteInstantFile(metadataMetaClient.getFs(),
metadataMetaClient.getMetaPath(), alreadyCompletedInstant);
+ metadataMetaClient.reloadActiveTimeline();
Review comment:
This reload is extra as the next line 123 also reloads.
##########
File path:
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHBaseIndex.java
##########
@@ -422,7 +424,8 @@ public void testSimpleTagLocationWithInvalidCommit() throws
Exception {
@Test
public void testEnsureTagLocationUsesCommitTimeline() throws Exception {
// Load to memory
- HoodieWriteConfig config = getConfig();
+ HoodieWriteConfig config = getConfigBuilder(100, false, false)
+
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()).build();
Review comment:
this is used so often that it may be good to introduce a helper function
here:
withDisabledMetadata()
--
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]