Timeline Service with Incremental View Syncing support
This commit is contained in:
committed by
vinoth chandar
parent
446f99aa0f
commit
64fec64097
@@ -0,0 +1,351 @@
|
||||
/*
|
||||
* Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.timeline.service;
|
||||
|
||||
import static com.uber.hoodie.common.table.view.RemoteHoodieTableFileSystemView.*;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.SyncableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.dto.CompactionOpDTO;
|
||||
import com.uber.hoodie.common.table.timeline.dto.DataFileDTO;
|
||||
import com.uber.hoodie.common.table.timeline.dto.FileGroupDTO;
|
||||
import com.uber.hoodie.common.table.timeline.dto.FileSliceDTO;
|
||||
import com.uber.hoodie.common.table.timeline.dto.InstantDTO;
|
||||
import com.uber.hoodie.common.table.timeline.dto.TimelineDTO;
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewManager;
|
||||
import com.uber.hoodie.timeline.service.handlers.DataFileHandler;
|
||||
import com.uber.hoodie.timeline.service.handlers.FileSliceHandler;
|
||||
import com.uber.hoodie.timeline.service.handlers.TimelineHandler;
|
||||
import io.javalin.Context;
|
||||
import io.javalin.Handler;
|
||||
import io.javalin.Javalin;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.jetbrains.annotations.NotNull;
|
||||
|
||||
/**
|
||||
* Main REST Handler class that handles local view staleness and delegates calls to slice/data-file/timeline handlers
|
||||
*/
|
||||
public class FileSystemViewHandler {
|
||||
|
||||
private static final ObjectMapper mapper = new ObjectMapper();
|
||||
private static final Logger logger = LogManager.getLogger(FileSystemViewHandler.class);
|
||||
|
||||
private final FileSystemViewManager viewManager;
|
||||
private final Javalin app;
|
||||
private final Configuration conf;
|
||||
private final TimelineHandler instantHandler;
|
||||
private final FileSliceHandler sliceHandler;
|
||||
private final DataFileHandler dataFileHandler;
|
||||
|
||||
public FileSystemViewHandler(Javalin app, Configuration conf, FileSystemViewManager viewManager) throws IOException {
|
||||
this.viewManager = viewManager;
|
||||
this.app = app;
|
||||
this.conf = conf;
|
||||
this.instantHandler = new TimelineHandler(conf, viewManager);
|
||||
this.sliceHandler = new FileSliceHandler(conf, viewManager);
|
||||
this.dataFileHandler = new DataFileHandler(conf, viewManager);
|
||||
}
|
||||
|
||||
public void register() {
|
||||
registerDataFilesAPI();
|
||||
registerFileSlicesAPI();
|
||||
registerTimelineAPI();
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines if local view of dataset's timeline is behind that of client's view
|
||||
* @param ctx
|
||||
* @return
|
||||
*/
|
||||
private boolean isLocalViewBehind(Context ctx) {
|
||||
String basePath = ctx.queryParam(BASEPATH_PARAM);
|
||||
String lastKnownInstantFromClient = ctx.queryParam(LAST_INSTANT_TS, HoodieTimeline.INVALID_INSTANT_TS);
|
||||
String timelineHashFromClient = ctx.queryParam(TIMELINE_HASH, "");
|
||||
HoodieTimeline localTimeline = viewManager.getFileSystemView(basePath).getTimeline()
|
||||
.filterCompletedAndCompactionInstants();
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("Client [ LastTs=" + lastKnownInstantFromClient
|
||||
+ ", TimelineHash=" + timelineHashFromClient + "], localTimeline="
|
||||
+ localTimeline.getInstants().collect(Collectors.toList()));
|
||||
}
|
||||
|
||||
if ((localTimeline.getInstants().count() == 0)
|
||||
&& lastKnownInstantFromClient.equals(HoodieTimeline.INVALID_INSTANT_TS)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
String localTimelineHash = localTimeline.getTimelineHash();
|
||||
if (!localTimelineHash.equals(timelineHashFromClient)) {
|
||||
return true;
|
||||
}
|
||||
|
||||
// As a safety check, even if hash is same, ensure instant is present
|
||||
return !localTimeline.containsOrBeforeTimelineStarts(lastKnownInstantFromClient);
|
||||
}
|
||||
|
||||
/**
|
||||
* Syncs data-set view if local view is behind
|
||||
* @param ctx
|
||||
*/
|
||||
private boolean syncIfLocalViewBehind(Context ctx) {
|
||||
if (isLocalViewBehind(ctx)) {
|
||||
String basePath = ctx.queryParam(BASEPATH_PARAM);
|
||||
String lastKnownInstantFromClient =
|
||||
ctx.queryParam(LAST_INSTANT_TS, HoodieTimeline.INVALID_INSTANT_TS);
|
||||
SyncableFileSystemView view = viewManager.getFileSystemView(basePath);
|
||||
synchronized (view) {
|
||||
if (isLocalViewBehind(ctx)) {
|
||||
HoodieTimeline localTimeline = viewManager.getFileSystemView(basePath).getTimeline();
|
||||
logger.warn("Syncing view as client passed last known instant " + lastKnownInstantFromClient
|
||||
+ " as last known instant but server has the folling timeline :"
|
||||
+ localTimeline.getInstants().collect(Collectors.toList()));
|
||||
view.sync();
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
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);
|
||||
long endJsonTs = System.currentTimeMillis();
|
||||
logger.debug("Jsonify TimeTaken=" + (endJsonTs - beginJsonTs));
|
||||
ctx.result(result);
|
||||
}
|
||||
|
||||
/**
|
||||
* Register Timeline API calls
|
||||
*/
|
||||
private void registerTimelineAPI() {
|
||||
app.get(LAST_INSTANT, new ViewHandler(ctx -> {
|
||||
List<InstantDTO> dtos = instantHandler.getLastInstant(ctx.validatedQueryParam(BASEPATH_PARAM).getValue());
|
||||
writeValueAsString(ctx, dtos);
|
||||
}, false));
|
||||
|
||||
app.get(TIMELINE, new ViewHandler(ctx -> {
|
||||
TimelineDTO dto = instantHandler.getTimeline(ctx.validatedQueryParam(BASEPATH_PARAM).getValue());
|
||||
writeValueAsString(ctx, dto);
|
||||
}, false));
|
||||
}
|
||||
|
||||
/**
|
||||
* Register Data-Files API calls
|
||||
*/
|
||||
private void registerDataFilesAPI() {
|
||||
app.get(LATEST_PARTITION_DATA_FILES_URL, new ViewHandler(ctx -> {
|
||||
List<DataFileDTO> dtos =
|
||||
dataFileHandler.getLatestDataFiles(ctx.validatedQueryParam(BASEPATH_PARAM).getOrThrow(),
|
||||
ctx.validatedQueryParam(PARTITION_PARAM).getOrThrow());
|
||||
writeValueAsString(ctx, dtos);
|
||||
}, true));
|
||||
|
||||
app.get(LATEST_PARTITION_DATA_FILE_URL, new ViewHandler(ctx -> {
|
||||
List<DataFileDTO> dtos =
|
||||
dataFileHandler.getLatestDataFile(ctx.validatedQueryParam(BASEPATH_PARAM).getOrThrow(),
|
||||
ctx.validatedQueryParam(PARTITION_PARAM).getOrThrow(),
|
||||
ctx.validatedQueryParam(FILEID_PARAM).getOrThrow());
|
||||
writeValueAsString(ctx, dtos);
|
||||
}, true));
|
||||
|
||||
app.get(LATEST_ALL_DATA_FILES, new ViewHandler(ctx -> {
|
||||
List<DataFileDTO> dtos =
|
||||
dataFileHandler.getLatestDataFiles(ctx.validatedQueryParam(BASEPATH_PARAM).getOrThrow());
|
||||
writeValueAsString(ctx, dtos);
|
||||
}, true));
|
||||
|
||||
app.get(LATEST_DATA_FILES_BEFORE_ON_INSTANT_URL, new ViewHandler(ctx -> {
|
||||
List<DataFileDTO> dtos =
|
||||
dataFileHandler.getLatestDataFilesBeforeOrOn(ctx.validatedQueryParam(BASEPATH_PARAM).getOrThrow(),
|
||||
ctx.validatedQueryParam(PARTITION_PARAM).getOrThrow(),
|
||||
ctx.validatedQueryParam(MAX_INSTANT_PARAM).getOrThrow());
|
||||
writeValueAsString(ctx, dtos);
|
||||
}, true));
|
||||
|
||||
app.get(LATEST_DATA_FILE_ON_INSTANT_URL, new ViewHandler(ctx -> {
|
||||
List<DataFileDTO> dtos =
|
||||
dataFileHandler.getLatestDataFileOn(ctx.validatedQueryParam(BASEPATH_PARAM).getOrThrow(),
|
||||
ctx.validatedQueryParam(PARTITION_PARAM).getOrThrow(),
|
||||
ctx.queryParam(INSTANT_PARAM), ctx.validatedQueryParam(FILEID_PARAM).getOrThrow());
|
||||
writeValueAsString(ctx, dtos);
|
||||
}, true));
|
||||
|
||||
app.get(ALL_DATA_FILES, new ViewHandler(ctx -> {
|
||||
List<DataFileDTO> dtos =
|
||||
dataFileHandler.getAllDataFiles(ctx.validatedQueryParam(BASEPATH_PARAM).getOrThrow(),
|
||||
ctx.validatedQueryParam(PARTITION_PARAM).getOrThrow());
|
||||
writeValueAsString(ctx, dtos);
|
||||
}, true));
|
||||
|
||||
app.get(LATEST_DATA_FILES_RANGE_INSTANT_URL, new ViewHandler(ctx -> {
|
||||
List<DataFileDTO> dtos =
|
||||
dataFileHandler.getLatestDataFilesInRange(ctx.validatedQueryParam(BASEPATH_PARAM).getOrThrow(),
|
||||
Arrays.asList(ctx.validatedQueryParam(INSTANTS_PARAM).getOrThrow().split(",")));
|
||||
writeValueAsString(ctx, dtos);
|
||||
}, true));
|
||||
}
|
||||
|
||||
/**
|
||||
* Register File Slices API calls
|
||||
*/
|
||||
private void registerFileSlicesAPI() {
|
||||
app.get(LATEST_PARTITION_SLICES_URL, new ViewHandler(ctx -> {
|
||||
List<FileSliceDTO> dtos =
|
||||
sliceHandler.getLatestFileSlices(ctx.validatedQueryParam(BASEPATH_PARAM).getOrThrow(),
|
||||
ctx.validatedQueryParam(PARTITION_PARAM).getOrThrow());
|
||||
writeValueAsString(ctx, dtos);
|
||||
}, true));
|
||||
|
||||
app.get(LATEST_PARTITION_SLICE_URL, new ViewHandler(ctx -> {
|
||||
List<FileSliceDTO> dtos =
|
||||
sliceHandler.getLatestFileSlice(ctx.validatedQueryParam(BASEPATH_PARAM).getOrThrow(),
|
||||
ctx.validatedQueryParam(PARTITION_PARAM).getOrThrow(),
|
||||
ctx.validatedQueryParam(FILEID_PARAM).getOrThrow());
|
||||
writeValueAsString(ctx, dtos);
|
||||
}, true));
|
||||
|
||||
app.get(LATEST_PARTITION_UNCOMPACTED_SLICES_URL, new ViewHandler(ctx -> {
|
||||
List<FileSliceDTO> dtos =
|
||||
sliceHandler.getLatestUnCompactedFileSlices(ctx.validatedQueryParam(BASEPATH_PARAM).getOrThrow(),
|
||||
ctx.validatedQueryParam(PARTITION_PARAM).getOrThrow());
|
||||
writeValueAsString(ctx, dtos);
|
||||
}, true));
|
||||
|
||||
app.get(ALL_SLICES_URL, new ViewHandler(ctx -> {
|
||||
List<FileSliceDTO> dtos =
|
||||
sliceHandler.getAllFileSlices(ctx.validatedQueryParam(BASEPATH_PARAM).getOrThrow(),
|
||||
ctx.validatedQueryParam(PARTITION_PARAM).getOrThrow());
|
||||
writeValueAsString(ctx, dtos);
|
||||
}, true));
|
||||
|
||||
app.get(LATEST_SLICES_RANGE_INSTANT_URL, new ViewHandler(ctx -> {
|
||||
List<FileSliceDTO> dtos =
|
||||
sliceHandler.getLatestFileSliceInRange(ctx.validatedQueryParam(BASEPATH_PARAM).getOrThrow(),
|
||||
Arrays.asList(ctx.validatedQueryParam(INSTANTS_PARAM).getOrThrow().split(",")));
|
||||
writeValueAsString(ctx, dtos);
|
||||
}, true));
|
||||
|
||||
app.get(LATEST_SLICES_MERGED_BEFORE_ON_INSTANT_URL, new ViewHandler(ctx -> {
|
||||
List<FileSliceDTO> dtos =
|
||||
sliceHandler.getLatestMergedFileSlicesBeforeOrOn(ctx.validatedQueryParam(BASEPATH_PARAM).getOrThrow(),
|
||||
ctx.validatedQueryParam(PARTITION_PARAM).getOrThrow(),
|
||||
ctx.validatedQueryParam(MAX_INSTANT_PARAM).getOrThrow());
|
||||
writeValueAsString(ctx, dtos);
|
||||
}, true));
|
||||
|
||||
app.get(LATEST_SLICES_BEFORE_ON_INSTANT_URL, new ViewHandler(ctx -> {
|
||||
List<FileSliceDTO> dtos =
|
||||
sliceHandler.getLatestFileSlicesBeforeOrOn(ctx.validatedQueryParam(BASEPATH_PARAM).getOrThrow(),
|
||||
ctx.validatedQueryParam(PARTITION_PARAM).getOrThrow(),
|
||||
ctx.validatedQueryParam(MAX_INSTANT_PARAM).getOrThrow());
|
||||
writeValueAsString(ctx, dtos);
|
||||
}, true));
|
||||
|
||||
app.get(PENDING_COMPACTION_OPS, new ViewHandler(ctx -> {
|
||||
List<CompactionOpDTO> dtos = sliceHandler.getPendingCompactionOperations(
|
||||
ctx.validatedQueryParam(BASEPATH_PARAM).getOrThrow());
|
||||
writeValueAsString(ctx, dtos);
|
||||
}, true));
|
||||
|
||||
app.get(ALL_FILEGROUPS_FOR_PARTITION_URL, new ViewHandler(ctx -> {
|
||||
List<FileGroupDTO> dtos = sliceHandler.getAllFileGroups(ctx.validatedQueryParam(BASEPATH_PARAM).getOrThrow(),
|
||||
ctx.validatedQueryParam(PARTITION_PARAM).getOrThrow());
|
||||
writeValueAsString(ctx, dtos);
|
||||
}, true));
|
||||
|
||||
app.post(REFRESH_DATASET, new ViewHandler(ctx -> {
|
||||
boolean success = sliceHandler.refreshDataset(ctx.validatedQueryParam(BASEPATH_PARAM).getOrThrow());
|
||||
writeValueAsString(ctx, success);
|
||||
}, false));
|
||||
}
|
||||
|
||||
private static boolean isRefreshCheckDisabledInQuery(Context ctxt) {
|
||||
return Boolean.valueOf(ctxt.queryParam(REFRESH_OFF));
|
||||
}
|
||||
|
||||
/**
|
||||
* Used for logging and performing refresh check.
|
||||
*/
|
||||
private class ViewHandler implements Handler {
|
||||
|
||||
private final Handler handler;
|
||||
private final boolean performRefreshCheck;
|
||||
|
||||
ViewHandler(Handler handler, boolean performRefreshCheck) {
|
||||
this.handler = handler;
|
||||
this.performRefreshCheck = performRefreshCheck;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void handle(@NotNull Context context) throws Exception {
|
||||
boolean success = true;
|
||||
long beginTs = System.currentTimeMillis();
|
||||
boolean synced = false;
|
||||
boolean refreshCheck = performRefreshCheck && !isRefreshCheckDisabledInQuery(context);
|
||||
long refreshCheckTimeTaken = 0;
|
||||
long handleTimeTaken = 0;
|
||||
long finalCheckTimeTaken = 0;
|
||||
try {
|
||||
if (refreshCheck) {
|
||||
long beginRefreshCheck = System.currentTimeMillis();
|
||||
synced = syncIfLocalViewBehind(context);
|
||||
long endRefreshCheck = System.currentTimeMillis();
|
||||
refreshCheckTimeTaken = endRefreshCheck - beginRefreshCheck;
|
||||
}
|
||||
|
||||
long handleBeginMs = System.currentTimeMillis();
|
||||
handler.handle(context);
|
||||
long handleEndMs = System.currentTimeMillis();
|
||||
handleTimeTaken = handleEndMs - handleBeginMs;
|
||||
|
||||
if (refreshCheck) {
|
||||
long beginFinalCheck = System.currentTimeMillis();
|
||||
String errMsg = "Last known instant from client was "
|
||||
+ context.queryParam(LAST_INSTANT_TS, HoodieTimeline.INVALID_INSTANT_TS)
|
||||
+ " but server has the following timeline "
|
||||
+ viewManager.getFileSystemView(context.queryParam(BASEPATH_PARAM))
|
||||
.getTimeline().getInstants().collect(Collectors.toList());
|
||||
Preconditions.checkArgument(!isLocalViewBehind(context), errMsg);
|
||||
long endFinalCheck = System.currentTimeMillis();
|
||||
finalCheckTimeTaken = endFinalCheck - beginFinalCheck;
|
||||
}
|
||||
} catch (RuntimeException re) {
|
||||
success = false;
|
||||
logger.error("Got runtime exception servicing request " + context.queryString(), re);
|
||||
throw re;
|
||||
} 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));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,161 @@
|
||||
/*
|
||||
* Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.timeline.service;
|
||||
|
||||
import com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
import com.uber.hoodie.common.SerializableConfiguration;
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewManager;
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig;
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewStorageType;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import io.javalin.Javalin;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* A stand alone timeline service exposing File-System View interfaces to clients
|
||||
*/
|
||||
public class TimelineService {
|
||||
|
||||
private static volatile Logger log = LogManager.getLogger(TimelineService.class);
|
||||
|
||||
private int serverPort;
|
||||
private Configuration conf;
|
||||
private transient FileSystem fs;
|
||||
private transient Javalin app = null;
|
||||
private transient FileSystemViewManager fsViewsManager;
|
||||
|
||||
public int getServerPort() {
|
||||
return serverPort;
|
||||
}
|
||||
|
||||
public TimelineService(int serverPort, FileSystemViewManager globalFileSystemViewManager) throws IOException {
|
||||
this.conf = FSUtils.prepareHadoopConf(new Configuration());
|
||||
this.fs = FileSystem.get(conf);
|
||||
this.serverPort = serverPort;
|
||||
this.fsViewsManager = globalFileSystemViewManager;
|
||||
}
|
||||
|
||||
public TimelineService(Config config) throws IOException {
|
||||
this(config.serverPort, buildFileSystemViewManager(config,
|
||||
new SerializableConfiguration(FSUtils.prepareHadoopConf(new Configuration()))));
|
||||
}
|
||||
|
||||
public static class Config implements Serializable {
|
||||
|
||||
@Parameter(names = {"--server-port", "-p"}, description = " Server Port")
|
||||
public Integer serverPort = 26754;
|
||||
|
||||
@Parameter(names = {"--view-storage", "-st"}, description = "View Storage Type. Defaut - SPILLABLE_DISK")
|
||||
public FileSystemViewStorageType viewStorageType = FileSystemViewStorageType.SPILLABLE_DISK;
|
||||
|
||||
@Parameter(names = {"--max-view-mem-per-table", "-mv"},
|
||||
description = "Maximum view memory per table in MB to be used for storing file-groups."
|
||||
+ " Overflow file-groups will be spilled to disk. Used for SPILLABLE_DISK storage type")
|
||||
public Integer maxViewMemPerTableInMB = 2048;
|
||||
|
||||
@Parameter(names = {"--mem-overhead-fraction-pending-compaction", "-cf"},
|
||||
description = "Memory Fraction of --max-view-mem-per-table to be allocated for managing pending compaction"
|
||||
+ " storage. Overflow entries will be spilled to disk. Used for SPILLABLE_DISK storage type")
|
||||
public Double memFractionForCompactionPerTable = 0.001;
|
||||
|
||||
@Parameter(names = {"--base-store-path", "-sp"},
|
||||
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")
|
||||
public String rocksDBPath = FileSystemViewStorageConfig.DEFAULT_ROCKSDB_BASE_PATH;
|
||||
|
||||
@Parameter(names = {"--help", "-h"})
|
||||
public Boolean help = false;
|
||||
}
|
||||
|
||||
public int startService() throws IOException {
|
||||
app = Javalin.create();
|
||||
FileSystemViewHandler router = new FileSystemViewHandler(app, conf, fsViewsManager);
|
||||
app.get("/", ctx -> ctx.result("Hello World"));
|
||||
router.register();
|
||||
app.start(serverPort);
|
||||
// If port = 0, a dynamic port is assigned. Store it.
|
||||
serverPort = app.port();
|
||||
log.info("Starting Timeline server on port :" + serverPort);
|
||||
return serverPort;
|
||||
}
|
||||
|
||||
public void run() throws IOException {
|
||||
startService();
|
||||
}
|
||||
|
||||
public static FileSystemViewManager buildFileSystemViewManager(Config config, SerializableConfiguration conf) {
|
||||
switch (config.viewStorageType) {
|
||||
case MEMORY:
|
||||
FileSystemViewStorageConfig.Builder inMemConfBuilder = FileSystemViewStorageConfig.newBuilder();
|
||||
inMemConfBuilder.withStorageType(FileSystemViewStorageType.MEMORY);
|
||||
return FileSystemViewManager.createViewManager(conf, inMemConfBuilder.build());
|
||||
case SPILLABLE_DISK: {
|
||||
FileSystemViewStorageConfig.Builder spillableConfBuilder = FileSystemViewStorageConfig.newBuilder();
|
||||
spillableConfBuilder.withStorageType(FileSystemViewStorageType.SPILLABLE_DISK)
|
||||
.withBaseStoreDir(config.baseStorePathForFileGroups)
|
||||
.withMaxMemoryForView(config.maxViewMemPerTableInMB * 1024 * 1024L)
|
||||
.withMemFractionForPendingCompaction(config.memFractionForCompactionPerTable);
|
||||
return FileSystemViewManager.createViewManager(conf, spillableConfBuilder.build());
|
||||
}
|
||||
case EMBEDDED_KV_STORE: {
|
||||
FileSystemViewStorageConfig.Builder rocksDBConfBuilder = FileSystemViewStorageConfig.newBuilder();
|
||||
rocksDBConfBuilder.withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE)
|
||||
.withRocksDBPath(config.rocksDBPath);
|
||||
return FileSystemViewManager.createViewManager(conf, rocksDBConfBuilder.build());
|
||||
}
|
||||
default:
|
||||
throw new IllegalArgumentException("Invalid view manager storage type :" + config.viewStorageType);
|
||||
}
|
||||
}
|
||||
|
||||
public void close() {
|
||||
this.app.stop();
|
||||
this.app = null;
|
||||
this.fsViewsManager.close();
|
||||
}
|
||||
|
||||
public Configuration getConf() {
|
||||
return conf;
|
||||
}
|
||||
|
||||
public FileSystem getFs() {
|
||||
return fs;
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
final Config cfg = new Config();
|
||||
JCommander cmd = new JCommander(cfg, args);
|
||||
if (cfg.help || args.length == 0) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
|
||||
Configuration conf = FSUtils.prepareHadoopConf(new Configuration());
|
||||
FileSystemViewManager viewManager = buildFileSystemViewManager(cfg, new SerializableConfiguration(conf));
|
||||
TimelineService service = new TimelineService(cfg.serverPort, viewManager);
|
||||
service.run();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,76 @@
|
||||
/*
|
||||
* Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.timeline.service.handlers;
|
||||
|
||||
import com.uber.hoodie.common.table.timeline.dto.DataFileDTO;
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewManager;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
/**
|
||||
* REST Handler servicing data-file requests
|
||||
*/
|
||||
public class DataFileHandler extends Handler {
|
||||
|
||||
public DataFileHandler(Configuration conf,
|
||||
FileSystemViewManager viewManager) throws IOException {
|
||||
super(conf, viewManager);
|
||||
}
|
||||
|
||||
public List<DataFileDTO> getLatestDataFiles(String basePath, String partitionPath) {
|
||||
return viewManager.getFileSystemView(basePath).getLatestDataFiles(partitionPath)
|
||||
.map(DataFileDTO::fromHoodieDataFile).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public List<DataFileDTO> getLatestDataFile(String basePath, String partitionPath, String fileId) {
|
||||
return viewManager.getFileSystemView(basePath).getLatestDataFile(partitionPath, fileId)
|
||||
.map(DataFileDTO::fromHoodieDataFile).map(dto -> Arrays.asList(dto)).orElse(new ArrayList<>());
|
||||
}
|
||||
|
||||
public List<DataFileDTO> getLatestDataFiles(String basePath) {
|
||||
return viewManager.getFileSystemView(basePath).getLatestDataFiles()
|
||||
.map(DataFileDTO::fromHoodieDataFile).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public List<DataFileDTO> getLatestDataFilesBeforeOrOn(String basePath, String partitionPath, String maxInstantTime) {
|
||||
return viewManager.getFileSystemView(basePath).getLatestDataFilesBeforeOrOn(partitionPath, maxInstantTime)
|
||||
.map(DataFileDTO::fromHoodieDataFile).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public List<DataFileDTO> getLatestDataFileOn(String basePath, String partitionPath, String instantTime,
|
||||
String fileId) {
|
||||
List<DataFileDTO> result = new ArrayList<>();
|
||||
viewManager.getFileSystemView(basePath).getDataFileOn(partitionPath, instantTime, fileId)
|
||||
.map(DataFileDTO::fromHoodieDataFile).ifPresent(result::add);
|
||||
return result;
|
||||
}
|
||||
|
||||
public List<DataFileDTO> getLatestDataFilesInRange(String basePath, List<String> instants) {
|
||||
return viewManager.getFileSystemView(basePath).getLatestDataFilesInRange(instants)
|
||||
.map(DataFileDTO::fromHoodieDataFile).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public List<DataFileDTO> getAllDataFiles(String basePath, String partitionPath) {
|
||||
return viewManager.getFileSystemView(basePath).getAllDataFiles(partitionPath)
|
||||
.map(DataFileDTO::fromHoodieDataFile).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,92 @@
|
||||
/*
|
||||
* Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.timeline.service.handlers;
|
||||
|
||||
import com.uber.hoodie.common.table.timeline.dto.CompactionOpDTO;
|
||||
import com.uber.hoodie.common.table.timeline.dto.FileGroupDTO;
|
||||
import com.uber.hoodie.common.table.timeline.dto.FileSliceDTO;
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewManager;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
/**
|
||||
* REST Handler servicing file-slice requests
|
||||
*/
|
||||
public class FileSliceHandler extends Handler {
|
||||
|
||||
public FileSliceHandler(Configuration conf,
|
||||
FileSystemViewManager viewManager) throws IOException {
|
||||
super(conf, viewManager);
|
||||
}
|
||||
|
||||
public List<FileSliceDTO> getAllFileSlices(String basePath, String partitionPath) {
|
||||
return viewManager.getFileSystemView(basePath).getAllFileSlices(partitionPath).map(FileSliceDTO::fromFileSlice)
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public List<FileSliceDTO> getLatestFileSliceInRange(String basePath, List<String> instantsToReturn) {
|
||||
return viewManager.getFileSystemView(basePath).getLatestFileSliceInRange(instantsToReturn)
|
||||
.map(FileSliceDTO::fromFileSlice).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
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) {
|
||||
return viewManager.getFileSystemView(basePath).getLatestFileSlicesBeforeOrOn(partitionPath, maxInstantTime)
|
||||
.map(FileSliceDTO::fromFileSlice).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public List<FileSliceDTO> getLatestUnCompactedFileSlices(String basePath, String partitionPath) {
|
||||
return viewManager.getFileSystemView(basePath).getLatestUnCompactedFileSlices(partitionPath)
|
||||
.map(FileSliceDTO::fromFileSlice).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public List<FileSliceDTO> getLatestFileSlices(String basePath, String partitionPath) {
|
||||
return viewManager.getFileSystemView(basePath).getLatestFileSlices(partitionPath)
|
||||
.map(FileSliceDTO::fromFileSlice).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public List<FileSliceDTO> getLatestFileSlice(String basePath, String partitionPath, String fileId) {
|
||||
return viewManager.getFileSystemView(basePath).getLatestFileSlice(partitionPath, fileId)
|
||||
.map(FileSliceDTO::fromFileSlice).map(dto -> Arrays.asList(dto)).orElse(new ArrayList<>());
|
||||
}
|
||||
|
||||
public List<CompactionOpDTO> getPendingCompactionOperations(String basePath) {
|
||||
return viewManager.getFileSystemView(basePath).getPendingCompactionOperations()
|
||||
.map(instantOp -> CompactionOpDTO.fromCompactionOperation(instantOp.getKey(), instantOp.getValue()))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public List<FileGroupDTO> getAllFileGroups(String basePath, String partitionPath) {
|
||||
return viewManager.getFileSystemView(basePath).getAllFileGroups(partitionPath)
|
||||
.map(FileGroupDTO::fromFileGroup).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public boolean refreshDataset(String basePath) {
|
||||
viewManager.clearFileSystemView(basePath);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,35 @@
|
||||
/*
|
||||
* Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.timeline.service.handlers;
|
||||
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewManager;
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
||||
public abstract class Handler {
|
||||
|
||||
protected final Configuration conf;
|
||||
protected final FileSystem fileSystem;
|
||||
protected final FileSystemViewManager viewManager;
|
||||
|
||||
public Handler(Configuration conf, FileSystemViewManager viewManager) throws IOException {
|
||||
this.conf = conf;
|
||||
this.fileSystem = FileSystem.get(conf);
|
||||
this.viewManager = viewManager;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,46 @@
|
||||
/*
|
||||
* Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.timeline.service.handlers;
|
||||
|
||||
import com.uber.hoodie.common.table.timeline.dto.InstantDTO;
|
||||
import com.uber.hoodie.common.table.timeline.dto.TimelineDTO;
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewManager;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
/**
|
||||
* REST Handler servicing timeline requests
|
||||
*/
|
||||
public class TimelineHandler extends Handler {
|
||||
|
||||
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<>());
|
||||
}
|
||||
|
||||
public TimelineDTO getTimeline(String basePath) {
|
||||
return TimelineDTO.fromTimeline(viewManager.getFileSystemView(basePath).getTimeline());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,57 @@
|
||||
/*
|
||||
* Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.timeline.table.view;
|
||||
|
||||
import com.uber.hoodie.common.SerializableConfiguration;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.SyncableFileSystemView;
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewManager;
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig;
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewStorageType;
|
||||
import com.uber.hoodie.common.table.view.HoodieTableFileSystemViewTest;
|
||||
import com.uber.hoodie.common.table.view.RemoteHoodieTableFileSystemView;
|
||||
import com.uber.hoodie.timeline.service.TimelineService;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Bring up a remote Timeline Server and run all test-cases of HoodieTableFileSystemViewTest against it
|
||||
*/
|
||||
public class RemoteHoodieTableFileSystemViewTest extends HoodieTableFileSystemViewTest {
|
||||
|
||||
private static Logger log = LogManager.getLogger(RemoteHoodieTableFileSystemViewTest.class);
|
||||
|
||||
private TimelineService server;
|
||||
private RemoteHoodieTableFileSystemView view;
|
||||
|
||||
protected SyncableFileSystemView getFileSystemView(HoodieTimeline timeline) {
|
||||
FileSystemViewStorageConfig sConf = FileSystemViewStorageConfig.newBuilder()
|
||||
.withStorageType(FileSystemViewStorageType.SPILLABLE_DISK).build();
|
||||
try {
|
||||
server =
|
||||
new TimelineService(0,
|
||||
FileSystemViewManager.createViewManager(new SerializableConfiguration(metaClient.getHadoopConf()),
|
||||
sConf));
|
||||
server.startService();
|
||||
} catch (Exception ex) {
|
||||
throw new RuntimeException(ex);
|
||||
}
|
||||
log.info("Connecting to Timeline Server :" + server.getServerPort());
|
||||
view = new RemoteHoodieTableFileSystemView("localhost", server.getServerPort(), metaClient);
|
||||
return view;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,21 @@
|
||||
#
|
||||
# Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License");
|
||||
# you may not use this file except in compliance with the License.
|
||||
# You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
#
|
||||
log4j.rootLogger=WARN, A1
|
||||
# A1 is set to be a ConsoleAppender.
|
||||
log4j.appender.A1=org.apache.log4j.ConsoleAppender
|
||||
# A1 uses PatternLayout.
|
||||
log4j.appender.A1.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.A1.layout.ConversionPattern=[%-5p] %d %c %x - %m%n
|
||||
@@ -0,0 +1,25 @@
|
||||
#
|
||||
# Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License");
|
||||
# you may not use this file except in compliance with the License.
|
||||
# You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
#
|
||||
log4j.rootLogger=WARN, A1
|
||||
log4j.category.com.uber=INFO
|
||||
log4j.category.com.uber.hoodie.table.log=WARN
|
||||
log4j.category.com.uber.hoodie.common.util=WARN
|
||||
log4j.category.org.apache.parquet.hadoop=WARN
|
||||
# A1 is set to be a ConsoleAppender.
|
||||
log4j.appender.A1=org.apache.log4j.ConsoleAppender
|
||||
# A1 uses PatternLayout.
|
||||
log4j.appender.A1.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
|
||||
Reference in New Issue
Block a user