Ensure TableMetaClient and FileSystem instances have exclusive copy of Configuration
This commit is contained in:
committed by
Balaji Varadarajan
parent
a0d7ab2384
commit
2c40e8419e
@@ -68,7 +68,7 @@ public class EmbeddedTimelineService {
|
||||
}
|
||||
|
||||
public void startServer() throws IOException {
|
||||
server = new TimelineService(0, viewManager);
|
||||
server = new TimelineService(0, viewManager, hadoopConf.get());
|
||||
serverPort = server.startService();
|
||||
logger.info("Started embedded timeline server at " + hostAddr + ":" + serverPort);
|
||||
}
|
||||
|
||||
@@ -29,11 +29,15 @@ public class SerializableConfiguration implements Serializable {
|
||||
private transient Configuration configuration;
|
||||
|
||||
public SerializableConfiguration(Configuration configuration) {
|
||||
this.configuration = configuration;
|
||||
this.configuration = new Configuration(configuration);
|
||||
}
|
||||
|
||||
public SerializableConfiguration(SerializableConfiguration configuration) {
|
||||
this.configuration = configuration.get();
|
||||
}
|
||||
|
||||
public Configuration get() {
|
||||
return configuration;
|
||||
return new Configuration(configuration);
|
||||
}
|
||||
|
||||
private void writeObject(ObjectOutputStream out) throws IOException {
|
||||
|
||||
@@ -85,7 +85,7 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
throws DatasetNotFoundException {
|
||||
log.info("Loading HoodieTableMetaClient from " + basePath);
|
||||
this.basePath = basePath;
|
||||
this.hadoopConf = new SerializableConfiguration(new Configuration(conf));
|
||||
this.hadoopConf = new SerializableConfiguration(conf);
|
||||
Path basePathDir = new Path(this.basePath);
|
||||
this.metaPath = basePath + File.separator + METAFOLDER_NAME;
|
||||
Path metaPathDir = new Path(this.metaPath);
|
||||
|
||||
@@ -65,7 +65,7 @@ public class FileSystemViewManager {
|
||||
|
||||
public FileSystemViewManager(SerializableConfiguration conf, FileSystemViewStorageConfig viewStorageConfig,
|
||||
Function2<String, FileSystemViewStorageConfig, SyncableFileSystemView> viewCreator) {
|
||||
this.conf = conf;
|
||||
this.conf = new SerializableConfiguration(conf);
|
||||
this.viewStorageConfig = viewStorageConfig;
|
||||
this.globalViewMap = new ConcurrentHashMap<>();
|
||||
this.viewCreator = viewCreator;
|
||||
|
||||
@@ -50,13 +50,18 @@ public class TimelineService {
|
||||
return serverPort;
|
||||
}
|
||||
|
||||
public TimelineService(int serverPort, FileSystemViewManager globalFileSystemViewManager) throws IOException {
|
||||
this.conf = FSUtils.prepareHadoopConf(new Configuration());
|
||||
public TimelineService(int serverPort, FileSystemViewManager globalFileSystemViewManager,
|
||||
Configuration conf) throws IOException {
|
||||
this.conf = FSUtils.prepareHadoopConf(conf);
|
||||
this.fs = FileSystem.get(conf);
|
||||
this.serverPort = serverPort;
|
||||
this.fsViewsManager = globalFileSystemViewManager;
|
||||
}
|
||||
|
||||
public TimelineService(int serverPort, FileSystemViewManager globalFileSystemViewManager) throws IOException {
|
||||
this(serverPort, globalFileSystemViewManager, new Configuration());
|
||||
}
|
||||
|
||||
public TimelineService(Config config) throws IOException {
|
||||
this(config.serverPort, buildFileSystemViewManager(config,
|
||||
new SerializableConfiguration(FSUtils.prepareHadoopConf(new Configuration()))));
|
||||
|
||||
Reference in New Issue
Block a user