1
0

HUDI-70 : Making DeltaStreamer run in continuous mode with concurrent compaction

This commit is contained in:
Balaji Varadarajan
2019-05-15 13:21:55 -07:00
committed by Balaji Varadarajan
parent 3a210ef08e
commit a0d7ab2384
32 changed files with 2000 additions and 441 deletions

View File

@@ -23,6 +23,7 @@ import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import java.io.IOException;
import java.io.Serializable;
import java.util.Optional;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -47,13 +48,21 @@ public abstract class AbstractHoodieClient implements Serializable {
* of the cached file-system view. New completed actions will be synced automatically
* in an incremental fashion.
*/
private transient EmbeddedTimelineService timelineServer;
private transient Optional<EmbeddedTimelineService> timelineServer;
private final boolean shouldStopTimelineServer;
protected AbstractHoodieClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) {
this(jsc, clientConfig, Optional.empty());
}
protected AbstractHoodieClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig,
Optional<EmbeddedTimelineService> timelineServer) {
this.fs = FSUtils.getFs(clientConfig.getBasePath(), jsc.hadoopConfiguration());
this.jsc = jsc;
this.basePath = clientConfig.getBasePath();
this.config = clientConfig;
this.timelineServer = timelineServer;
shouldStopTimelineServer = !timelineServer.isPresent();
startEmbeddedServerView();
}
@@ -65,28 +74,30 @@ public abstract class AbstractHoodieClient implements Serializable {
}
private synchronized void stopEmbeddedServerView(boolean resetViewStorageConfig) {
if (timelineServer != null) {
if (timelineServer.isPresent() && shouldStopTimelineServer) {
// Stop only if owner
logger.info("Stopping Timeline service !!");
timelineServer.stop();
timelineServer = null;
// Reset Storage Config to Client specified config
if (resetViewStorageConfig) {
config.resetViewStorageConfig();
}
timelineServer.get().stop();
}
timelineServer = Optional.empty();
// Reset Storage Config to Client specified config
if (resetViewStorageConfig) {
config.resetViewStorageConfig();
}
}
private synchronized void startEmbeddedServerView() {
if (config.isEmbeddedTimelineServerEnabled()) {
if (timelineServer == null) {
if (!timelineServer.isPresent()) {
// Run Embedded Timeline Server
logger.info("Starting Timeline service !!");
timelineServer = new EmbeddedTimelineService(jsc.hadoopConfiguration(), jsc.getConf(),
config.getClientSpecifiedViewStorageConfig());
timelineServer = Optional.of(new EmbeddedTimelineService(jsc.hadoopConfiguration(), jsc.getConf(),
config.getClientSpecifiedViewStorageConfig()));
try {
timelineServer.startServer();
timelineServer.get().startServer();
// Allow executor to find this newly instantiated timeline service
config.setViewStorageConfig(timelineServer.getRemoteFileSystemViewConfig());
config.setViewStorageConfig(timelineServer.get().getRemoteFileSystemViewConfig());
} catch (IOException e) {
logger.warn("Unable to start timeline service. Proceeding as if embedded server is disabled", e);
stopEmbeddedServerView(false);
@@ -98,4 +109,12 @@ public abstract class AbstractHoodieClient implements Serializable {
logger.info("Embedded Timeline Server is disabled. Not starting timeline service");
}
}
public HoodieWriteConfig getConfig() {
return config;
}
public Optional<EmbeddedTimelineService> getTimelineServer() {
return timelineServer;
}
}