IndexerCoordinatorNode: Only create tables when in "db" storage mode

This commit is contained in:
Gian Merlino 2013-04-19 15:26:00 -07:00
parent 5edbf2b4b9
commit 376df4032d

View File

@ -46,6 +46,7 @@ import com.metamx.druid.config.ConfigManager;
import com.metamx.druid.config.ConfigManagerConfig;
import com.metamx.druid.config.JacksonConfigManager;
import com.metamx.druid.db.DbConnector;
import com.metamx.druid.db.DbConnectorConfig;
import com.metamx.druid.http.GuiceServletConfig;
import com.metamx.druid.http.RedirectFilter;
import com.metamx.druid.http.RedirectInfo;
@ -150,7 +151,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
private RestS3Service s3Service = null;
private List<Monitor> monitors = null;
private ServiceEmitter emitter = null;
private IndexerDbConnectorConfig dbConnectorConfig = null;
private DbConnectorConfig dbConnectorConfig = null;
private DBI dbi = null;
private IndexerCoordinatorConfig config = null;
private MergerDBCoordinator mergerDBCoordinator = null;
@ -250,10 +251,6 @@ public class IndexerCoordinatorNode extends RegisteringNode
final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle);
initializeDB();
DbConnector.createTaskTable(dbi, dbConnectorConfig.getTaskTable());
DbConnector.createTaskLogTable(dbi, dbConnectorConfig.getTaskLogTable());
DbConnector.createTaskLockTable(dbi, dbConnectorConfig.getTaskLockTable());
final ConfigManagerConfig managerConfig = configFactory.build(ConfigManagerConfig.class);
DbConnector.createConfigTable(dbi, managerConfig.getConfigTable());
JacksonConfigManager configManager =
@ -544,7 +541,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
private void initializeDB()
{
if (dbConnectorConfig == null) {
dbConnectorConfig = configFactory.build(IndexerDbConnectorConfig.class);
dbConnectorConfig = configFactory.build(DbConnectorConfig.class);
}
if (dbi == null) {
dbi = new DbConnector(dbConnectorConfig).getDBI();
@ -609,6 +606,10 @@ public class IndexerCoordinatorNode extends RegisteringNode
taskStorage = new HeapMemoryTaskStorage();
} else if (config.getStorageImpl().equals("db")) {
final IndexerDbConnectorConfig dbConnectorConfig = configFactory.build(IndexerDbConnectorConfig.class);
DbConnector.createTaskTable(dbi, dbConnectorConfig.getTaskTable());
DbConnector.createTaskLogTable(dbi, dbConnectorConfig.getTaskLogTable());
DbConnector.createTaskLockTable(dbi, dbConnectorConfig.getTaskLockTable());
taskStorage = new DbTaskStorage(
getJsonMapper(),
dbConnectorConfig,