Timeline Service with Incremental View Syncing support
This commit is contained in:
committed by
vinoth chandar
parent
446f99aa0f
commit
64fec64097
@@ -21,14 +21,17 @@ import com.uber.hoodie.avro.model.HoodieCompactionPlan;
|
||||
import com.uber.hoodie.avro.model.HoodieSavepointMetadata;
|
||||
import com.uber.hoodie.common.HoodieCleanStat;
|
||||
import com.uber.hoodie.common.HoodieRollbackStat;
|
||||
import com.uber.hoodie.common.SerializableConfiguration;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieWriteStat;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.SyncableFileSystemView;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewManager;
|
||||
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
|
||||
import com.uber.hoodie.common.util.AvroUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
@@ -62,12 +65,26 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
protected final HoodieTableMetaClient metaClient;
|
||||
protected final HoodieIndex<T> index;
|
||||
|
||||
private SerializableConfiguration hadoopConfiguration;
|
||||
private transient FileSystemViewManager viewManager;
|
||||
|
||||
protected HoodieTable(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
this.config = config;
|
||||
this.hadoopConfiguration = new SerializableConfiguration(jsc.hadoopConfiguration());
|
||||
this.viewManager = FileSystemViewManager.createViewManager(
|
||||
new SerializableConfiguration(jsc.hadoopConfiguration()), config.getViewStorageConfig());
|
||||
this.metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true);
|
||||
this.index = HoodieIndex.createIndex(config, jsc);
|
||||
}
|
||||
|
||||
private synchronized FileSystemViewManager getViewManager() {
|
||||
if (null == viewManager) {
|
||||
viewManager = FileSystemViewManager.createViewManager(hadoopConfiguration,
|
||||
config.getViewStorageConfig());
|
||||
}
|
||||
return viewManager;
|
||||
}
|
||||
|
||||
public static <T extends HoodieRecordPayload> HoodieTable<T> getHoodieTable(
|
||||
HoodieTableMetaClient metaClient, HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
switch (metaClient.getTableType()) {
|
||||
@@ -118,22 +135,21 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
* Get the read optimized view of the file system for this table
|
||||
*/
|
||||
public TableFileSystemView.ReadOptimizedView getROFileSystemView() {
|
||||
return new HoodieTableFileSystemView(metaClient, getCompletedCommitsTimeline());
|
||||
return getViewManager().getFileSystemView(metaClient.getBasePath());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the real time view of the file system for this table
|
||||
*/
|
||||
public TableFileSystemView.RealtimeView getRTFileSystemView() {
|
||||
return new HoodieTableFileSystemView(metaClient,
|
||||
metaClient.getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants());
|
||||
return getViewManager().getFileSystemView(metaClient.getBasePath());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the completed (commit + compaction) view of the file system for this table
|
||||
* Get complete view of the file system for this table with ability to force sync
|
||||
*/
|
||||
public TableFileSystemView getCompletedFileSystemView() {
|
||||
return new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline());
|
||||
public SyncableFileSystemView getHoodieView() {
|
||||
return getViewManager().getFileSystemView(metaClient.getBasePath());
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
Reference in New Issue
Block a user