1
0

[HUDI-895] Remove unnecessary listing .hoodie folder when using timeline server (#1636)

This commit is contained in:
Balaji Varadarajan
2020-05-17 18:18:53 -07:00
committed by GitHub
parent 25a0080b2f
commit 3c9da2e5f0
3 changed files with 40 additions and 31 deletions

View File

@@ -540,10 +540,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
HoodieTimeline.compareTimestamps(latestPending.getTimestamp(), HoodieTimeline.LESSER_THAN, instantTime), HoodieTimeline.compareTimestamps(latestPending.getTimestamp(), HoodieTimeline.LESSER_THAN, instantTime),
"Latest pending compaction instant time must be earlier than this instant time. Latest Compaction :" "Latest pending compaction instant time must be earlier than this instant time. Latest Compaction :"
+ latestPending + ", Ingesting at " + instantTime)); + latestPending + ", Ingesting at " + instantTime));
HoodieTable<T> table = HoodieTable.create(metaClient, config, hadoopConf); metaClient.getActiveTimeline().createNewInstant(new HoodieInstant(State.REQUESTED, metaClient.getCommitActionType(),
HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); instantTime));
String commitActionType = table.getMetaClient().getCommitActionType();
activeTimeline.createNewInstant(new HoodieInstant(State.REQUESTED, commitActionType, instantTime));
} }
/** /**

View File

@@ -231,21 +231,21 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
* Get the base file only view of the file system for this table. * Get the base file only view of the file system for this table.
*/ */
public BaseFileOnlyView getBaseFileOnlyView() { public BaseFileOnlyView getBaseFileOnlyView() {
return getViewManager().getFileSystemView(metaClient.getBasePath()); return getViewManager().getFileSystemView(metaClient);
} }
/** /**
* Get the full view of the file system for this table. * Get the full view of the file system for this table.
*/ */
public SliceView getSliceView() { public SliceView getSliceView() {
return getViewManager().getFileSystemView(metaClient.getBasePath()); return getViewManager().getFileSystemView(metaClient);
} }
/** /**
* Get complete view of the file system for this table with ability to force sync. * Get complete view of the file system for this table with ability to force sync.
*/ */
public SyncableFileSystemView getHoodieView() { public SyncableFileSystemView getHoodieView() {
return getViewManager().getFileSystemView(metaClient.getBasePath()); return getViewManager().getFileSystemView(metaClient);
} }
/** /**

View File

@@ -58,10 +58,10 @@ public class FileSystemViewManager {
// Map from Base-Path to View // Map from Base-Path to View
private final ConcurrentHashMap<String, SyncableFileSystemView> globalViewMap; private final ConcurrentHashMap<String, SyncableFileSystemView> globalViewMap;
// Factory Map to create file-system views // Factory Map to create file-system views
private final Function2<String, FileSystemViewStorageConfig, SyncableFileSystemView> viewCreator; private final Function2<HoodieTableMetaClient, FileSystemViewStorageConfig, SyncableFileSystemView> viewCreator;
public FileSystemViewManager(SerializableConfiguration conf, FileSystemViewStorageConfig viewStorageConfig, public FileSystemViewManager(SerializableConfiguration conf, FileSystemViewStorageConfig viewStorageConfig,
Function2<String, FileSystemViewStorageConfig, SyncableFileSystemView> viewCreator) { Function2<HoodieTableMetaClient, FileSystemViewStorageConfig, SyncableFileSystemView> viewCreator) {
this.conf = new SerializableConfiguration(conf); this.conf = new SerializableConfiguration(conf);
this.viewStorageConfig = viewStorageConfig; this.viewStorageConfig = viewStorageConfig;
this.globalViewMap = new ConcurrentHashMap<>(); this.globalViewMap = new ConcurrentHashMap<>();
@@ -87,7 +87,21 @@ public class FileSystemViewManager {
* @return * @return
*/ */
public SyncableFileSystemView getFileSystemView(String basePath) { public SyncableFileSystemView getFileSystemView(String basePath) {
return globalViewMap.computeIfAbsent(basePath, (path) -> viewCreator.apply(path, viewStorageConfig)); return globalViewMap.computeIfAbsent(basePath, (path) -> {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf.newCopy(), path);
return viewCreator.apply(metaClient, viewStorageConfig);
});
}
/**
* Main API to get the file-system view for the base-path.
*
* @param metaClient HoodieTableMetaClient
* @return
*/
public SyncableFileSystemView getFileSystemView(HoodieTableMetaClient metaClient) {
return globalViewMap.computeIfAbsent(metaClient.getBasePath(),
(path) -> viewCreator.apply(metaClient, viewStorageConfig));
} }
/** /**
@@ -105,12 +119,11 @@ public class FileSystemViewManager {
* *
* @param conf Hadoop Configuration * @param conf Hadoop Configuration
* @param viewConf View Storage Configuration * @param viewConf View Storage Configuration
* @param basePath Base Path of table * @param metaClient HoodieTableMetaClient
* @return * @return
*/ */
private static RocksDbBasedFileSystemView createRocksDBBasedFileSystemView(SerializableConfiguration conf, private static RocksDbBasedFileSystemView createRocksDBBasedFileSystemView(SerializableConfiguration conf,
FileSystemViewStorageConfig viewConf, String basePath) { FileSystemViewStorageConfig viewConf, HoodieTableMetaClient metaClient) {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf.newCopy(), basePath, true);
HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants(); HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
return new RocksDbBasedFileSystemView(metaClient, timeline, viewConf); return new RocksDbBasedFileSystemView(metaClient, timeline, viewConf);
} }
@@ -120,13 +133,12 @@ public class FileSystemViewManager {
* *
* @param conf Hadoop Configuration * @param conf Hadoop Configuration
* @param viewConf View Storage Configuration * @param viewConf View Storage Configuration
* @param basePath Base Path of table * @param metaClient HoodieTableMetaClient
* @return * @return
*/ */
private static SpillableMapBasedFileSystemView createSpillableMapBasedFileSystemView(SerializableConfiguration conf, private static SpillableMapBasedFileSystemView createSpillableMapBasedFileSystemView(SerializableConfiguration conf,
FileSystemViewStorageConfig viewConf, String basePath) { FileSystemViewStorageConfig viewConf, HoodieTableMetaClient metaClient) {
LOG.info("Creating SpillableMap based view for basePath " + basePath); LOG.info("Creating SpillableMap based view for basePath " + metaClient.getBasePath());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf.newCopy(), basePath, true);
HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants(); HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
return new SpillableMapBasedFileSystemView(metaClient, timeline, viewConf); return new SpillableMapBasedFileSystemView(metaClient, timeline, viewConf);
} }
@@ -136,13 +148,12 @@ public class FileSystemViewManager {
* *
* @param conf Hadoop Configuration * @param conf Hadoop Configuration
* @param viewConf View Storage Configuration * @param viewConf View Storage Configuration
* @param basePath Base Path of table * @param metaClient HoodieTableMetaClient
* @return * @return
*/ */
private static HoodieTableFileSystemView createInMemoryFileSystemView(SerializableConfiguration conf, private static HoodieTableFileSystemView createInMemoryFileSystemView(SerializableConfiguration conf,
FileSystemViewStorageConfig viewConf, String basePath) { FileSystemViewStorageConfig viewConf, HoodieTableMetaClient metaClient) {
LOG.info("Creating InMemory based view for basePath " + basePath); LOG.info("Creating InMemory based view for basePath " + metaClient.getBasePath());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf.newCopy(), basePath, true);
HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants(); HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
return new HoodieTableFileSystemView(metaClient, timeline, viewConf.isIncrementalTimelineSyncEnabled()); return new HoodieTableFileSystemView(metaClient, timeline, viewConf.isIncrementalTimelineSyncEnabled());
} }
@@ -177,34 +188,34 @@ public class FileSystemViewManager {
case EMBEDDED_KV_STORE: case EMBEDDED_KV_STORE:
LOG.info("Creating embedded rocks-db based Table View"); LOG.info("Creating embedded rocks-db based Table View");
return new FileSystemViewManager(conf, config, return new FileSystemViewManager(conf, config,
(basePath, viewConf) -> createRocksDBBasedFileSystemView(conf, viewConf, basePath)); (metaClient, viewConf) -> createRocksDBBasedFileSystemView(conf, viewConf, metaClient));
case SPILLABLE_DISK: case SPILLABLE_DISK:
LOG.info("Creating Spillable Disk based Table View"); LOG.info("Creating Spillable Disk based Table View");
return new FileSystemViewManager(conf, config, return new FileSystemViewManager(conf, config,
(basePath, viewConf) -> createSpillableMapBasedFileSystemView(conf, viewConf, basePath)); (metaClient, viewConf) -> createSpillableMapBasedFileSystemView(conf, viewConf, metaClient));
case MEMORY: case MEMORY:
LOG.info("Creating in-memory based Table View"); LOG.info("Creating in-memory based Table View");
return new FileSystemViewManager(conf, config, return new FileSystemViewManager(conf, config,
(basePath, viewConfig) -> createInMemoryFileSystemView(conf, viewConfig, basePath)); (metaClient, viewConfig) -> createInMemoryFileSystemView(conf, viewConfig, metaClient));
case REMOTE_ONLY: case REMOTE_ONLY:
LOG.info("Creating remote only table view"); LOG.info("Creating remote only table view");
return new FileSystemViewManager(conf, config, (basePath, viewConfig) -> createRemoteFileSystemView(conf, return new FileSystemViewManager(conf, config, (metaClient, viewConfig) -> createRemoteFileSystemView(conf,
viewConfig, new HoodieTableMetaClient(conf.newCopy(), basePath))); viewConfig, metaClient));
case REMOTE_FIRST: case REMOTE_FIRST:
LOG.info("Creating remote first table view"); LOG.info("Creating remote first table view");
return new FileSystemViewManager(conf, config, (basePath, viewConfig) -> { return new FileSystemViewManager(conf, config, (metaClient, viewConfig) -> {
RemoteHoodieTableFileSystemView remoteFileSystemView = RemoteHoodieTableFileSystemView remoteFileSystemView =
createRemoteFileSystemView(conf, viewConfig, new HoodieTableMetaClient(conf.newCopy(), basePath)); createRemoteFileSystemView(conf, viewConfig, metaClient);
SyncableFileSystemView secondaryView; SyncableFileSystemView secondaryView;
switch (viewConfig.getSecondaryStorageType()) { switch (viewConfig.getSecondaryStorageType()) {
case MEMORY: case MEMORY:
secondaryView = createInMemoryFileSystemView(conf, viewConfig, basePath); secondaryView = createInMemoryFileSystemView(conf, viewConfig, metaClient);
break; break;
case EMBEDDED_KV_STORE: case EMBEDDED_KV_STORE:
secondaryView = createRocksDBBasedFileSystemView(conf, viewConfig, basePath); secondaryView = createRocksDBBasedFileSystemView(conf, viewConfig, metaClient);
break; break;
case SPILLABLE_DISK: case SPILLABLE_DISK:
secondaryView = createSpillableMapBasedFileSystemView(conf, viewConfig, basePath); secondaryView = createSpillableMapBasedFileSystemView(conf, viewConfig, metaClient);
break; break;
default: default:
throw new IllegalArgumentException("Secondary Storage type can only be in-memory or spillable. Was :" throw new IllegalArgumentException("Secondary Storage type can only be in-memory or spillable. Was :"