1
0

[HUDI-296] Explore use of spotless to auto fix formatting errors (#945)

- Add spotless format fixing to project
- One time reformatting for conformity
- Build fails for formatting changes and mvn spotless:apply autofixes them
This commit is contained in:
leesf
2019-10-10 20:19:40 +08:00
committed by vinoth chandar
parent 834c591955
commit b19bed442d
381 changed files with 7350 additions and 9064 deletions

View File

@@ -81,15 +81,14 @@ public class FileSystemViewHandler {
*/
private boolean isLocalViewBehind(Context ctx) {
String basePath = ctx.queryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM);
String lastKnownInstantFromClient = ctx
.queryParam(RemoteHoodieTableFileSystemView.LAST_INSTANT_TS, HoodieTimeline.INVALID_INSTANT_TS);
String lastKnownInstantFromClient =
ctx.queryParam(RemoteHoodieTableFileSystemView.LAST_INSTANT_TS, HoodieTimeline.INVALID_INSTANT_TS);
String timelineHashFromClient = ctx.queryParam(RemoteHoodieTableFileSystemView.TIMELINE_HASH, "");
HoodieTimeline localTimeline = viewManager.getFileSystemView(basePath).getTimeline()
.filterCompletedAndCompactionInstants();
HoodieTimeline localTimeline =
viewManager.getFileSystemView(basePath).getTimeline().filterCompletedAndCompactionInstants();
if (logger.isDebugEnabled()) {
logger.debug("Client [ LastTs=" + lastKnownInstantFromClient
+ ", TimelineHash=" + timelineHashFromClient + "], localTimeline="
+ localTimeline.getInstants().collect(Collectors.toList()));
logger.debug("Client [ LastTs=" + lastKnownInstantFromClient + ", TimelineHash=" + timelineHashFromClient
+ "], localTimeline=" + localTimeline.getInstants().collect(Collectors.toList()));
}
if ((localTimeline.getInstants().count() == 0)
@@ -132,8 +131,8 @@ public class FileSystemViewHandler {
private void writeValueAsString(Context ctx, Object obj) throws JsonProcessingException {
boolean prettyPrint = ctx.queryParam("pretty") != null ? true : false;
long beginJsonTs = System.currentTimeMillis();
String result = prettyPrint ? mapper.writerWithDefaultPrettyPrinter().writeValueAsString(obj)
: mapper.writeValueAsString(obj);
String result =
prettyPrint ? mapper.writerWithDefaultPrettyPrinter().writeValueAsString(obj) : mapper.writeValueAsString(obj);
long endJsonTs = System.currentTimeMillis();
logger.debug("Jsonify TimeTaken=" + (endJsonTs - beginJsonTs));
ctx.result(result);
@@ -144,14 +143,14 @@ public class FileSystemViewHandler {
*/
private void registerTimelineAPI() {
app.get(RemoteHoodieTableFileSystemView.LAST_INSTANT, new ViewHandler(ctx -> {
List<InstantDTO> dtos = instantHandler.getLastInstant(ctx.validatedQueryParam(
RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getValue());
List<InstantDTO> dtos = instantHandler
.getLastInstant(ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getValue());
writeValueAsString(ctx, dtos);
}, false));
app.get(RemoteHoodieTableFileSystemView.TIMELINE, new ViewHandler(ctx -> {
TimelineDTO dto = instantHandler.getTimeline(ctx.validatedQueryParam(
RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getValue());
TimelineDTO dto = instantHandler
.getTimeline(ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getValue());
writeValueAsString(ctx, dto);
}, false));
}
@@ -161,62 +160,54 @@ public class FileSystemViewHandler {
*/
private void registerDataFilesAPI() {
app.get(RemoteHoodieTableFileSystemView.LATEST_PARTITION_DATA_FILES_URL, new ViewHandler(ctx -> {
List<DataFileDTO> dtos =
dataFileHandler
.getLatestDataFiles(ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow());
List<DataFileDTO> dtos = dataFileHandler.getLatestDataFiles(
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow());
writeValueAsString(ctx, dtos);
}, true));
app.get(RemoteHoodieTableFileSystemView.LATEST_PARTITION_DATA_FILE_URL, new ViewHandler(ctx -> {
List<DataFileDTO> dtos =
dataFileHandler
.getLatestDataFile(ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.FILEID_PARAM).getOrThrow());
List<DataFileDTO> dtos = dataFileHandler.getLatestDataFile(
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.FILEID_PARAM).getOrThrow());
writeValueAsString(ctx, dtos);
}, true));
app.get(RemoteHoodieTableFileSystemView.LATEST_ALL_DATA_FILES, new ViewHandler(ctx -> {
List<DataFileDTO> dtos =
dataFileHandler
.getLatestDataFiles(ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow());
List<DataFileDTO> dtos = dataFileHandler
.getLatestDataFiles(ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow());
writeValueAsString(ctx, dtos);
}, true));
app.get(RemoteHoodieTableFileSystemView.LATEST_DATA_FILES_BEFORE_ON_INSTANT_URL, new ViewHandler(ctx -> {
List<DataFileDTO> dtos =
dataFileHandler.getLatestDataFilesBeforeOrOn(ctx.validatedQueryParam(
RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.MAX_INSTANT_PARAM).getOrThrow());
List<DataFileDTO> dtos = dataFileHandler.getLatestDataFilesBeforeOrOn(
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.MAX_INSTANT_PARAM).getOrThrow());
writeValueAsString(ctx, dtos);
}, true));
app.get(RemoteHoodieTableFileSystemView.LATEST_DATA_FILE_ON_INSTANT_URL, new ViewHandler(ctx -> {
List<DataFileDTO> dtos =
dataFileHandler
.getLatestDataFileOn(ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow(),
ctx.queryParam(RemoteHoodieTableFileSystemView.INSTANT_PARAM), ctx.validatedQueryParam(
RemoteHoodieTableFileSystemView.FILEID_PARAM).getOrThrow());
List<DataFileDTO> dtos = dataFileHandler.getLatestDataFileOn(
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow(),
ctx.queryParam(RemoteHoodieTableFileSystemView.INSTANT_PARAM),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.FILEID_PARAM).getOrThrow());
writeValueAsString(ctx, dtos);
}, true));
app.get(RemoteHoodieTableFileSystemView.ALL_DATA_FILES, new ViewHandler(ctx -> {
List<DataFileDTO> dtos =
dataFileHandler
.getAllDataFiles(ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow());
List<DataFileDTO> dtos = dataFileHandler.getAllDataFiles(
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow());
writeValueAsString(ctx, dtos);
}, true));
app.get(RemoteHoodieTableFileSystemView.LATEST_DATA_FILES_RANGE_INSTANT_URL, new ViewHandler(ctx -> {
List<DataFileDTO> dtos =
dataFileHandler.getLatestDataFilesInRange(ctx.validatedQueryParam(
RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
Arrays.asList(
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.INSTANTS_PARAM).getOrThrow().split(",")));
List<DataFileDTO> dtos = dataFileHandler.getLatestDataFilesInRange(
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(), Arrays
.asList(ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.INSTANTS_PARAM).getOrThrow().split(",")));
writeValueAsString(ctx, dtos);
}, true));
}
@@ -226,64 +217,57 @@ public class FileSystemViewHandler {
*/
private void registerFileSlicesAPI() {
app.get(RemoteHoodieTableFileSystemView.LATEST_PARTITION_SLICES_URL, new ViewHandler(ctx -> {
List<FileSliceDTO> dtos =
sliceHandler
.getLatestFileSlices(ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow());
List<FileSliceDTO> dtos = sliceHandler.getLatestFileSlices(
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow());
writeValueAsString(ctx, dtos);
}, true));
app.get(RemoteHoodieTableFileSystemView.LATEST_PARTITION_SLICE_URL, new ViewHandler(ctx -> {
List<FileSliceDTO> dtos =
sliceHandler
.getLatestFileSlice(ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.FILEID_PARAM).getOrThrow());
List<FileSliceDTO> dtos = sliceHandler.getLatestFileSlice(
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.FILEID_PARAM).getOrThrow());
writeValueAsString(ctx, dtos);
}, true));
app.get(RemoteHoodieTableFileSystemView.LATEST_PARTITION_UNCOMPACTED_SLICES_URL, new ViewHandler(ctx -> {
List<FileSliceDTO> dtos =
sliceHandler.getLatestUnCompactedFileSlices(ctx.validatedQueryParam(
RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow());
List<FileSliceDTO> dtos = sliceHandler.getLatestUnCompactedFileSlices(
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow());
writeValueAsString(ctx, dtos);
}, true));
app.get(RemoteHoodieTableFileSystemView.ALL_SLICES_URL, new ViewHandler(ctx -> {
List<FileSliceDTO> dtos =
sliceHandler
.getAllFileSlices(ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow());
List<FileSliceDTO> dtos = sliceHandler.getAllFileSlices(
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow());
writeValueAsString(ctx, dtos);
}, true));
app.get(RemoteHoodieTableFileSystemView.LATEST_SLICES_RANGE_INSTANT_URL, new ViewHandler(ctx -> {
List<FileSliceDTO> dtos =
sliceHandler.getLatestFileSliceInRange(
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
Arrays.asList(
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.INSTANTS_PARAM).getOrThrow().split(",")));
List<FileSliceDTO> dtos = sliceHandler.getLatestFileSliceInRange(
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(), Arrays
.asList(ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.INSTANTS_PARAM).getOrThrow().split(",")));
writeValueAsString(ctx, dtos);
}, true));
app.get(RemoteHoodieTableFileSystemView.LATEST_SLICES_MERGED_BEFORE_ON_INSTANT_URL, new ViewHandler(ctx -> {
List<FileSliceDTO> dtos =
sliceHandler.getLatestMergedFileSlicesBeforeOrOn(ctx.validatedQueryParam(
RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.MAX_INSTANT_PARAM).getOrThrow());
List<FileSliceDTO> dtos = sliceHandler.getLatestMergedFileSlicesBeforeOrOn(
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.MAX_INSTANT_PARAM).getOrThrow());
writeValueAsString(ctx, dtos);
}, true));
app.get(RemoteHoodieTableFileSystemView.LATEST_SLICES_BEFORE_ON_INSTANT_URL, new ViewHandler(ctx -> {
List<FileSliceDTO> dtos =
sliceHandler.getLatestFileSlicesBeforeOrOn(ctx.validatedQueryParam(
RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.MAX_INSTANT_PARAM).getOrThrow(),
Boolean.valueOf(ctx.validatedQueryParam(
RemoteHoodieTableFileSystemView.INCLUDE_FILES_IN_PENDING_COMPACTION_PARAM).getOrThrow()));
List<FileSliceDTO> dtos = sliceHandler.getLatestFileSlicesBeforeOrOn(
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.MAX_INSTANT_PARAM).getOrThrow(),
Boolean.valueOf(
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.INCLUDE_FILES_IN_PENDING_COMPACTION_PARAM)
.getOrThrow()));
writeValueAsString(ctx, dtos);
}, true));
@@ -294,15 +278,15 @@ public class FileSystemViewHandler {
}, true));
app.get(RemoteHoodieTableFileSystemView.ALL_FILEGROUPS_FOR_PARTITION_URL, new ViewHandler(ctx -> {
List<FileGroupDTO> dtos = sliceHandler.getAllFileGroups(ctx.validatedQueryParam(
RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
List<FileGroupDTO> dtos = sliceHandler.getAllFileGroups(
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(),
ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM).getOrThrow());
writeValueAsString(ctx, dtos);
}, true));
app.post(RemoteHoodieTableFileSystemView.REFRESH_DATASET, new ViewHandler(ctx -> {
boolean success = sliceHandler.refreshDataset(ctx.validatedQueryParam(
RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow());
boolean success = sliceHandler
.refreshDataset(ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow());
writeValueAsString(ctx, success);
}, false));
}
@@ -348,11 +332,13 @@ public class FileSystemViewHandler {
if (refreshCheck) {
long beginFinalCheck = System.currentTimeMillis();
String errMsg = "Last known instant from client was "
+ context.queryParam(RemoteHoodieTableFileSystemView.LAST_INSTANT_TS, HoodieTimeline.INVALID_INSTANT_TS)
+ " but server has the following timeline "
+ viewManager.getFileSystemView(context.queryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM))
.getTimeline().getInstants().collect(Collectors.toList());
String errMsg =
"Last known instant from client was "
+ context.queryParam(RemoteHoodieTableFileSystemView.LAST_INSTANT_TS,
HoodieTimeline.INVALID_INSTANT_TS)
+ " but server has the following timeline "
+ viewManager.getFileSystemView(context.queryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM))
.getTimeline().getInstants().collect(Collectors.toList());
Preconditions.checkArgument(!isLocalViewBehind(context), errMsg);
long endFinalCheck = System.currentTimeMillis();
finalCheckTimeTaken = endFinalCheck - beginFinalCheck;
@@ -364,9 +350,12 @@ public class FileSystemViewHandler {
} finally {
long endTs = System.currentTimeMillis();
long timeTakenMillis = endTs - beginTs;
logger.info(String.format("TimeTakenMillis[Total=%d, Refresh=%d, handle=%d, Check=%d], "
+ "Success=%s, Query=%s, Host=%s, synced=%s", timeTakenMillis, refreshCheckTimeTaken, handleTimeTaken,
finalCheckTimeTaken, success, context.queryString(), context.host(), synced));
logger
.info(String.format(
"TimeTakenMillis[Total=%d, Refresh=%d, handle=%d, Check=%d], "
+ "Success=%s, Query=%s, Host=%s, synced=%s",
timeTakenMillis, refreshCheckTimeTaken, handleTimeTaken, finalCheckTimeTaken, success,
context.queryString(), context.host(), synced));
}
}
}

View File

@@ -50,8 +50,8 @@ public class TimelineService {
return serverPort;
}
public TimelineService(int serverPort, FileSystemViewManager globalFileSystemViewManager,
Configuration conf) throws IOException {
public TimelineService(int serverPort, FileSystemViewManager globalFileSystemViewManager, Configuration conf)
throws IOException {
this.conf = FSUtils.prepareHadoopConf(conf);
this.fs = FileSystem.get(conf);
this.serverPort = serverPort;
@@ -89,8 +89,7 @@ public class TimelineService {
description = "Directory where spilled view entries will be stored. Used for SPILLABLE_DISK storage type")
public String baseStorePathForFileGroups = FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR;
@Parameter(names = {"--rocksdb-path", "-rp"},
description = "Root directory for RocksDB")
@Parameter(names = {"--rocksdb-path", "-rp"}, description = "Root directory for RocksDB")
public String rocksDBPath = FileSystemViewStorageConfig.DEFAULT_ROCKSDB_BASE_PATH;
@Parameter(names = {"--help", "-h"})

View File

@@ -32,8 +32,7 @@ import org.apache.hudi.common.table.view.FileSystemViewManager;
*/
public class DataFileHandler extends Handler {
public DataFileHandler(Configuration conf,
FileSystemViewManager viewManager) throws IOException {
public DataFileHandler(Configuration conf, FileSystemViewManager viewManager) throws IOException {
super(conf, viewManager);
}
@@ -48,8 +47,8 @@ public class DataFileHandler extends Handler {
}
public List<DataFileDTO> getLatestDataFiles(String basePath) {
return viewManager.getFileSystemView(basePath).getLatestDataFiles()
.map(DataFileDTO::fromHoodieDataFile).collect(Collectors.toList());
return viewManager.getFileSystemView(basePath).getLatestDataFiles().map(DataFileDTO::fromHoodieDataFile)
.collect(Collectors.toList());
}
public List<DataFileDTO> getLatestDataFilesBeforeOrOn(String basePath, String partitionPath, String maxInstantTime) {
@@ -71,8 +70,8 @@ public class DataFileHandler extends Handler {
}
public List<DataFileDTO> getAllDataFiles(String basePath, String partitionPath) {
return viewManager.getFileSystemView(basePath).getAllDataFiles(partitionPath)
.map(DataFileDTO::fromHoodieDataFile).collect(Collectors.toList());
return viewManager.getFileSystemView(basePath).getAllDataFiles(partitionPath).map(DataFileDTO::fromHoodieDataFile)
.collect(Collectors.toList());
}
}

View File

@@ -34,8 +34,7 @@ import org.apache.hudi.common.table.view.FileSystemViewManager;
*/
public class FileSliceHandler extends Handler {
public FileSliceHandler(Configuration conf,
FileSystemViewManager viewManager) throws IOException {
public FileSliceHandler(Configuration conf, FileSystemViewManager viewManager) throws IOException {
super(conf, viewManager);
}
@@ -49,16 +48,17 @@ public class FileSliceHandler extends Handler {
.map(FileSliceDTO::fromFileSlice).collect(Collectors.toList());
}
public List<FileSliceDTO> getLatestMergedFileSlicesBeforeOrOn(String basePath,
String partitionPath, String maxInstantTime) {
public List<FileSliceDTO> getLatestMergedFileSlicesBeforeOrOn(String basePath, String partitionPath,
String maxInstantTime) {
return viewManager.getFileSystemView(basePath).getLatestMergedFileSlicesBeforeOrOn(partitionPath, maxInstantTime)
.map(FileSliceDTO::fromFileSlice).collect(Collectors.toList());
}
public List<FileSliceDTO> getLatestFileSlicesBeforeOrOn(String basePath, String partitionPath,
String maxInstantTime, boolean includeFileSlicesInPendingCompaction) {
return viewManager.getFileSystemView(basePath).getLatestFileSlicesBeforeOrOn(partitionPath, maxInstantTime,
includeFileSlicesInPendingCompaction).map(FileSliceDTO::fromFileSlice).collect(Collectors.toList());
public List<FileSliceDTO> getLatestFileSlicesBeforeOrOn(String basePath, String partitionPath, String maxInstantTime,
boolean includeFileSlicesInPendingCompaction) {
return viewManager.getFileSystemView(basePath)
.getLatestFileSlicesBeforeOrOn(partitionPath, maxInstantTime, includeFileSlicesInPendingCompaction)
.map(FileSliceDTO::fromFileSlice).collect(Collectors.toList());
}
public List<FileSliceDTO> getLatestUnCompactedFileSlices(String basePath, String partitionPath) {
@@ -67,8 +67,8 @@ public class FileSliceHandler extends Handler {
}
public List<FileSliceDTO> getLatestFileSlices(String basePath, String partitionPath) {
return viewManager.getFileSystemView(basePath).getLatestFileSlices(partitionPath)
.map(FileSliceDTO::fromFileSlice).collect(Collectors.toList());
return viewManager.getFileSystemView(basePath).getLatestFileSlices(partitionPath).map(FileSliceDTO::fromFileSlice)
.collect(Collectors.toList());
}
public List<FileSliceDTO> getLatestFileSlice(String basePath, String partitionPath, String fileId) {
@@ -83,8 +83,8 @@ public class FileSliceHandler extends Handler {
}
public List<FileGroupDTO> getAllFileGroups(String basePath, String partitionPath) {
return viewManager.getFileSystemView(basePath).getAllFileGroups(partitionPath)
.map(FileGroupDTO::fromFileGroup).collect(Collectors.toList());
return viewManager.getFileSystemView(basePath).getAllFileGroups(partitionPath).map(FileGroupDTO::fromFileGroup)
.collect(Collectors.toList());
}
public boolean refreshDataset(String basePath) {

View File

@@ -32,14 +32,13 @@ import org.apache.hudi.common.table.view.FileSystemViewManager;
*/
public class TimelineHandler extends Handler {
public TimelineHandler(Configuration conf,
FileSystemViewManager viewManager) throws IOException {
public TimelineHandler(Configuration conf, FileSystemViewManager viewManager) throws IOException {
super(conf, viewManager);
}
public List<InstantDTO> getLastInstant(String basePath) {
return viewManager.getFileSystemView(basePath).getLastInstant()
.map(InstantDTO::fromInstant).map(dto -> Arrays.asList(dto)).orElse(new ArrayList<>());
return viewManager.getFileSystemView(basePath).getLastInstant().map(InstantDTO::fromInstant)
.map(dto -> Arrays.asList(dto)).orElse(new ArrayList<>());
}
public TimelineDTO getTimeline(String basePath) {

View File

@@ -41,13 +41,11 @@ public class RemoteHoodieTableFileSystemViewTest extends HoodieTableFileSystemVi
private RemoteHoodieTableFileSystemView view;
protected SyncableFileSystemView getFileSystemView(HoodieTimeline timeline) {
FileSystemViewStorageConfig sConf = FileSystemViewStorageConfig.newBuilder()
.withStorageType(FileSystemViewStorageType.SPILLABLE_DISK).build();
FileSystemViewStorageConfig sConf =
FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.SPILLABLE_DISK).build();
try {
server =
new TimelineService(0,
FileSystemViewManager.createViewManager(new SerializableConfiguration(metaClient.getHadoopConf()),
sConf));
server = new TimelineService(0,
FileSystemViewManager.createViewManager(new SerializableConfiguration(metaClient.getHadoopConf()), sConf));
server.startService();
} catch (Exception ex) {
throw new RuntimeException(ex);