[HUDI-990] Timeline API : filterCompletedAndCompactionInstants needs to handle requested state correctly. Also ensure timeline gets reloaded after we revert committed transactions
This commit is contained in:
committed by
Balaji Varadarajan
parent
a68180b179
commit
fb283934a3
@@ -89,7 +89,9 @@ public class EmbeddedTimelineService {
|
||||
* Retrieves proper view storage configs for remote clients to access this service.
|
||||
*/
|
||||
public FileSystemViewStorageConfig getRemoteFileSystemViewConfig() {
|
||||
return FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.REMOTE_FIRST)
|
||||
FileSystemViewStorageType viewStorageType = config.shouldEnableBackupForRemoteFileSystemView()
|
||||
? FileSystemViewStorageType.REMOTE_FIRST : FileSystemViewStorageType.REMOTE_ONLY;
|
||||
return FileSystemViewStorageConfig.newBuilder().withStorageType(viewStorageType)
|
||||
.withRemoteServerHost(hostAddr).withRemoteServerPort(serverPort).build();
|
||||
}
|
||||
|
||||
|
||||
@@ -66,6 +66,8 @@ public class CopyOnWriteRollbackActionExecutor extends BaseRollbackActionExecuto
|
||||
if (instantToRollback.isCompleted()) {
|
||||
LOG.info("Unpublishing instant " + instantToRollback);
|
||||
resolvedInstant = activeTimeline.revertToInflight(instantToRollback);
|
||||
// reload meta-client to reflect latest timeline status
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
}
|
||||
|
||||
// For Requested State (like failure during index lookup), there is nothing to do rollback other than
|
||||
|
||||
@@ -74,6 +74,8 @@ public class MergeOnReadRollbackActionExecutor extends BaseRollbackActionExecuto
|
||||
if (instantToRollback.isCompleted()) {
|
||||
LOG.error("Un-publishing instant " + instantToRollback + ", deleteInstants=" + deleteInstants);
|
||||
resolvedInstant = table.getActiveTimeline().revertToInflight(instantToRollback);
|
||||
// reload meta-client to reflect latest timeline status
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
}
|
||||
|
||||
List<HoodieRollbackStat> allRollbackStats = new ArrayList<>();
|
||||
|
||||
@@ -35,6 +35,7 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.table.view.SyncableFileSystemView;
|
||||
import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
|
||||
@@ -1327,7 +1328,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
*/
|
||||
newCommitTime = "002";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
writeRecords = jsc.parallelize(records, 1);
|
||||
statuses = client.upsert(writeRecords, newCommitTime).collect();
|
||||
@@ -1340,6 +1341,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
final String partitionPath = records.get(0).getPartitionPath();
|
||||
final String fileId = statuses.get(0).getFileId();
|
||||
client.startCommitWithTime(newDeleteTime);
|
||||
metaClient.reloadActiveTimeline();
|
||||
|
||||
List<HoodieRecord> fewRecordsForDelete = dataGen.generateDeletesFromExistingRecords(records);
|
||||
JavaRDD<HoodieRecord> deleteRDD = jsc.parallelize(fewRecordsForDelete, 1);
|
||||
@@ -1377,6 +1379,8 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
.withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024 * 1024).build())
|
||||
.withEmbeddedTimelineServerEnabled(true).forTable("test-trip-table")
|
||||
.withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder()
|
||||
.withEnableBackupForRemoteFileSystemView(false).build())
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build());
|
||||
}
|
||||
|
||||
|
||||
@@ -132,6 +132,7 @@ public class HoodieClientTestBase extends HoodieClientTestHarness {
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build())
|
||||
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||
.withEnableBackupForRemoteFileSystemView(false) // Fail test if problem connecting to timeline-server
|
||||
.withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build());
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user