mirror of https://github.com/apache/druid.git
commit
c448755dc5
|
@ -29,7 +29,13 @@ public class MetadataStorageConnectorConfig
|
|||
private boolean createTables = true;
|
||||
|
||||
@JsonProperty
|
||||
private String connectURI = null;
|
||||
private String host = "localhost";
|
||||
|
||||
@JsonProperty
|
||||
private int port = 1527;
|
||||
|
||||
@JsonProperty
|
||||
private String connectURI;
|
||||
|
||||
@JsonProperty
|
||||
private String user = null;
|
||||
|
@ -37,16 +43,26 @@ public class MetadataStorageConnectorConfig
|
|||
@JsonProperty
|
||||
private String password = null;
|
||||
|
||||
@JsonProperty
|
||||
private String validationQuery = "SELECT 1";
|
||||
|
||||
public boolean isCreateTables()
|
||||
{
|
||||
return createTables;
|
||||
}
|
||||
|
||||
public String getHost()
|
||||
{
|
||||
return host;
|
||||
}
|
||||
|
||||
public int getPort()
|
||||
{
|
||||
return port;
|
||||
}
|
||||
|
||||
public String getConnectURI()
|
||||
{
|
||||
if (connectURI == null) {
|
||||
return String.format("jdbc:derby://%s:%s/druid;create=true", host, port);
|
||||
}
|
||||
return connectURI;
|
||||
}
|
||||
|
||||
|
@ -60,20 +76,14 @@ public class MetadataStorageConnectorConfig
|
|||
return password;
|
||||
}
|
||||
|
||||
public String getValidationQuery()
|
||||
{
|
||||
return validationQuery;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "DbConnectorConfig{" +
|
||||
"createTables=" + createTables +
|
||||
", connectURI='" + connectURI + '\'' +
|
||||
", connectURI='" + getConnectURI() + '\'' +
|
||||
", user='" + user + '\'' +
|
||||
", password=****" +
|
||||
", validationQuery='" + validationQuery + '\'' +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -72,7 +72,7 @@ public class MySQLConnector extends SQLMetadataConnector
|
|||
}
|
||||
|
||||
@Override
|
||||
protected boolean tableExists(Handle handle, String tableName)
|
||||
public boolean tableExists(Handle handle, String tableName)
|
||||
{
|
||||
return !handle.createQuery("SHOW tables LIKE :tableName")
|
||||
.bind("tableName", tableName)
|
||||
|
|
|
@ -65,7 +65,7 @@ public class PostgreSQLConnector extends SQLMetadataConnector
|
|||
}
|
||||
|
||||
@Override
|
||||
protected boolean tableExists(final Handle handle, final String tableName)
|
||||
public boolean tableExists(final Handle handle, final String tableName)
|
||||
{
|
||||
return !handle.createQuery(
|
||||
"SELECT tablename FROM pg_catalog.pg_tables WHERE schemaname = 'public' AND tablename LIKE :tableName"
|
||||
|
|
|
@ -21,12 +21,12 @@ package io.druid.guice;
|
|||
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Module;
|
||||
import io.druid.metadata.MetadataStorageConnectorConfig;
|
||||
import io.druid.metadata.MetadataRuleManagerConfig;
|
||||
import io.druid.metadata.MetadataSegmentManagerConfig;
|
||||
import io.druid.metadata.MetadataStorageConnectorConfig;
|
||||
import io.druid.metadata.MetadataStorageTablesConfig;
|
||||
|
||||
public class MetadataDbConfigModule implements Module
|
||||
public class MetadataConfigModule implements Module
|
||||
{
|
||||
@Override
|
||||
public void configure(Binder binder)
|
|
@ -34,6 +34,8 @@ import io.druid.metadata.MetadataSegmentManagerProvider;
|
|||
import io.druid.metadata.MetadataSegmentPublisher;
|
||||
import io.druid.metadata.MetadataSegmentPublisherProvider;
|
||||
import io.druid.metadata.MetadataStorageConnector;
|
||||
import io.druid.metadata.MetadataStorageProvider;
|
||||
import io.druid.metadata.NoopMetadataStorageProvider;
|
||||
import io.druid.metadata.SQLMetadataConnector;
|
||||
import io.druid.metadata.SQLMetadataRuleManager;
|
||||
import io.druid.metadata.SQLMetadataRuleManagerProvider;
|
||||
|
@ -62,6 +64,13 @@ public class SQLMetadataStorageDruidModule implements Module
|
|||
PolyBind.createChoiceWithDefault(
|
||||
binder, PROPERTY, Key.get(MetadataStorageConnector.class), null, defaultPropertyValue
|
||||
);
|
||||
PolyBind.createChoiceWithDefault(
|
||||
binder,
|
||||
PROPERTY,
|
||||
Key.get(MetadataStorageProvider.class),
|
||||
Key.get(NoopMetadataStorageProvider.class),
|
||||
defaultPropertyValue
|
||||
);
|
||||
PolyBind.createChoiceWithDefault(
|
||||
binder, PROPERTY, Key.get(SQLMetadataConnector.class), null, defaultPropertyValue
|
||||
);
|
||||
|
|
|
@ -36,7 +36,7 @@ import io.druid.curator.CuratorModule;
|
|||
import io.druid.curator.discovery.DiscoveryModule;
|
||||
import io.druid.guice.AWSModule;
|
||||
import io.druid.guice.AnnouncerModule;
|
||||
import io.druid.guice.DerbyMetadataStorageDruidModule;
|
||||
import io.druid.metadata.storage.derby.DerbyMetadataStorageDruidModule;
|
||||
import io.druid.guice.DruidProcessingModule;
|
||||
import io.druid.guice.DruidSecondaryModule;
|
||||
import io.druid.guice.ExtensionsConfig;
|
||||
|
@ -45,7 +45,7 @@ import io.druid.guice.IndexingServiceDiscoveryModule;
|
|||
import io.druid.guice.JacksonConfigManagerModule;
|
||||
import io.druid.guice.LifecycleModule;
|
||||
import io.druid.guice.LocalDataStorageDruidModule;
|
||||
import io.druid.guice.MetadataDbConfigModule;
|
||||
import io.druid.guice.MetadataConfigModule;
|
||||
import io.druid.guice.ParsersModule;
|
||||
import io.druid.guice.QueryRunnerFactoryModule;
|
||||
import io.druid.guice.QueryableModule;
|
||||
|
@ -333,7 +333,7 @@ public class Initialization
|
|||
new QueryRunnerFactoryModule(),
|
||||
new DiscoveryModule(),
|
||||
new ServerViewModule(),
|
||||
new MetadataDbConfigModule(),
|
||||
new MetadataConfigModule(),
|
||||
new DerbyMetadataStorageDruidModule(),
|
||||
new JacksonConfigManagerModule(),
|
||||
new IndexingServiceDiscoveryModule(),
|
||||
|
|
|
@ -0,0 +1,20 @@
|
|||
package io.druid.metadata;
|
||||
|
||||
import com.metamx.common.lifecycle.LifecycleStart;
|
||||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import io.druid.guice.ManageLifecycle;
|
||||
|
||||
public abstract class MetadataStorage
|
||||
{
|
||||
@LifecycleStart
|
||||
public void start()
|
||||
{
|
||||
// do nothing
|
||||
}
|
||||
|
||||
@LifecycleStop
|
||||
public void stop()
|
||||
{
|
||||
// do nothing
|
||||
}
|
||||
}
|
|
@ -0,0 +1,26 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2014 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.metadata;
|
||||
|
||||
import com.google.inject.Provider;
|
||||
|
||||
public interface MetadataStorageProvider extends Provider<MetadataStorage>
|
||||
{
|
||||
}
|
|
@ -1,6 +1,6 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
* Copyright (C) 2014 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
|
@ -19,29 +19,11 @@
|
|||
|
||||
package io.druid.metadata;
|
||||
|
||||
import com.google.api.client.repackaged.com.google.common.base.Throwables;
|
||||
import org.skife.jdbi.v2.tweak.ConnectionFactory;
|
||||
|
||||
import java.sql.Connection;
|
||||
import java.sql.DriverManager;
|
||||
import java.sql.SQLException;
|
||||
|
||||
public class DerbyConnectionFactory implements ConnectionFactory
|
||||
public class NoopMetadataStorageProvider implements MetadataStorageProvider
|
||||
{
|
||||
final private String dbName;
|
||||
|
||||
public DerbyConnectionFactory(String dbName) {
|
||||
this.dbName = dbName;
|
||||
}
|
||||
|
||||
public Connection openConnection() throws SQLException {
|
||||
final String nsURL=String.format("jdbc:derby://localhost:1527/%s;create=true", dbName);
|
||||
try {
|
||||
Class.forName("org.apache.derby.jdbc.ClientDriver");
|
||||
} catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
return DriverManager.getConnection(nsURL);
|
||||
@Override
|
||||
public MetadataStorage get()
|
||||
{
|
||||
return new MetadataStorage() {};
|
||||
}
|
||||
}
|
|
@ -75,7 +75,9 @@ public abstract class SQLMetadataConnector implements MetadataStorageConnector
|
|||
*/
|
||||
protected abstract String getSerialType();
|
||||
|
||||
protected abstract boolean tableExists(Handle handle, final String tableName);
|
||||
public String getValidationQuery() { return "SELECT 1"; }
|
||||
|
||||
public abstract boolean tableExists(Handle handle, final String tableName);
|
||||
|
||||
protected boolean isTransientException(Throwable e) {
|
||||
return false;
|
||||
|
@ -367,7 +369,7 @@ public abstract class SQLMetadataConnector implements MetadataStorageConnector
|
|||
String uri = connectorConfig.getConnectURI();
|
||||
dataSource.setUrl(uri);
|
||||
|
||||
dataSource.setValidationQuery(connectorConfig.getValidationQuery());
|
||||
dataSource.setValidationQuery(getValidationQuery());
|
||||
dataSource.setTestOnBorrow(true);
|
||||
|
||||
return dataSource;
|
||||
|
|
|
@ -17,17 +17,16 @@
|
|||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.metadata;
|
||||
package io.druid.metadata.storage.derby;
|
||||
|
||||
import com.google.api.client.repackaged.com.google.common.base.Throwables;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.derby.drda.NetworkServerControl;
|
||||
import io.druid.metadata.MetadataStorageConnectorConfig;
|
||||
import io.druid.metadata.MetadataStorageTablesConfig;
|
||||
import io.druid.metadata.SQLMetadataConnector;
|
||||
import org.apache.commons.dbcp2.BasicDataSource;
|
||||
import org.skife.jdbi.v2.DBI;
|
||||
import org.skife.jdbi.v2.Handle;
|
||||
import org.skife.jdbi.v2.tweak.ConnectionFactory;
|
||||
|
||||
import java.net.InetAddress;
|
||||
|
||||
public class DerbyConnector extends SQLMetadataConnector
|
||||
{
|
||||
|
@ -37,17 +36,27 @@ public class DerbyConnector extends SQLMetadataConnector
|
|||
@Inject
|
||||
public DerbyConnector(Supplier<MetadataStorageConnectorConfig> config, Supplier<MetadataStorageTablesConfig> dbTables)
|
||||
{
|
||||
this(config, dbTables, new DBI(getConnectionFactory("druidDerbyDb")));
|
||||
super(config, dbTables);
|
||||
|
||||
final BasicDataSource datasource = getDatasource();
|
||||
datasource.setDriverClassLoader(getClass().getClassLoader());
|
||||
datasource.setDriverClassName("org.apache.derby.jdbc.ClientDriver");
|
||||
|
||||
this.dbi = new DBI(datasource);
|
||||
}
|
||||
|
||||
public DerbyConnector(Supplier<MetadataStorageConnectorConfig> config, Supplier<MetadataStorageTablesConfig> dbTables, DBI dbi)
|
||||
public DerbyConnector(
|
||||
Supplier<MetadataStorageConnectorConfig> config,
|
||||
Supplier<MetadataStorageTablesConfig> dbTables,
|
||||
DBI dbi
|
||||
)
|
||||
{
|
||||
super(config, dbTables);
|
||||
this.dbi = dbi;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean tableExists(Handle handle, String tableName)
|
||||
public boolean tableExists(Handle handle, String tableName)
|
||||
{
|
||||
return !handle.createQuery("select * from SYS.SYSTABLES where tablename = :tableName")
|
||||
.bind("tableName", tableName.toUpperCase())
|
||||
|
@ -64,14 +73,6 @@ public class DerbyConnector extends SQLMetadataConnector
|
|||
@Override
|
||||
public DBI getDBI() { return dbi; }
|
||||
|
||||
private static ConnectionFactory getConnectionFactory(String dbName)
|
||||
{
|
||||
try {
|
||||
NetworkServerControl server = new NetworkServerControl(InetAddress.getByName("localhost"),1527);
|
||||
server.start(null);
|
||||
} catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
return new DerbyConnectionFactory(dbName);
|
||||
}
|
||||
@Override
|
||||
public String getValidationQuery() { return "VALUES 1"; }
|
||||
}
|
|
@ -0,0 +1,78 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.metadata.storage.derby;
|
||||
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.common.lifecycle.LifecycleStart;
|
||||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import io.druid.guice.ManageLifecycle;
|
||||
import io.druid.metadata.MetadataStorage;
|
||||
import io.druid.metadata.MetadataStorageConnectorConfig;
|
||||
import org.apache.derby.drda.NetworkServerControl;
|
||||
|
||||
import java.net.InetAddress;
|
||||
|
||||
|
||||
@ManageLifecycle
|
||||
public class DerbyMetadataStorage extends MetadataStorage
|
||||
{
|
||||
private static final Logger log = new Logger(DerbyMetadataStorage.class);
|
||||
|
||||
private final NetworkServerControl server;
|
||||
|
||||
@Inject
|
||||
public DerbyMetadataStorage(MetadataStorageConnectorConfig config)
|
||||
{
|
||||
try {
|
||||
this.server = new NetworkServerControl(InetAddress.getByName(config.getHost()), config.getPort());
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
@LifecycleStart
|
||||
public void start()
|
||||
{
|
||||
try {
|
||||
log.info("Starting Derby Metadata Storage");
|
||||
server.start(null);
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@LifecycleStop
|
||||
public void stop()
|
||||
{
|
||||
try {
|
||||
server.shutdown();
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -17,15 +17,16 @@
|
|||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.guice;
|
||||
package io.druid.metadata.storage.derby;
|
||||
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Key;
|
||||
import com.google.inject.Provides;
|
||||
import io.druid.metadata.DerbyConnector;
|
||||
import io.druid.guice.LazySingleton;
|
||||
import io.druid.guice.PolyBind;
|
||||
import io.druid.guice.SQLMetadataStorageDruidModule;
|
||||
import io.druid.metadata.MetadataStorageConnector;
|
||||
import io.druid.metadata.MetadataStorageProvider;
|
||||
import io.druid.metadata.SQLMetadataConnector;
|
||||
import org.skife.jdbi.v2.IDBI;
|
||||
|
||||
public class DerbyMetadataStorageDruidModule extends SQLMetadataStorageDruidModule
|
||||
{
|
||||
|
@ -42,6 +43,11 @@ public class DerbyMetadataStorageDruidModule extends SQLMetadataStorageDruidModu
|
|||
createBindingChoices(binder, TYPE);
|
||||
super.configure(binder);
|
||||
|
||||
PolyBind.optionBinder(binder, Key.get(MetadataStorageProvider.class))
|
||||
.addBinding(TYPE)
|
||||
.to(DerbyMetadataStorageProvider.class)
|
||||
.in(LazySingleton.class);
|
||||
|
||||
PolyBind.optionBinder(binder, Key.get(MetadataStorageConnector.class))
|
||||
.addBinding(TYPE)
|
||||
.to(DerbyConnector.class)
|
|
@ -0,0 +1,42 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2014 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package io.druid.metadata.storage.derby;
|
||||
|
||||
import com.google.inject.Inject;
|
||||
import io.druid.metadata.MetadataStorage;
|
||||
import io.druid.metadata.MetadataStorageConnectorConfig;
|
||||
import io.druid.metadata.MetadataStorageProvider;
|
||||
|
||||
public class DerbyMetadataStorageProvider implements MetadataStorageProvider
|
||||
{
|
||||
private final DerbyMetadataStorage storage;
|
||||
|
||||
@Inject
|
||||
public DerbyMetadataStorageProvider(MetadataStorageConnectorConfig config)
|
||||
{
|
||||
this.storage = new DerbyMetadataStorage(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public MetadataStorage get()
|
||||
{
|
||||
return storage;
|
||||
}
|
||||
}
|
|
@ -20,6 +20,7 @@
|
|||
package io.druid.metadata;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import io.druid.metadata.storage.derby.DerbyConnector;
|
||||
import org.junit.Assert;
|
||||
import org.skife.jdbi.v2.DBI;
|
||||
import org.skife.jdbi.v2.exceptions.UnableToObtainConnectionException;
|
||||
|
|
|
@ -29,18 +29,20 @@ import com.metamx.common.concurrent.ScheduledExecutorFactory;
|
|||
import com.metamx.common.logger.Logger;
|
||||
import io.airlift.command.Command;
|
||||
import io.druid.client.indexing.IndexingServiceClient;
|
||||
import io.druid.metadata.MetadataRuleManager;
|
||||
import io.druid.metadata.MetadataRuleManagerConfig;
|
||||
import io.druid.metadata.MetadataRuleManagerProvider;
|
||||
import io.druid.metadata.MetadataSegmentManager;
|
||||
import io.druid.metadata.MetadataSegmentManagerConfig;
|
||||
import io.druid.metadata.MetadataSegmentManagerProvider;
|
||||
import io.druid.guice.ConfigProvider;
|
||||
import io.druid.guice.Jerseys;
|
||||
import io.druid.guice.JsonConfigProvider;
|
||||
import io.druid.guice.LazySingleton;
|
||||
import io.druid.guice.LifecycleModule;
|
||||
import io.druid.guice.ManageLifecycle;
|
||||
import io.druid.metadata.MetadataRuleManager;
|
||||
import io.druid.metadata.MetadataRuleManagerConfig;
|
||||
import io.druid.metadata.MetadataRuleManagerProvider;
|
||||
import io.druid.metadata.MetadataSegmentManager;
|
||||
import io.druid.metadata.MetadataSegmentManagerConfig;
|
||||
import io.druid.metadata.MetadataSegmentManagerProvider;
|
||||
import io.druid.metadata.MetadataStorage;
|
||||
import io.druid.metadata.MetadataStorageProvider;
|
||||
import io.druid.server.coordinator.DruidCoordinator;
|
||||
import io.druid.server.coordinator.DruidCoordinatorConfig;
|
||||
import io.druid.server.coordinator.LoadQueueTaskMaster;
|
||||
|
@ -49,9 +51,9 @@ import io.druid.server.http.BackwardsCompatibleInfoResource;
|
|||
import io.druid.server.http.CoordinatorDynamicConfigsResource;
|
||||
import io.druid.server.http.CoordinatorRedirectInfo;
|
||||
import io.druid.server.http.CoordinatorResource;
|
||||
import io.druid.server.http.MetadataResource;
|
||||
import io.druid.server.http.DatasourcesResource;
|
||||
import io.druid.server.http.InfoResource;
|
||||
import io.druid.server.http.MetadataResource;
|
||||
import io.druid.server.http.RedirectFilter;
|
||||
import io.druid.server.http.RedirectInfo;
|
||||
import io.druid.server.http.RulesResource;
|
||||
|
@ -93,6 +95,10 @@ public class CliCoordinator extends ServerRunnable
|
|||
|
||||
ConfigProvider.bind(binder, DruidCoordinatorConfig.class);
|
||||
|
||||
binder.bind(MetadataStorage.class)
|
||||
.toProvider(MetadataStorageProvider.class)
|
||||
.in(ManageLifecycle.class);
|
||||
|
||||
JsonConfigProvider.bind(binder, "druid.manager.segments", MetadataSegmentManagerConfig.class);
|
||||
JsonConfigProvider.bind(binder, "druid.manager.rules", MetadataRuleManagerConfig.class);
|
||||
|
||||
|
@ -111,6 +117,7 @@ public class CliCoordinator extends ServerRunnable
|
|||
|
||||
binder.bind(DruidCoordinator.class);
|
||||
|
||||
LifecycleModule.register(binder, MetadataStorage.class);
|
||||
LifecycleModule.register(binder, DruidCoordinator.class);
|
||||
|
||||
binder.bind(JettyServerInitializer.class)
|
||||
|
|
Loading…
Reference in New Issue