Merge pull request #758 from metamx/jisoo-metadata

make metadata storage pluggable
This commit is contained in:
Fangjin Yang 2014-11-07 11:30:11 -07:00
commit 2336e6c167
115 changed files with 3866 additions and 1600 deletions

View File

@ -46,9 +46,8 @@
<artifactId>commons-codec</artifactId>
</dependency>
<dependency>
<groupId>commons-dbcp</groupId>
<artifactId>commons-dbcp</artifactId>
<version>1.4</version>
<groupId>org.apache.commons</groupId>
<artifactId>commons-dbcp2</artifactId>
</dependency>
<dependency>
<groupId>commons-pool</groupId>

View File

@ -27,8 +27,8 @@ import com.metamx.common.concurrent.ScheduledExecutors;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import io.druid.db.DbConnector;
import io.druid.db.DbTablesConfig;
import io.druid.metadata.MetadataStorageConnector;
import io.druid.metadata.MetadataStorageTablesConfig;
import org.joda.time.Duration;
import java.util.Arrays;
@ -48,7 +48,7 @@ public class ConfigManager
private final Object lock = new Object();
private boolean started = false;
private final DbConnector dbConnector;
private final MetadataStorageConnector dbConnector;
private final Supplier<ConfigManagerConfig> config;
private final ScheduledExecutorService exec;
@ -58,7 +58,7 @@ public class ConfigManager
private volatile ConfigManager.PollingCallable poller;
@Inject
public ConfigManager(DbConnector dbConnector, Supplier<DbTablesConfig> dbTables, Supplier<ConfigManagerConfig> config)
public ConfigManager(MetadataStorageConnector dbConnector, Supplier<MetadataStorageTablesConfig> dbTables, Supplier<ConfigManagerConfig> config)
{
this.dbConnector = dbConnector;
this.config = config;

View File

@ -1,389 +0,0 @@
/*
* 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.db;
import com.google.common.base.Supplier;
import com.google.inject.Inject;
import com.metamx.common.ISE;
import com.metamx.common.logger.Logger;
import org.apache.commons.dbcp.BasicDataSource;
import org.skife.jdbi.v2.DBI;
import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.IDBI;
import org.skife.jdbi.v2.StatementContext;
import org.skife.jdbi.v2.tweak.HandleCallback;
import org.skife.jdbi.v2.tweak.ResultSetMapper;
import javax.sql.DataSource;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.util.List;
import java.util.Map;
/**
*/
public class DbConnector
{
private static final Logger log = new Logger(DbConnector.class);
public static void createSegmentTable(final IDBI dbi, final String segmentTableName, boolean isPostgreSQL)
{
createTable(
dbi,
segmentTableName,
String.format(
isPostgreSQL ?
"CREATE TABLE %1$s (id VARCHAR(255) NOT NULL, dataSource VARCHAR(255) NOT NULL, created_date TEXT NOT NULL, start TEXT NOT NULL, \"end\" TEXT NOT NULL, partitioned SMALLINT NOT NULL, version TEXT NOT NULL, used BOOLEAN NOT NULL, payload bytea NOT NULL, PRIMARY KEY (id));" +
"CREATE INDEX ON %1$s(dataSource);"+
"CREATE INDEX ON %1$s(used);":
"CREATE table %s (id VARCHAR(255) NOT NULL, dataSource VARCHAR(255) NOT NULL, created_date TINYTEXT NOT NULL, start TINYTEXT NOT NULL, end TINYTEXT NOT NULL, partitioned BOOLEAN NOT NULL, version TINYTEXT NOT NULL, used BOOLEAN NOT NULL, payload LONGTEXT NOT NULL, INDEX(dataSource), INDEX(used), PRIMARY KEY (id))",
segmentTableName
),
isPostgreSQL
);
}
public static void createRuleTable(final IDBI dbi, final String ruleTableName, boolean isPostgreSQL)
{
createTable(
dbi,
ruleTableName,
String.format(
isPostgreSQL ?
"CREATE TABLE %1$s (id VARCHAR(255) NOT NULL, dataSource VARCHAR(255) NOT NULL, version TEXT NOT NULL, payload bytea NOT NULL, PRIMARY KEY (id));"+
"CREATE INDEX ON %1$s(dataSource);":
"CREATE table %s (id VARCHAR(255) NOT NULL, dataSource VARCHAR(255) NOT NULL, version TINYTEXT NOT NULL, payload LONGTEXT NOT NULL, INDEX(dataSource), PRIMARY KEY (id))",
ruleTableName
),
isPostgreSQL
);
}
public static void createConfigTable(final IDBI dbi, final String configTableName, boolean isPostgreSQL)
{
createTable(
dbi,
configTableName,
String.format(
isPostgreSQL ?
"CREATE TABLE %s (name VARCHAR(255) NOT NULL, payload bytea NOT NULL, PRIMARY KEY(name))":
"CREATE table %s (name VARCHAR(255) NOT NULL, payload BLOB NOT NULL, PRIMARY KEY(name))",
configTableName
),
isPostgreSQL
);
}
public static void createTaskTable(final IDBI dbi, final String taskTableName, boolean isPostgreSQL)
{
createTable(
dbi,
taskTableName,
String.format(
isPostgreSQL ?
"CREATE TABLE %1$s (\n"
+ " id varchar(255) NOT NULL,\n"
+ " created_date TEXT NOT NULL,\n"
+ " datasource varchar(255) NOT NULL,\n"
+ " payload bytea NOT NULL,\n"
+ " status_payload bytea NOT NULL,\n"
+ " active SMALLINT NOT NULL DEFAULT '0',\n"
+ " PRIMARY KEY (id)\n"
+ ");\n" +
"CREATE INDEX ON %1$s(active, created_date);":
"CREATE TABLE `%s` (\n"
+ " `id` varchar(255) NOT NULL,\n"
+ " `created_date` tinytext NOT NULL,\n"
+ " `datasource` varchar(255) NOT NULL,\n"
+ " `payload` longblob NOT NULL,\n"
+ " `status_payload` longblob NOT NULL,\n"
+ " `active` tinyint(1) NOT NULL DEFAULT '0',\n"
+ " PRIMARY KEY (`id`),\n"
+ " KEY (active, created_date(100))\n"
+ ")",
taskTableName
),
isPostgreSQL
);
}
public static void createTaskLogTable(final IDBI dbi, final String taskLogsTableName, boolean isPostgreSQL)
{
createTable(
dbi,
taskLogsTableName,
String.format(
isPostgreSQL ?
"CREATE TABLE %1$s (\n"
+ " id bigserial NOT NULL,\n"
+ " task_id varchar(255) DEFAULT NULL,\n"
+ " log_payload bytea,\n"
+ " PRIMARY KEY (id)\n"
+ ");\n"+
"CREATE INDEX ON %1$s(task_id);":
"CREATE TABLE `%s` (\n"
+ " `id` bigint(20) NOT NULL AUTO_INCREMENT,\n"
+ " `task_id` varchar(255) DEFAULT NULL,\n"
+ " `log_payload` longblob,\n"
+ " PRIMARY KEY (`id`),\n"
+ " KEY `task_id` (`task_id`)\n"
+ ")",
taskLogsTableName
),
isPostgreSQL
);
}
public static void createTaskLockTable(final IDBI dbi, final String taskLocksTableName, boolean isPostgreSQL)
{
createTable(
dbi,
taskLocksTableName,
String.format(
isPostgreSQL ?
"CREATE TABLE %1$s (\n"
+ " id bigserial NOT NULL,\n"
+ " task_id varchar(255) DEFAULT NULL,\n"
+ " lock_payload bytea,\n"
+ " PRIMARY KEY (id)\n"
+ ");\n"+
"CREATE INDEX ON %1$s(task_id);":
"CREATE TABLE `%s` (\n"
+ " `id` bigint(20) NOT NULL AUTO_INCREMENT,\n"
+ " `task_id` varchar(255) DEFAULT NULL,\n"
+ " `lock_payload` longblob,\n"
+ " PRIMARY KEY (`id`),\n"
+ " KEY `task_id` (`task_id`)\n"
+ ")",
taskLocksTableName
),
isPostgreSQL
);
}
public static void createTable(
final IDBI dbi,
final String tableName,
final String sql,
final boolean isPostgreSQL
)
{
try {
dbi.withHandle(
new HandleCallback<Void>()
{
@Override
public Void withHandle(Handle handle) throws Exception
{
List<Map<String, Object>> table;
if ( isPostgreSQL ) {
table = handle.select(String.format("SELECT tablename FROM pg_catalog.pg_tables WHERE schemaname = 'public' AND tablename LIKE '%s'", tableName));
} else {
table = handle.select(String.format("SHOW tables LIKE '%s'", tableName));
}
if (table.isEmpty()) {
log.info("Creating table[%s]", tableName);
handle.createStatement(sql).execute();
} else {
log.info("Table[%s] existed: [%s]", tableName, table);
}
return null;
}
}
);
}
catch (Exception e) {
log.warn(e, "Exception creating table");
}
}
public Void insertOrUpdate(
final String tableName,
final String keyColumn,
final String valueColumn,
final String key,
final byte[] value
) throws SQLException
{
final String insertOrUpdateStatement = String.format(
isPostgreSQL ?
"BEGIN;\n" +
"LOCK TABLE %1$s IN SHARE ROW EXCLUSIVE MODE;\n" +
"WITH upsert AS (UPDATE %1$s SET %3$s=:value WHERE %2$s=:key RETURNING *)\n" +
" INSERT INTO %1$s (%2$s, %3$s) SELECT :key, :value WHERE NOT EXISTS (SELECT * FROM upsert)\n;" +
"COMMIT;" :
"INSERT INTO %1$s (%2$s, %3$s) VALUES (:key, :value) ON DUPLICATE KEY UPDATE %3$s = :value",
tableName, keyColumn, valueColumn
);
return dbi.withHandle(
new HandleCallback<Void>()
{
@Override
public Void withHandle(Handle handle) throws Exception
{
handle.createStatement(insertOrUpdateStatement)
.bind("key", key)
.bind("value", value)
.execute();
return null;
}
}
);
}
public byte[] lookup(
final String tableName,
final String keyColumn,
final String valueColumn,
final String key
)
{
final String selectStatement = String.format("SELECT %s FROM %s WHERE %s = :key", valueColumn, tableName, keyColumn);
return dbi.withHandle(
new HandleCallback<byte[]>()
{
@Override
public byte[] withHandle(Handle handle) throws Exception
{
List<byte[]> matched = handle.createQuery(selectStatement)
.bind("key", key)
.map(
new ResultSetMapper<byte[]>()
{
@Override
public byte[] map(int index, ResultSet r, StatementContext ctx)
throws SQLException
{
return r.getBytes(valueColumn);
}
}
).list();
if (matched.isEmpty()) {
return null;
}
if (matched.size() > 1) {
throw new ISE("Error! More than one matching entry[%d] found for [%s]?!", matched.size(), key);
}
return matched.get(0);
}
}
);
}
public static Boolean isPostgreSQL(final IDBI dbi)
{
return dbi.withHandle(
new HandleCallback<Boolean>()
{
@Override
public Boolean withHandle(Handle handle) throws Exception
{
return isPostgreSQL(handle);
}
}
);
}
protected static Boolean isPostgreSQL(final Handle handle) throws SQLException
{
return handle.getConnection().getMetaData().getDatabaseProductName().contains("PostgreSQL");
}
private final Supplier<DbConnectorConfig> config;
private final Supplier<DbTablesConfig> dbTables;
private final DBI dbi;
private boolean isPostgreSQL = false;
@Inject
public DbConnector(Supplier<DbConnectorConfig> config, Supplier<DbTablesConfig> dbTables)
{
this.config = config;
this.dbTables = dbTables;
this.dbi = new DBI(getDatasource());
}
public DBI getDBI()
{
return dbi;
}
public boolean isPostgreSQL()
{
return isPostgreSQL;
}
private DataSource getDatasource()
{
DbConnectorConfig connectorConfig = config.get();
BasicDataSource dataSource = new BasicDataSource();
dataSource.setUsername(connectorConfig.getUser());
dataSource.setPassword(connectorConfig.getPassword());
String uri = connectorConfig.getConnectURI();
isPostgreSQL = uri.startsWith("jdbc:postgresql");
dataSource.setUrl(uri);
if (connectorConfig.isUseValidationQuery()) {
dataSource.setValidationQuery(connectorConfig.getValidationQuery());
dataSource.setTestOnBorrow(true);
}
return dataSource;
}
public void createSegmentTable()
{
if (config.get().isCreateTables()) {
createSegmentTable(dbi, dbTables.get().getSegmentsTable(), isPostgreSQL);
}
}
public void createRulesTable()
{
if (config.get().isCreateTables()) {
createRuleTable(dbi, dbTables.get().getRulesTable(), isPostgreSQL);
}
}
public void createConfigTable()
{
if (config.get().isCreateTables()) {
createConfigTable(dbi, dbTables.get().getConfigTable(), isPostgreSQL);
}
}
public void createTaskTables()
{
if (config.get().isCreateTables()) {
final DbTablesConfig dbTablesConfig = dbTables.get();
createTaskTable(dbi, dbTablesConfig.getTasksTable(), isPostgreSQL);
createTaskLogTable(dbi, dbTablesConfig.getTaskLogTable(), isPostgreSQL);
createTaskLockTable(dbi, dbTablesConfig.getTaskLockTable(), isPostgreSQL);
}
}
}

View File

@ -27,8 +27,8 @@ import com.metamx.common.lifecycle.Lifecycle;
import io.druid.common.config.ConfigManager;
import io.druid.common.config.ConfigManagerConfig;
import io.druid.common.config.JacksonConfigManager;
import io.druid.db.DbConnector;
import io.druid.db.DbTablesConfig;
import io.druid.metadata.MetadataStorageConnector;
import io.druid.metadata.MetadataStorageTablesConfig;
/**
*/
@ -43,8 +43,8 @@ public class JacksonConfigManagerModule implements Module
@Provides @ManageLifecycle
public ConfigManager getConfigManager(
final DbConnector dbConnector,
final Supplier<DbTablesConfig> dbTables,
final MetadataStorageConnector dbConnector,
final Supplier<MetadataStorageTablesConfig> dbTables,
final Supplier<ConfigManagerConfig> config,
final Lifecycle lifecycle
)

View File

@ -0,0 +1,30 @@
/*
* 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.indexer;
import com.fasterxml.jackson.databind.ObjectMapper;
import io.druid.timeline.DataSegment;
import java.util.List;
public interface MetadataStorageUpdaterJobHandler
{
public void publishSegments(String tableName, List<DataSegment> segments, ObjectMapper mapper);
}

View File

@ -17,25 +17,25 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.indexing.overlord;
package io.druid.metadata;
public class TaskExistsException extends Exception
public class EntryExistsException extends Exception
{
private final String taskId;
private final String entryId;
public TaskExistsException(String taskId, Throwable t)
public EntryExistsException(String entryId, Throwable t)
{
super(String.format("Task exists: %s", taskId), t);
this.taskId = taskId;
super(String.format("Entry already exists: %s", entryId), t);
this.entryId = entryId;
}
public TaskExistsException(String taskId)
public EntryExistsException(String entryId)
{
this(taskId, null);
this(entryId, null);
}
public String getTaskId()
public String getEntryId()
{
return taskId;
return entryId;
}
}

View File

@ -0,0 +1,137 @@
/*
* 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;
import com.google.common.base.Optional;
import com.metamx.common.Pair;
import org.joda.time.DateTime;
import javax.annotation.Nullable;
import javax.validation.constraints.NotNull;
import java.util.List;
import java.util.Map;
public interface MetadataStorageActionHandler<EntryType, StatusType, LogType, LockType>
{
/**
* Creates a new entry.
*
* @param id entry id
* @param timestamp timestamp this entry was created
* @param dataSource datasource associated with this entry
* @param entry object representing this entry
* @param active active or inactive flag
* @param status status object associated wit this object, can be null
* @throws EntryExistsException
*/
public void insert(
@NotNull String id,
@NotNull DateTime timestamp,
@NotNull String dataSource,
@NotNull EntryType entry,
boolean active,
@Nullable StatusType status
) throws EntryExistsException;
/**
* Sets or updates the status for any active entry with the given id.
* Once an entry has been set inactive, its status cannot be updated anymore.
*
* @param entryId entry id
* @param active active
* @param status status
* @return true if the status was updated, false if the entry did not exist of if the entry was inactive
*/
public boolean setStatus(String entryId, boolean active, StatusType status);
/**
* Retrieves the entry with the given id.
*
* @param entryId entry id
* @return optional entry, absent if the given id does not exist
*/
public Optional<EntryType> getEntry(String entryId);
/**
* Retrieve the status for the entry with the given id.
*
* @param entryId entry id
* @return optional status, absent if entry does not exist or status is not set
*/
public Optional<StatusType> getStatus(String entryId);
/**
* Return all active entries with their respective status
*
* @return list of (entry, status) pairs
*/
public List<Pair<EntryType, StatusType>> getActiveEntriesWithStatus();
/**
* Return all statuses for inactive entries created on or later than the given timestamp
*
* @param timestamp timestamp
* @return list of statuses
*/
public List<StatusType> getInactiveStatusesSince(DateTime timestamp);
/**
* Add a lock to the given entry
*
* @param entryId entry id
* @param lock lock to add
* @return true if the lock was added
*/
public boolean addLock(String entryId, LockType lock);
/**
* Remove the lock with the given lock id.
*
* @param lockId lock id
* @return true if the lock was removed, false if the given lock id did not exist
*/
public boolean removeLock(long lockId);
/**
* Add a log to the entry with the given id.
*
* @param entryId entry id
* @param log log to add
* @return true if the log was added
*/
public boolean addLog(String entryId, LogType log);
/**
* Returns the logs for the entry with the given id.
*
* @param entryId entry id
* @return list of logs
*/
public List<LogType> getLogs(String entryId);
/**
* Returns the locks for the given entry
*
* @param entryId entry id
* @return map of lockId to lock
*/
public Map<Long, LockType> getLocks(String entryId);
}

View File

@ -0,0 +1,28 @@
/*
* 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;
public interface MetadataStorageActionHandlerFactory
{
public <EntryType, StatusType, LogType, LockType> MetadataStorageActionHandler<EntryType, StatusType, LogType, LockType> create(
final String entryType,
MetadataStorageActionHandlerTypes<EntryType, StatusType, LogType, LockType> payloadTypes
);
}

View File

@ -0,0 +1,30 @@
/*
* 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.fasterxml.jackson.core.type.TypeReference;
public interface MetadataStorageActionHandlerTypes<EntryType, StatusType, LogType, LockType>
{
public TypeReference<EntryType> getEntryType();
public TypeReference<StatusType> getStatusType();
public TypeReference<LogType> getLogType();
public TypeReference<LockType> getLockType();
}

View File

@ -0,0 +1,49 @@
/*
* 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;
/**
*/
public interface MetadataStorageConnector
{
public Void insertOrUpdate(
final String tableName,
final String keyColumn,
final String valueColumn,
final String key,
final byte[] value
) throws Exception;
public byte[] lookup(
final String tableName,
final String keyColumn,
final String valueColumn,
final String key
);
public void createSegmentTable();
public void createRulesTable();
public void createConfigTable();
public void createTaskTables();
}

View File

@ -17,34 +17,26 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.db;
package io.druid.metadata;
import com.fasterxml.jackson.annotation.JsonProperty;
import javax.validation.constraints.NotNull;
/**
*/
public class DbConnectorConfig
public class MetadataStorageConnectorConfig
{
@JsonProperty
private boolean createTables = true;
@JsonProperty
@NotNull
private String connectURI = null;
@JsonProperty
@NotNull
private String user = null;
@JsonProperty
@NotNull
private String password = null;
@JsonProperty
private boolean useValidationQuery = false;
@JsonProperty
private String validationQuery = "SELECT 1";
@ -68,12 +60,8 @@ public class DbConnectorConfig
return password;
}
public boolean isUseValidationQuery()
public String getValidationQuery()
{
return useValidationQuery;
}
public String getValidationQuery() {
return validationQuery;
}
@ -85,7 +73,6 @@ public class DbConnectorConfig
", connectURI='" + connectURI + '\'' +
", user='" + user + '\'' +
", password=****" +
", useValidationQuery=" + useValidationQuery +
", validationQuery='" + validationQuery + '\'' +
'}';
}

View File

@ -17,21 +17,30 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.db;
package io.druid.metadata;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.Maps;
import java.util.Map;
/**
*/
public class DbTablesConfig
public class MetadataStorageTablesConfig
{
public static DbTablesConfig fromBase(String base)
public static MetadataStorageTablesConfig fromBase(String base)
{
return new DbTablesConfig(base, null, null, null, null, null, null);
return new MetadataStorageTablesConfig(base, null, null, null, null, null, null);
}
private static String defaultBase = "druid";
public static final String TASK_ENTRY_TYPE = "task";
private static final String DEFAULT_BASE = "druid";
private final Map<String, String> entryTables = Maps.newHashMap();
private final Map<String, String> logTables = Maps.newHashMap();
private final Map<String, String> lockTables = Maps.newHashMap();
@JsonProperty("base")
private final String base;
@ -55,7 +64,7 @@ public class DbTablesConfig
private final String taskLockTable;
@JsonCreator
public DbTablesConfig(
public MetadataStorageTablesConfig(
@JsonProperty("base") String base,
@JsonProperty("segments") String segmentsTable,
@JsonProperty("rules") String rulesTable,
@ -65,13 +74,18 @@ public class DbTablesConfig
@JsonProperty("taskLock") String taskLockTable
)
{
this.base = (base == null) ? defaultBase : base;
this.base = (base == null) ? DEFAULT_BASE : base;
this.segmentsTable = makeTableName(segmentsTable, "segments");
this.rulesTable = makeTableName(rulesTable, "rules");
this.configTable = makeTableName(configTable, "config");
this.tasksTable = makeTableName(tasksTable, "tasks");
this.taskLogTable = makeTableName(taskLogTable, "tasklogs");
this.taskLockTable = makeTableName(taskLockTable, "tasklocks");
entryTables.put(TASK_ENTRY_TYPE, this.tasksTable);
logTables.put(TASK_ENTRY_TYPE, this.taskLogTable);
lockTables.put(TASK_ENTRY_TYPE, this.taskLockTable);
}
private String makeTableName(String explicitTableName, String defaultSuffix)
@ -106,18 +120,23 @@ public class DbTablesConfig
return configTable;
}
public String getTasksTable()
public String getEntryTable(final String entryType)
{
return tasksTable;
return entryTables.get(entryType);
}
public String getTaskLogTable()
public String getLogTable(final String entryType)
{
return taskLogTable;
return logTables.get(entryType);
}
public String getTaskLockTable()
public String getLockTable(final String entryType)
{
return taskLockTable;
return lockTables.get(entryType);
}
public String getTaskEntryType()
{
return TASK_ENTRY_TYPE;
}
}

View File

@ -245,18 +245,18 @@ These properties specify the jdbc connection and other configuration around the
|Property|Description|Default|
|--------|-----------|-------|
|`druid.db.connector.user`|The username to connect with.|none|
|`druid.db.connector.password`|The password to connect with.|none|
|`druid.db.connector.createTables`|If Druid requires a table and it doesn't exist, create it?|true|
|`druid.db.connector.useValidationQuery`|Validate a table with a query.|false|
|`druid.db.connector.validationQuery`|The query to validate with.|SELECT 1|
|`druid.db.tables.base`|The base name for tables.|druid|
|`druid.db.tables.segmentTable`|The table to use to look for segments.|druid_segments|
|`druid.db.tables.ruleTable`|The table to use to look for segment load/drop rules.|druid_rules|
|`druid.db.tables.configTable`|The table to use to look for configs.|druid_config|
|`druid.db.tables.tasks`|Used by the indexing service to store tasks.|druid_tasks|
|`druid.db.tables.taskLog`|Used by the indexing service to store task logs.|druid_taskLog|
|`druid.db.tables.taskLock`|Used by the indexing service to store task locks.|druid_taskLock|
|`druid.metadata.storage.connector.user`|The username to connect with.|none|
|`druid.metadata.storage.connector.password`|The password to connect with.|none|
|`druid.metadata.storage.connector.createTables`|If Druid requires a table and it doesn't exist, create it?|true|
|`druid.metadata.storage.connector.useValidationQuery`|Validate a table with a query.|false|
|`druid.metadata.storage.connector.validationQuery`|The query to validate with.|SELECT 1|
|`druid.metadata.storage.tables.base`|The base name for tables.|druid|
|`druid.metadata.storage.tables.segmentTable`|The table to use to look for segments.|druid_segments|
|`druid.metadata.storage.tables.ruleTable`|The table to use to look for segment load/drop rules.|druid_rules|
|`druid.metadata.storage.tables.configTable`|The table to use to look for configs.|druid_config|
|`druid.metadata.storage.tables.tasks`|Used by the indexing service to store tasks.|druid_tasks|
|`druid.metadata.storage.tables.taskLog`|Used by the indexing service to store task logs.|druid_taskLog|
|`druid.metadata.storage.tables.taskLock`|Used by the indexing service to store task locks.|druid_taskLock|
### Jackson Config Manager Module

View File

@ -10,7 +10,7 @@ In addition to the configuration of some of the default modules in [Configuratio
|Property|Description|Default|
|--------|-----------|-------|
|`druid.indexer.runner.type`|Choices "local" or "remote". Indicates whether tasks should be run locally or in a distributed environment.|local|
|`druid.indexer.storage.type`|Choices are "local" or "db". Indicates whether incoming tasks should be stored locally (in heap) or in a database. Storing incoming tasks in a database allows for tasks to be resumed if the overlord should fail.|local|
|`druid.indexer.storage.type`|Choices are "local" or "metadata". Indicates whether incoming tasks should be stored locally (in heap) or in metadata storage. Storing incoming tasks in metadata storage allows for tasks to be resumed if the overlord should fail.|local|
|`druid.indexer.storage.recentlyFinishedThreshold`|A duration of time to store task results.|PT24H|
|`druid.indexer.queue.maxSize`|Maximum number of active tasks at one time.|Integer.MAX_VALUE|
|`druid.indexer.queue.startDelay`|Sleep this long before starting overlord queue management. This can be useful to give a cluster time to re-orient itself after e.g. a widespread network issue.|PT1M|

View File

@ -12,14 +12,14 @@ Depending on what `druid.storage.type` is set to, Druid will upload segments to
## My realtime node is not handing segments off
Make sure that the `druid.publish.type` on your real-time nodes is set to "db". Also make sure that `druid.storage.type` is set to a deep storage that makes sense. Some example configs:
Make sure that the `druid.publish.type` on your real-time nodes is set to `metadata`. Also make sure that `druid.storage.type` is set to a deep storage that makes sense. Some example configs:
```
druid.publish.type=db
druid.publish.type=metadata
druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
druid.db.connector.user=druid
druid.db.connector.password=diurd
druid.metadata.storage.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
druid.metadata.storage.connector.user=druid
druid.metadata.storage.connector.password=diurd
druid.storage.type=s3
druid.storage.bucket=druid

View File

@ -29,9 +29,9 @@ With the following JVM configuration:
-Ddruid.zk.service.host=localhost
-Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid
-Ddruid.db.connector.user=druid
-Ddruid.db.connector.password=diurd
-Ddruid.metadata.storage.connector.connectURI=jdbc:mysql://localhost:3306/druid
-Ddruid.metadata.storage.connector.user=druid
-Ddruid.metadata.storage.connector.password=diurd
-Ddruid.selectors.indexing.serviceName=overlord
-Ddruid.indexer.runner.startPort=8092
-Ddruid.indexer.fork.property.druid.computation.buffer.size=268435456

View File

@ -7,7 +7,7 @@ MySQL is an external dependency of Druid. We use it to store various metadata ab
Segments Table
--------------
This is dictated by the `druid.db.tables.segments` property.
This is dictated by the `druid.metadata.storage.tables.segments` property.
This table stores metadata about the segments that are available in the system. The table is polled by the [Coordinator](Coordinator.html) to determine the set of segments that should be available for querying in the system. The table has two main functional columns, the other columns are for indexing purposes.

View File

@ -67,11 +67,11 @@ druid.discovery.curator.path=/prod/discovery
druid.s3.accessKey=#{ACCESS_KEY}
druid.s3.secretKey=#{SECRET_KEY}
druid.db.connector.connectURI=jdbc:mysql://#{MYSQL_URL}:3306/druid
druid.db.connector.user=#{MYSQL_USER}
druid.db.connector.password=#{MYSQL_PW}
druid.db.connector.useValidationQuery=true
druid.db.tables.base=prod
druid.metadata.storage.connector.connectURI=jdbc:mysql://#{MYSQL_URL}:3306/druid
druid.metadata.storage.connector.user=#{MYSQL_USER}
druid.metadata.storage.connector.password=#{MYSQL_PW}
druid.metadata.storage.connector.useValidationQuery=true
druid.metadata.storage.tables.base=prod
# Only required if you are autoscaling middle managers
druid.indexer.autoscale.doAutoscale=true
@ -91,7 +91,7 @@ druid.indexer.runner.compressZnodes=true
druid.indexer.runner.minWorkerVersion=#{WORKER_VERSION}
# Store all task state in MySQL
druid.indexer.storage.type=db
druid.indexer.storage.type=metadata
druid.monitoring.monitors=["com.metamx.metrics.SysMonitor","com.metamx.metrics.JvmMonitor"]
@ -229,11 +229,11 @@ druid.zk.paths.base=/druid/prod
druid.discovery.curator.path=/prod/discovery
druid.db.connector.connectURI=jdbc:mysql://#{MYSQL_URL}:3306/druid
druid.db.connector.user=#{MYSQL_USER}
druid.db.connector.password=#{MYSQL_PW}
druid.db.connector.useValidationQuery=true
druid.db.tables.base=prod
druid.metadata.storage.connector.connectURI=jdbc:mysql://#{MYSQL_URL}:3306/druid
druid.metadata.storage.connector.user=#{MYSQL_USER}
druid.metadata.storage.connector.password=#{MYSQL_PW}
druid.metadata.storage.connector.useValidationQuery=true
druid.metadata.storage.tables.base=prod
druid.selectors.indexing.serviceName=druid:prod:overlord
@ -387,4 +387,4 @@ druid.emitter.http.recipientBaseUrl=#{EMITTER_URL}
# If you choose to compress ZK announcements, you must do so for every node type
druid.announcer.type=batch
druid.curator.compress=true
```
```

View File

@ -35,12 +35,12 @@ druid.zk.service.host=localhost
# The realtime config file.
druid.realtime.specFile=/path/to/specFile
# Choices: db (hand off segments), noop (do not hand off segments).
druid.publish.type=db
# Choices: metadata (hand off segments), noop (do not hand off segments).
druid.publish.type=metadata
druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
druid.db.connector.user=druid
druid.db.connector.password=diurd
druid.metadata.storage.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
druid.metadata.storage.connector.user=druid
druid.metadata.storage.connector.password=diurd
druid.processing.buffer.sizeBytes=100000000
```
@ -84,13 +84,13 @@ druid.zk.paths.base=/druid/prod
druid.s3.accessKey=#{ACCESS_KEY}
druid.s3.secretKey=#{SECRET_KEY}
druid.db.connector.connectURI=jdbc:mysql://#{MYSQL_URL}:3306/druid
druid.db.connector.user=#{MYSQL_USER}
druid.db.connector.password=#{MYSQL_PW}
druid.db.connector.useValidationQuery=true
druid.db.tables.base=prod
druid.metadata.storage.connector.connectURI=jdbc:mysql://#{MYSQL_URL}:3306/druid
druid.metadata.storage.connector.user=#{MYSQL_USER}
druid.metadata.storage.connector.password=#{MYSQL_PW}
druid.metadata.storage.connector.useValidationQuery=true
druid.metadata.storage.tables.base=prod
druid.publish.type=db
druid.publish.type=metadata
druid.processing.numThreads=3

View File

@ -30,9 +30,9 @@ Configuration:
-Ddruid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.160"]
-Ddruid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid
-Ddruid.db.connector.user=druid
-Ddruid.db.connector.password=diurd
-Ddruid.metadata.storage.connector.connectURI=jdbc:mysql://localhost:3306/druid
-Ddruid.metadata.storage.connector.user=druid
-Ddruid.metadata.storage.connector.password=diurd
-Ddruid.selectors.indexing.serviceName=overlord
-Ddruid.indexer.queue.startDelay=PT0M
@ -66,9 +66,9 @@ druid.port=8082
druid.zk.service.host=localhost
druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
druid.db.connector.user=druid
druid.db.connector.password=diurd
druid.metadata.storage.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
druid.metadata.storage.connector.user=druid
druid.metadata.storage.connector.password=diurd
druid.coordinator.startDelay=PT70s
```

View File

@ -93,9 +93,9 @@ druid.zk.service.host=localhost
druid.extensions.coordinates=["io.druid.extensions:druid-kafka-seven:0.6.160"]
druid.db.connector.connectURI=jdbc:mysql://localhost:3306/druid
druid.db.connector.user=druid
druid.db.connector.password=diurd
druid.metadata.storage.connector.connectURI=jdbc:mysql://localhost:3306/druid
druid.metadata.storage.connector.user=druid
druid.metadata.storage.connector.password=diurd
druid.selectors.indexing.serviceName=overlord
druid.indexer.queue.startDelay=PT0M

View File

@ -116,9 +116,9 @@ druid.port=8082
druid.zk.service.host=localhost
druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
druid.db.connector.user=druid
druid.db.connector.password=diurd
druid.metadata.storage.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
druid.metadata.storage.connector.user=druid
druid.metadata.storage.connector.password=diurd
druid.coordinator.startDelay=PT70s
```
@ -246,9 +246,9 @@ druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.6.160","io.d
druid.publish.type=noop
# These configs are only required for real hand off
# druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
# druid.db.connector.user=druid
# druid.db.connector.password=diurd
# druid.metadata.storage.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
# druid.metadata.storage.connector.user=druid
# druid.metadata.storage.connector.password=diurd
druid.processing.buffer.sizeBytes=100000000
druid.processing.numThreads=1

View File

@ -5,9 +5,10 @@ druid.extensions.coordinates=["io.druid.extensions:druid-examples:0.7.0","io.dru
druid.zk.service.host=localhost
# Metadata Storage
druid.db.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
druid.db.connector.user=druid
druid.db.connector.password=diurd
druid.metadata.storage.type=mysql
druid.metadata.storage.connector.connectURI=jdbc\:mysql\://localhost\:3306/druid
druid.metadata.storage.connector.user=druid
druid.metadata.storage.connector.password=diurd
# Deep storage
druid.storage.type=local

View File

@ -2,7 +2,7 @@ druid.host=localhost
druid.service=realtime
druid.port=8083
# Change this config to db to hand off to the rest of the Druid cluster
# Change this config to metadata to hand off to the rest of the Druid cluster
druid.publish.type=noop
druid.processing.buffer.sizeBytes=100000000

View File

@ -32,7 +32,7 @@ public class HadoopDruidIndexerJob implements Jobby
{
private static final Logger log = new Logger(HadoopDruidIndexerJob.class);
private final HadoopDruidIndexerConfig config;
private final DbUpdaterJob dbUpdaterJob;
private final MetadataStorageUpdaterJob metadataStorageUpdaterJob;
private IndexGeneratorJob indexJob;
private volatile List<DataSegment> publishedSegments = null;
@ -45,9 +45,9 @@ public class HadoopDruidIndexerJob implements Jobby
this.config = config;
if (config.isUpdaterJobSpecSet()) {
dbUpdaterJob = new DbUpdaterJob(config);
metadataStorageUpdaterJob = new MetadataStorageUpdaterJob(config);
} else {
dbUpdaterJob = null;
metadataStorageUpdaterJob = null;
}
}
@ -64,8 +64,8 @@ public class HadoopDruidIndexerJob implements Jobby
}
jobs.add(indexJob);
if (dbUpdaterJob != null) {
jobs.add(dbUpdaterJob);
if (metadataStorageUpdaterJob != null) {
jobs.add(metadataStorageUpdaterJob);
} else {
log.info("No updaterJobSpec set, not uploading to database");
}

View File

@ -21,7 +21,7 @@ package io.druid.indexer;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import io.druid.indexer.updater.DbUpdaterJobSpec;
import io.druid.indexer.updater.MetadataStorageUpdaterJobSpec;
import io.druid.segment.indexing.IOConfig;
import java.util.Map;
@ -32,12 +32,12 @@ import java.util.Map;
public class HadoopIOConfig implements IOConfig
{
private final Map<String, Object> pathSpec;
private final DbUpdaterJobSpec metadataUpdateSpec;
private final MetadataStorageUpdaterJobSpec metadataUpdateSpec;
private final String segmentOutputPath;
public HadoopIOConfig(
final @JsonProperty("inputSpec") Map<String, Object> pathSpec,
final @JsonProperty("metadataUpdateSpec") DbUpdaterJobSpec metadataUpdateSpec,
final @JsonProperty("metadataUpdateSpec") MetadataStorageUpdaterJobSpec metadataUpdateSpec,
final @JsonProperty("segmentOutputPath") String segmentOutputPath
)
{
@ -53,7 +53,7 @@ public class HadoopIOConfig implements IOConfig
}
@JsonProperty("metadataUpdateSpec")
public DbUpdaterJobSpec getMetadataUpdateSpec()
public MetadataStorageUpdaterJobSpec getMetadataUpdateSpec()
{
return metadataUpdateSpec;
}

View File

@ -30,7 +30,7 @@ import io.druid.data.input.impl.TimestampSpec;
import io.druid.indexer.partitions.PartitionsSpec;
import io.druid.indexer.partitions.SingleDimensionPartitionsSpec;
import io.druid.indexer.rollup.DataRollupSpec;
import io.druid.indexer.updater.DbUpdaterJobSpec;
import io.druid.indexer.updater.MetadataStorageUpdaterJobSpec;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.IngestionSpec;
@ -70,7 +70,7 @@ public class HadoopIngestionSpec extends IngestionSpec<HadoopIOConfig, HadoopTun
final @JsonProperty("shardSpecs") Map<DateTime, List<HadoopyShardSpec>> shardSpecs,
final @JsonProperty("overwriteFiles") boolean overwriteFiles,
final @JsonProperty("rollupSpec") DataRollupSpec rollupSpec,
final @JsonProperty("updaterJobSpec") DbUpdaterJobSpec updaterJobSpec,
final @JsonProperty("updaterJobSpec") MetadataStorageUpdaterJobSpec updaterJobSpec,
final @JsonProperty("ignoreInvalidRows") boolean ignoreInvalidRows,
final @JsonProperty("jobProperties") Map<String, String> jobProperties,
final @JsonProperty("combineText") boolean combineText,

View File

@ -0,0 +1,52 @@
/*
* 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.indexer;
import com.google.inject.Inject;
import io.druid.timeline.DataSegment;
import java.util.List;
/**
*/
public class MetadataStorageUpdaterJob implements Jobby
{
private final HadoopDruidIndexerConfig config;
@Inject
private MetadataStorageUpdaterJobHandler handler;
public MetadataStorageUpdaterJob(
HadoopDruidIndexerConfig config
)
{
this.config = config;
}
@Override
public boolean run()
{
final List<DataSegment> segments = IndexGeneratorJob.getPublishedSegments(config);
final String segmentTable = config.getSchema().getIOConfig().getMetadataUpdateSpec().getSegmentTable();
handler.publishSegments(segmentTable, segments, HadoopDruidIndexerConfig.jsonMapper);
return true;
}
}

View File

@ -21,11 +21,11 @@ package io.druid.indexer.updater;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Supplier;
import io.druid.db.DbConnectorConfig;
import io.druid.metadata.MetadataStorageConnectorConfig;
/**
*/
public class DbUpdaterJobSpec implements Supplier<DbConnectorConfig>
public class MetadataStorageUpdaterJobSpec implements Supplier<MetadataStorageConnectorConfig>
{
@JsonProperty("connectURI")
public String connectURI;
@ -45,9 +45,9 @@ public class DbUpdaterJobSpec implements Supplier<DbConnectorConfig>
}
@Override
public DbConnectorConfig get()
public MetadataStorageConnectorConfig get()
{
return new DbConnectorConfig()
return new MetadataStorageConnectorConfig()
{
@Override
public String getConnectURI()

View File

@ -22,11 +22,11 @@ package io.druid.indexer;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.google.common.collect.Lists;
import io.druid.db.DbConnectorConfig;
import io.druid.metadata.MetadataStorageConnectorConfig;
import io.druid.indexer.partitions.PartitionsSpec;
import io.druid.indexer.partitions.RandomPartitionsSpec;
import io.druid.indexer.partitions.SingleDimensionPartitionsSpec;
import io.druid.indexer.updater.DbUpdaterJobSpec;
import io.druid.indexer.updater.MetadataStorageUpdaterJobSpec;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.joda.time.Interval;
@ -354,14 +354,13 @@ public class HadoopIngestionSpecTest
HadoopIngestionSpec.class
);
final DbUpdaterJobSpec spec = schema.getIOConfig().getMetadataUpdateSpec();
final DbConnectorConfig connectorConfig = spec.get();
final MetadataStorageUpdaterJobSpec spec = schema.getIOConfig().getMetadataUpdateSpec();
final MetadataStorageConnectorConfig connectorConfig = spec.get();
Assert.assertEquals("segments", spec.getSegmentTable());
Assert.assertEquals("jdbc:mysql://localhost/druid", connectorConfig.getConnectURI());
Assert.assertEquals("rofl", connectorConfig.getUser());
Assert.assertEquals("p4ssw0rd", connectorConfig.getPassword());
Assert.assertEquals(false, connectorConfig.isUseValidationQuery());
}
@Test

View File

@ -47,11 +47,6 @@
<artifactId>druid-indexing-hadoop</artifactId>
<version>${project.parent.version}</version>
</dependency>
<dependency>
<groupId>mysql</groupId>
<artifactId>mysql-connector-java</artifactId>
<version>5.1.18</version>
</dependency>
<!-- Tests -->
<dependency>

View File

@ -81,7 +81,7 @@ public class SegmentInsertAction implements TaskAction<Set<DataSegment>>
{
toolbox.verifyTaskLocksAndSinglePartitionSettitude(task, segments, true);
final Set<DataSegment> retVal = toolbox.getIndexerDBCoordinator().announceHistoricalSegments(segments);
final Set<DataSegment> retVal = toolbox.getIndexerMetadataStorageCoordinator().announceHistoricalSegments(segments);
// Emit metrics
final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder()

View File

@ -68,7 +68,7 @@ public class SegmentListUnusedAction implements TaskAction<List<DataSegment>>
@Override
public List<DataSegment> perform(Task task, TaskActionToolbox toolbox) throws IOException
{
return toolbox.getIndexerDBCoordinator().getUnusedSegmentsForInterval(dataSource, interval);
return toolbox.getIndexerMetadataStorageCoordinator().getUnusedSegmentsForInterval(dataSource, interval);
}
@Override

View File

@ -68,7 +68,7 @@ public class SegmentListUsedAction implements TaskAction<List<DataSegment>>
@Override
public List<DataSegment> perform(Task task, TaskActionToolbox toolbox) throws IOException
{
return toolbox.getIndexerDBCoordinator().getUsedSegmentsForInterval(dataSource, interval);
return toolbox.getIndexerMetadataStorageCoordinator().getUsedSegmentsForInterval(dataSource, interval);
}
@Override

View File

@ -42,7 +42,7 @@ public class SegmentMetadataUpdateAction implements TaskAction<Void>
) throws IOException
{
toolbox.verifyTaskLocksAndSinglePartitionSettitude(task, segments, true);
toolbox.getIndexerDBCoordinator().updateSegmentMetadata(segments);
toolbox.getIndexerMetadataStorageCoordinator().updateSegmentMetadata(segments);
// Emit metrics
final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder()

View File

@ -59,7 +59,7 @@ public class SegmentNukeAction implements TaskAction<Void>
public Void perform(Task task, TaskActionToolbox toolbox) throws IOException
{
toolbox.verifyTaskLocksAndSinglePartitionSettitude(task, segments, true);
toolbox.getIndexerDBCoordinator().deleteSegments(segments);
toolbox.getIndexerMetadataStorageCoordinator().deleteSegments(segments);
// Emit metrics
final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder()

View File

@ -27,7 +27,7 @@ import com.metamx.common.ISE;
import com.metamx.emitter.service.ServiceEmitter;
import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.task.Task;
import io.druid.indexing.overlord.IndexerDBCoordinator;
import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
import io.druid.indexing.overlord.TaskLockbox;
import io.druid.timeline.DataSegment;
@ -37,18 +37,18 @@ import java.util.Set;
public class TaskActionToolbox
{
private final TaskLockbox taskLockbox;
private final IndexerDBCoordinator indexerDBCoordinator;
private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
private final ServiceEmitter emitter;
@Inject
public TaskActionToolbox(
TaskLockbox taskLockbox,
IndexerDBCoordinator indexerDBCoordinator,
IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator,
ServiceEmitter emitter
)
{
this.taskLockbox = taskLockbox;
this.indexerDBCoordinator = indexerDBCoordinator;
this.indexerMetadataStorageCoordinator = indexerMetadataStorageCoordinator;
this.emitter = emitter;
}
@ -57,9 +57,9 @@ public class TaskActionToolbox
return taskLockbox;
}
public IndexerDBCoordinator getIndexerDBCoordinator()
public IndexerMetadataStorageCoordinator getIndexerMetadataStorageCoordinator()
{
return indexerDBCoordinator;
return indexerMetadataStorageCoordinator;
}
public ServiceEmitter getEmitter()

View File

@ -1,552 +0,0 @@
/*
* 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.indexing.overlord;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.inject.Inject;
import com.metamx.common.RetryUtils;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.emitter.EmittingLogger;
import com.mysql.jdbc.exceptions.MySQLTransientException;
import io.druid.db.DbConnector;
import io.druid.db.DbTablesConfig;
import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.actions.TaskAction;
import io.druid.indexing.common.config.TaskStorageConfig;
import io.druid.indexing.common.task.Task;
import org.joda.time.DateTime;
import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.IDBI;
import org.skife.jdbi.v2.exceptions.CallbackFailedException;
import org.skife.jdbi.v2.exceptions.DBIException;
import org.skife.jdbi.v2.exceptions.StatementException;
import org.skife.jdbi.v2.exceptions.UnableToObtainConnectionException;
import org.skife.jdbi.v2.tweak.HandleCallback;
import java.sql.SQLException;
import java.sql.SQLRecoverableException;
import java.sql.SQLTransientException;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Callable;
public class DbTaskStorage implements TaskStorage
{
private final ObjectMapper jsonMapper;
private final DbConnector dbConnector;
private final DbTablesConfig dbTables;
private final IDBI dbi;
private final TaskStorageConfig config;
private static final EmittingLogger log = new EmittingLogger(DbTaskStorage.class);
@Inject
public DbTaskStorage(
final ObjectMapper jsonMapper,
final DbConnector dbConnector,
final DbTablesConfig dbTables,
final IDBI dbi,
final TaskStorageConfig config
)
{
this.jsonMapper = jsonMapper;
this.dbConnector = dbConnector;
this.dbTables = dbTables;
this.dbi = dbi;
this.config = config;
}
@LifecycleStart
public void start()
{
dbConnector.createTaskTables();
}
@LifecycleStop
public void stop()
{
// do nothing
}
@Override
public void insert(final Task task, final TaskStatus status) throws TaskExistsException
{
Preconditions.checkNotNull(task, "task");
Preconditions.checkNotNull(status, "status");
Preconditions.checkArgument(
task.getId().equals(status.getId()),
"Task/Status ID mismatch[%s/%s]",
task.getId(),
status.getId()
);
log.info("Inserting task %s with status: %s", task.getId(), status);
try {
retryingHandle(
new HandleCallback<Void>()
{
@Override
public Void withHandle(Handle handle) throws Exception
{
handle.createStatement(
String.format(
"INSERT INTO %s (id, created_date, datasource, payload, active, status_payload) VALUES (:id, :created_date, :datasource, :payload, :active, :status_payload)",
dbTables.getTasksTable()
)
)
.bind("id", task.getId())
.bind("created_date", new DateTime().toString())
.bind("datasource", task.getDataSource())
.bind("payload", jsonMapper.writeValueAsBytes(task))
.bind("active", status.isRunnable() ? 1 : 0)
.bind("status_payload", jsonMapper.writeValueAsBytes(status))
.execute();
return null;
}
}
);
}
catch (Exception e) {
final boolean isStatementException = e instanceof StatementException ||
(e instanceof CallbackFailedException
&& e.getCause() instanceof StatementException);
if (isStatementException && getTask(task.getId()).isPresent()) {
throw new TaskExistsException(task.getId(), e);
} else {
throw e;
}
}
}
@Override
public void setStatus(final TaskStatus status)
{
Preconditions.checkNotNull(status, "status");
log.info("Updating task %s to status: %s", status.getId(), status);
int updated = retryingHandle(
new HandleCallback<Integer>()
{
@Override
public Integer withHandle(Handle handle) throws Exception
{
return handle.createStatement(
String.format(
"UPDATE %s SET active = :active, status_payload = :status_payload WHERE id = :id AND active = 1",
dbTables.getTasksTable()
)
)
.bind("id", status.getId())
.bind("active", status.isRunnable() ? 1 : 0)
.bind("status_payload", jsonMapper.writeValueAsBytes(status))
.execute();
}
}
);
if (updated != 1) {
throw new IllegalStateException(String.format("Active task not found: %s", status.getId()));
}
}
@Override
public Optional<Task> getTask(final String taskid)
{
return retryingHandle(
new HandleCallback<Optional<Task>>()
{
@Override
public Optional<Task> withHandle(Handle handle) throws Exception
{
final List<Map<String, Object>> dbTasks =
handle.createQuery(
String.format(
"SELECT payload FROM %s WHERE id = :id",
dbTables.getTasksTable()
)
)
.bind("id", taskid)
.list();
if (dbTasks.size() == 0) {
return Optional.absent();
} else {
final Map<String, Object> dbStatus = Iterables.getOnlyElement(dbTasks);
return Optional.of(jsonMapper.readValue((byte[]) dbStatus.get("payload"), Task.class));
}
}
}
);
}
@Override
public Optional<TaskStatus> getStatus(final String taskid)
{
return retryingHandle(
new HandleCallback<Optional<TaskStatus>>()
{
@Override
public Optional<TaskStatus> withHandle(Handle handle) throws Exception
{
final List<Map<String, Object>> dbStatuses =
handle.createQuery(
String.format(
"SELECT status_payload FROM %s WHERE id = :id",
dbTables.getTasksTable()
)
)
.bind("id", taskid)
.list();
if (dbStatuses.size() == 0) {
return Optional.absent();
} else {
final Map<String, Object> dbStatus = Iterables.getOnlyElement(dbStatuses);
return Optional.of(jsonMapper.readValue((byte[]) dbStatus.get("status_payload"), TaskStatus.class));
}
}
}
);
}
@Override
public List<Task> getActiveTasks()
{
return retryingHandle(
new HandleCallback<List<Task>>()
{
@Override
public List<Task> withHandle(Handle handle) throws Exception
{
final List<Map<String, Object>> dbTasks =
handle.createQuery(
String.format(
"SELECT id, payload, status_payload FROM %s WHERE active = 1 ORDER BY created_date",
dbTables.getTasksTable()
)
)
.list();
final ImmutableList.Builder<Task> tasks = ImmutableList.builder();
for (final Map<String, Object> row : dbTasks) {
final String id = row.get("id").toString();
try {
final Task task = jsonMapper.readValue((byte[]) row.get("payload"), Task.class);
final TaskStatus status = jsonMapper.readValue((byte[]) row.get("status_payload"), TaskStatus.class);
if (status.isRunnable()) {
tasks.add(task);
}
}
catch (Exception e) {
log.makeAlert(e, "Failed to parse task payload").addData("task", id).emit();
}
}
return tasks.build();
}
}
);
}
@Override
public List<TaskStatus> getRecentlyFinishedTaskStatuses()
{
final DateTime recent = new DateTime().minus(config.getRecentlyFinishedThreshold());
return retryingHandle(
new HandleCallback<List<TaskStatus>>()
{
@Override
public List<TaskStatus> withHandle(Handle handle) throws Exception
{
final List<Map<String, Object>> dbTasks =
handle.createQuery(
String.format(
"SELECT id, status_payload FROM %s WHERE active = 0 AND created_date >= :recent ORDER BY created_date DESC",
dbTables.getTasksTable()
)
).bind("recent", recent.toString()).list();
final ImmutableList.Builder<TaskStatus> statuses = ImmutableList.builder();
for (final Map<String, Object> row : dbTasks) {
final String id = row.get("id").toString();
try {
final TaskStatus status = jsonMapper.readValue((byte[]) row.get("status_payload"), TaskStatus.class);
if (status.isComplete()) {
statuses.add(status);
}
}
catch (Exception e) {
log.makeAlert(e, "Failed to parse status payload").addData("task", id).emit();
}
}
return statuses.build();
}
}
);
}
@Override
public void addLock(final String taskid, final TaskLock taskLock)
{
Preconditions.checkNotNull(taskid, "taskid");
Preconditions.checkNotNull(taskLock, "taskLock");
log.info(
"Adding lock on interval[%s] version[%s] for task: %s",
taskLock.getInterval(),
taskLock.getVersion(),
taskid
);
retryingHandle(
new HandleCallback<Integer>()
{
@Override
public Integer withHandle(Handle handle) throws Exception
{
return handle.createStatement(
String.format(
"INSERT INTO %s (task_id, lock_payload) VALUES (:task_id, :lock_payload)",
dbTables.getTaskLockTable()
)
)
.bind("task_id", taskid)
.bind("lock_payload", jsonMapper.writeValueAsBytes(taskLock))
.execute();
}
}
);
}
@Override
public void removeLock(String taskid, TaskLock taskLockToRemove)
{
Preconditions.checkNotNull(taskid, "taskid");
Preconditions.checkNotNull(taskLockToRemove, "taskLockToRemove");
final Map<Long, TaskLock> taskLocks = getLocksWithIds(taskid);
for (final Map.Entry<Long, TaskLock> taskLockWithId : taskLocks.entrySet()) {
final long id = taskLockWithId.getKey();
final TaskLock taskLock = taskLockWithId.getValue();
if (taskLock.equals(taskLockToRemove)) {
log.info("Deleting TaskLock with id[%d]: %s", id, taskLock);
retryingHandle(
new HandleCallback<Integer>()
{
@Override
public Integer withHandle(Handle handle) throws Exception
{
return handle.createStatement(
String.format(
"DELETE FROM %s WHERE id = :id",
dbTables.getTaskLockTable()
)
)
.bind("id", id)
.execute();
}
}
);
}
}
}
@Override
public List<TaskLock> getLocks(String taskid)
{
return ImmutableList.copyOf(
Iterables.transform(
getLocksWithIds(taskid).entrySet(), new Function<Map.Entry<Long, TaskLock>, TaskLock>()
{
@Override
public TaskLock apply(Map.Entry<Long, TaskLock> e)
{
return e.getValue();
}
}
)
);
}
@Override
public <T> void addAuditLog(final Task task, final TaskAction<T> taskAction)
{
Preconditions.checkNotNull(taskAction, "taskAction");
log.info("Logging action for task[%s]: %s", task.getId(), taskAction);
retryingHandle(
new HandleCallback<Integer>()
{
@Override
public Integer withHandle(Handle handle) throws Exception
{
return handle.createStatement(
String.format(
"INSERT INTO %s (task_id, log_payload) VALUES (:task_id, :log_payload)",
dbTables.getTaskLogTable()
)
)
.bind("task_id", task.getId())
.bind("log_payload", jsonMapper.writeValueAsBytes(taskAction))
.execute();
}
}
);
}
@Override
public List<TaskAction> getAuditLogs(final String taskid)
{
return retryingHandle(
new HandleCallback<List<TaskAction>>()
{
@Override
public List<TaskAction> withHandle(Handle handle) throws Exception
{
final List<Map<String, Object>> dbTaskLogs =
handle.createQuery(
String.format(
"SELECT log_payload FROM %s WHERE task_id = :task_id",
dbTables.getTaskLogTable()
)
)
.bind("task_id", taskid)
.list();
final List<TaskAction> retList = Lists.newArrayList();
for (final Map<String, Object> dbTaskLog : dbTaskLogs) {
try {
retList.add(jsonMapper.readValue((byte[]) dbTaskLog.get("log_payload"), TaskAction.class));
}
catch (Exception e) {
log.makeAlert(e, "Failed to deserialize TaskLog")
.addData("task", taskid)
.addData("logPayload", dbTaskLog)
.emit();
}
}
return retList;
}
}
);
}
private Map<Long, TaskLock> getLocksWithIds(final String taskid)
{
return retryingHandle(
new HandleCallback<Map<Long, TaskLock>>()
{
@Override
public Map<Long, TaskLock> withHandle(Handle handle) throws Exception
{
final List<Map<String, Object>> dbTaskLocks =
handle.createQuery(
String.format(
"SELECT id, lock_payload FROM %s WHERE task_id = :task_id",
dbTables.getTaskLockTable()
)
)
.bind("task_id", taskid)
.list();
final Map<Long, TaskLock> retMap = Maps.newHashMap();
for (final Map<String, Object> row : dbTaskLocks) {
try {
retMap.put(
(Long) row.get("id"),
jsonMapper.readValue((byte[]) row.get("lock_payload"), TaskLock.class)
);
}
catch (Exception e) {
log.makeAlert(e, "Failed to deserialize TaskLock")
.addData("task", taskid)
.addData("lockPayload", row)
.emit();
}
}
return retMap;
}
}
);
}
/**
* Retry SQL operations
*/
private <T> T retryingHandle(final HandleCallback<T> callback)
{
final Callable<T> call = new Callable<T>()
{
@Override
public T call() throws Exception
{
return dbi.withHandle(callback);
}
};
final Predicate<Throwable> shouldRetry = new Predicate<Throwable>()
{
@Override
public boolean apply(Throwable e)
{
return shouldRetryException(e);
}
};
final int maxTries = 10;
try {
return RetryUtils.retry(call, shouldRetry, maxTries);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
private static boolean shouldRetryException(final Throwable e)
{
return e != null && (e instanceof SQLTransientException
|| e instanceof MySQLTransientException
|| e instanceof SQLRecoverableException
|| e instanceof UnableToObtainConnectionException
|| (e instanceof SQLException && ((SQLException) e).getErrorCode() == 1317)
|| (e instanceof SQLException && shouldRetryException(e.getCause()))
|| (e instanceof DBIException && shouldRetryException(e.getCause())));
}
}

View File

@ -56,8 +56,6 @@ import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.io.RandomAccessFile;
import java.nio.channels.Channels;
import java.util.Collection;
import java.util.List;
import java.util.Map;

View File

@ -35,6 +35,7 @@ import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.actions.TaskAction;
import io.druid.indexing.common.config.TaskStorageConfig;
import io.druid.indexing.common.task.Task;
import io.druid.metadata.EntryExistsException;
import org.joda.time.DateTime;
import java.util.List;
@ -63,7 +64,7 @@ public class HeapMemoryTaskStorage implements TaskStorage
}
@Override
public void insert(Task task, TaskStatus status) throws TaskExistsException
public void insert(Task task, TaskStatus status) throws EntryExistsException
{
giant.lock();
@ -78,7 +79,7 @@ public class HeapMemoryTaskStorage implements TaskStorage
);
if(tasks.containsKey(task.getId())) {
throw new TaskExistsException(task.getId());
throw new EntryExistsException(task.getId());
}
log.info("Inserting task %s with status: %s", task.getId(), status);

View File

@ -0,0 +1,304 @@
/*
* 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.indexing.overlord;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.base.Function;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.inject.Inject;
import com.metamx.common.Pair;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.emitter.EmittingLogger;
import io.druid.indexing.common.TaskStatus;
import io.druid.metadata.EntryExistsException;
import io.druid.metadata.MetadataStorageActionHandler;
import io.druid.metadata.MetadataStorageActionHandlerFactory;
import io.druid.metadata.MetadataStorageActionHandlerTypes;
import io.druid.metadata.MetadataStorageConnector;
import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.actions.TaskAction;
import io.druid.indexing.common.config.TaskStorageConfig;
import io.druid.indexing.common.task.Task;
import io.druid.metadata.MetadataStorageTablesConfig;
import org.joda.time.DateTime;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Map;
public class MetadataTaskStorage implements TaskStorage
{
private static final MetadataStorageActionHandlerTypes<Task, TaskStatus, TaskAction, TaskLock> TASK_TYPES = new MetadataStorageActionHandlerTypes<Task, TaskStatus, TaskAction, TaskLock>()
{
@Override
public TypeReference<Task> getEntryType()
{
return new TypeReference<Task>()
{
};
}
@Override
public TypeReference<TaskStatus> getStatusType()
{
return new TypeReference<TaskStatus>()
{
};
}
@Override
public TypeReference<TaskAction> getLogType()
{
return new TypeReference<TaskAction>()
{
};
}
@Override
public TypeReference<TaskLock> getLockType()
{
return new TypeReference<TaskLock>()
{
};
}
};
private final MetadataStorageConnector metadataStorageConnector;
private final TaskStorageConfig config;
private final MetadataStorageActionHandler<Task, TaskStatus, TaskAction, TaskLock> handler;
private static final EmittingLogger log = new EmittingLogger(MetadataTaskStorage.class);
@Inject
public MetadataTaskStorage(
final MetadataStorageConnector metadataStorageConnector,
final TaskStorageConfig config,
final MetadataStorageActionHandlerFactory factory
)
{
this.metadataStorageConnector = metadataStorageConnector;
this.config = config;
this.handler = factory.create(MetadataStorageTablesConfig.TASK_ENTRY_TYPE, TASK_TYPES);
}
@LifecycleStart
public void start()
{
metadataStorageConnector.createTaskTables();
}
@LifecycleStop
public void stop()
{
// do nothing
}
@Override
public void insert(final Task task, final TaskStatus status) throws EntryExistsException
{
Preconditions.checkNotNull(task, "task");
Preconditions.checkNotNull(status, "status");
Preconditions.checkArgument(
task.getId().equals(status.getId()),
"Task/Status ID mismatch[%s/%s]",
task.getId(),
status.getId()
);
log.info("Inserting task %s with status: %s", task.getId(), status);
try {
handler.insert(
task.getId(),
new DateTime(),
task.getDataSource(),
task,
status.isRunnable(),
status
);
}
catch (Exception e) {
if(e instanceof EntryExistsException) {
throw (EntryExistsException) e;
} else {
Throwables.propagate(e);
}
}
}
@Override
public void setStatus(final TaskStatus status)
{
Preconditions.checkNotNull(status, "status");
log.info("Updating task %s to status: %s", status.getId(), status);
final boolean set = handler.setStatus(
status.getId(),
status.isRunnable(),
status
);
if (!set) {
throw new IllegalStateException(String.format("Active task not found: %s", status.getId()));
}
}
@Override
public Optional<Task> getTask(final String taskId)
{
return handler.getEntry(taskId);
}
@Override
public Optional<TaskStatus> getStatus(final String taskId)
{
return handler.getStatus(taskId);
}
@Override
public List<Task> getActiveTasks()
{
return ImmutableList.copyOf(
Iterables.transform(
Iterables.filter(
handler.getActiveEntriesWithStatus(),
new Predicate<Pair<Task, TaskStatus>>()
{
@Override
public boolean apply(
@Nullable Pair<Task, TaskStatus> input
)
{
return input.rhs.isRunnable();
}
}
),
new Function<Pair<Task, TaskStatus>, Task>()
{
@Nullable
@Override
public Task apply(@Nullable Pair<Task, TaskStatus> input)
{
return input.lhs;
}
}
)
);
}
@Override
public List<TaskStatus> getRecentlyFinishedTaskStatuses()
{
final DateTime start = new DateTime().minus(config.getRecentlyFinishedThreshold());
return ImmutableList.copyOf(
Iterables.filter(
handler.getInactiveStatusesSince(start),
new Predicate<TaskStatus>()
{
@Override
public boolean apply(TaskStatus status)
{
return status.isComplete();
}
}
)
);
}
@Override
public void addLock(final String taskid, final TaskLock taskLock)
{
Preconditions.checkNotNull(taskid, "taskid");
Preconditions.checkNotNull(taskLock, "taskLock");
log.info(
"Adding lock on interval[%s] version[%s] for task: %s",
taskLock.getInterval(),
taskLock.getVersion(),
taskid
);
handler.addLock(taskid, taskLock);
}
@Override
public void removeLock(String taskid, TaskLock taskLockToRemove)
{
Preconditions.checkNotNull(taskid, "taskid");
Preconditions.checkNotNull(taskLockToRemove, "taskLockToRemove");
final Map<Long, TaskLock> taskLocks = getLocksWithIds(taskid);
for (final Map.Entry<Long, TaskLock> taskLockWithId : taskLocks.entrySet()) {
final long id = taskLockWithId.getKey();
final TaskLock taskLock = taskLockWithId.getValue();
if (taskLock.equals(taskLockToRemove)) {
log.info("Deleting TaskLock with id[%d]: %s", id, taskLock);
handler.removeLock(id);
}
}
}
@Override
public List<TaskLock> getLocks(String taskid)
{
return ImmutableList.copyOf(
Iterables.transform(
getLocksWithIds(taskid).entrySet(), new Function<Map.Entry<Long, TaskLock>, TaskLock>()
{
@Override
public TaskLock apply(Map.Entry<Long, TaskLock> e)
{
return e.getValue();
}
}
)
);
}
@Override
public <T> void addAuditLog(final Task task, final TaskAction<T> taskAction)
{
Preconditions.checkNotNull(taskAction, "taskAction");
log.info("Logging action for task[%s]: %s", task.getId(), taskAction);
handler.addLog(task.getId(), taskAction);
}
@Override
public List<TaskAction> getAuditLogs(final String taskId)
{
return handler.getLogs(taskId);
}
private Map<Long, TaskLock> getLocksWithIds(final String taskid)
{
return handler.getLocks(taskid);
}
}

View File

@ -43,6 +43,7 @@ import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.actions.TaskActionClientFactory;
import io.druid.indexing.common.task.Task;
import io.druid.indexing.overlord.config.TaskQueueConfig;
import io.druid.metadata.EntryExistsException;
import java.util.List;
import java.util.Map;
@ -292,9 +293,9 @@ public class TaskQueue
* @param task task to add
*
* @return true
* @throws TaskExistsException if the task already exists
* @throws io.druid.metadata.EntryExistsException if the task already exists
*/
public boolean add(final Task task) throws TaskExistsException
public boolean add(final Task task) throws EntryExistsException
{
giant.lock();

View File

@ -24,6 +24,7 @@ import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.actions.TaskAction;
import io.druid.indexing.common.task.Task;
import io.druid.metadata.EntryExistsException;
import java.util.List;
@ -34,9 +35,9 @@ public interface TaskStorage
*
* @param task task to add
* @param status task status
* @throws io.druid.indexing.overlord.TaskExistsException if the task ID already exists
* @throws io.druid.metadata.EntryExistsException if the task ID already exists
*/
public void insert(Task task, TaskStatus status) throws TaskExistsException;
public void insert(Task task, TaskStatus status) throws EntryExistsException;
/**
* Persists task status in the storage facility. This method should throw an exception if the task status lifecycle

View File

@ -36,7 +36,7 @@ import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.indexing.common.actions.TaskActionHolder;
import io.druid.indexing.common.task.Task;
import io.druid.indexing.overlord.TaskExistsException;
import io.druid.metadata.EntryExistsException;
import io.druid.indexing.overlord.TaskMaster;
import io.druid.indexing.overlord.TaskQueue;
import io.druid.indexing.overlord.TaskRunner;
@ -131,7 +131,7 @@ public class OverlordResource
taskQueue.add(task);
return Response.ok(ImmutableMap.of("task", task.getId())).build();
}
catch (TaskExistsException e) {
catch (EntryExistsException e) {
return Response.status(Response.Status.BAD_REQUEST)
.entity(ImmutableMap.of("error", String.format("Task[%s] already exists!", task.getId())))
.build();

View File

@ -25,14 +25,14 @@
<meta http-equiv="Content-Type" content="text/html; charset=UTF-8"/>
<meta name="Description" content="Druid Indexer Coordinator Console"/>
<style type="text/css">@import "css/style.css";</style>
<style type="text/css">@import "css/demo_table.css";</style>
<style type="text/css">@import "old-console/css/style.css";</style>
<style type="text/css">@import "old-console/css/demo_table.css";</style>
<script type="text/javascript" src="js/underscore-1.2.2.js"></script>
<script type="text/javascript" src="js/jquery-1.11.0.min.js"></script>
<script type="text/javascript" src="js/jquery.dataTables-1.8.2.js"></script>
<script type="text/javascript" src="js/druidTable-0.0.1.js"></script>
<script type="text/javascript" src="js/tablehelper-0.0.2.js"></script>
<script type="text/javascript" src="old-console/js/underscore-1.2.2.js"></script>
<script type="text/javascript" src="old-console/js/jquery-1.11.0.min.js"></script>
<script type="text/javascript" src="old-console/js/jquery.dataTables-1.8.2.js"></script>
<script type="text/javascript" src="old-console/js/druidTable-0.0.1.js"></script>
<script type="text/javascript" src="old-console/js/tablehelper-0.0.2.js"></script>
<script type="text/javascript" src="js/console-0.0.1.js"></script>
</head>
@ -64,4 +64,4 @@
<table id="eventTable"></table>
</div>
</body>
</html>
</html>

View File

@ -29,7 +29,6 @@ import com.google.common.collect.Sets;
import com.google.common.util.concurrent.ListenableFuture;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.ServiceEmitter;
import io.druid.common.guava.DSuppliers;
import io.druid.curator.PotentiallyGzippedCompressionProvider;
import io.druid.curator.cache.SimplePathChildrenCacheFactory;
import io.druid.indexing.common.IndexingServiceCondition;
@ -41,7 +40,6 @@ import io.druid.indexing.common.task.Task;
import io.druid.indexing.common.task.TaskResource;
import io.druid.indexing.overlord.config.RemoteTaskRunnerConfig;
import io.druid.indexing.overlord.setup.FillCapacityWorkerSelectStrategy;
import io.druid.indexing.overlord.setup.WorkerSetupData;
import io.druid.indexing.worker.TaskAnnouncement;
import io.druid.indexing.worker.Worker;
import io.druid.jackson.DefaultObjectMapper;
@ -59,7 +57,6 @@ import org.junit.Test;
import java.util.Set;
import java.util.concurrent.Future;
import java.util.concurrent.atomic.AtomicReference;
public class RemoteTaskRunnerTest
{

View File

@ -42,6 +42,7 @@ import io.druid.data.input.FirehoseFactory;
import io.druid.data.input.InputRow;
import io.druid.data.input.MapBasedInputRow;
import io.druid.data.input.impl.InputRowParser;
import io.druid.metadata.IndexerSQLMetadataStorageCoordinator;
import io.druid.granularity.QueryGranularity;
import io.druid.indexing.common.SegmentLoaderFactory;
import io.druid.indexing.common.TaskLock;
@ -63,6 +64,7 @@ import io.druid.indexing.common.task.Task;
import io.druid.indexing.common.task.TaskResource;
import io.druid.indexing.overlord.config.TaskQueueConfig;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.metadata.SQLMetadataConnector;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
@ -85,6 +87,8 @@ import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.skife.jdbi.v2.DBI;
import org.skife.jdbi.v2.Handle;
import java.io.File;
import java.io.IOException;
@ -100,7 +104,7 @@ public class TaskLifecycleTest
private TaskLockbox tl = null;
private TaskQueue tq = null;
private TaskRunner tr = null;
private MockIndexerDBCoordinator mdc = null;
private MockIndexerMetadataStorageCoordinator mdc = null;
private TaskActionClientFactory tac = null;
private TaskToolboxFactory tb = null;
TaskStorageQueryAdapter tsqa = null;
@ -509,12 +513,12 @@ public class TaskLifecycleTest
return retVal;
}
private static class MockIndexerDBCoordinator extends IndexerDBCoordinator
private static class MockIndexerMetadataStorageCoordinator extends IndexerSQLMetadataStorageCoordinator
{
final private Set<DataSegment> published = Sets.newHashSet();
final private Set<DataSegment> nuked = Sets.newHashSet();
private MockIndexerDBCoordinator()
private MockIndexerMetadataStorageCoordinator()
{
super(null, null, null);
}
@ -561,9 +565,9 @@ public class TaskLifecycleTest
}
}
private static MockIndexerDBCoordinator newMockMDC()
private static MockIndexerMetadataStorageCoordinator newMockMDC()
{
return new MockIndexerDBCoordinator();
return new MockIndexerMetadataStorageCoordinator();
}
private static ServiceEmitter newMockEmitter()

View File

@ -0,0 +1,78 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
~ 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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<groupId>io.druid.extensions</groupId>
<artifactId>mysql-metadata-storage</artifactId>
<name>mysql-metadata-storage</name>
<description>mysql-metadata-storage</description>
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.7.0-SNAPSHOT</version>
</parent>
<dependencies>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-api</artifactId>
</dependency>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-common</artifactId>
<version>${project.parent.version}</version>
</dependency>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-server</artifactId>
<version>${project.parent.version}</version>
</dependency>
<dependency>
<groupId>mysql</groupId>
<artifactId>mysql-connector-java</artifactId>
<version>5.1.33</version>
</dependency>
<dependency>
<groupId>org.jdbi</groupId>
<artifactId>jdbi</artifactId>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<artifactId>maven-jar-plugin</artifactId>
<configuration>
<archive>
<manifest>
<addDefaultImplementationEntries>true</addDefaultImplementationEntries>
<addDefaultSpecificationEntries>true</addDefaultSpecificationEntries>
</manifest>
</archive>
</configuration>
</plugin>
</plugins>
</build>
</project>

View File

@ -0,0 +1,125 @@
/*
* 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.mysql;
import com.google.common.base.Supplier;
import com.google.common.collect.ImmutableList;
import com.google.inject.Inject;
import com.metamx.common.logger.Logger;
import com.mysql.jdbc.exceptions.MySQLTransientException;
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.HandleCallback;
import java.sql.SQLException;
public class MySQLConnector extends SQLMetadataConnector
{
private static final Logger log = new Logger(MySQLConnector.class);
private static final String PAYLOAD_TYPE = "LONGBLOB";
private static final String SERIAL_TYPE = "BIGINT(20) AUTO_INCREMENT";
private final DBI dbi;
@Inject
public MySQLConnector(Supplier<MetadataStorageConnectorConfig> config, Supplier<MetadataStorageTablesConfig> dbTables)
{
super(config, dbTables);
final BasicDataSource datasource = getDatasource();
// MySQL driver is classloader isolated as part of the extension
// so we need to help JDBC find the driver
datasource.setDriverClassLoader(getClass().getClassLoader());
datasource.setDriverClassName("com.mysql.jdbc.Driver");
// use double-quotes for quoting columns, so we can write SQL that works with most databases
datasource.setConnectionInitSqls(ImmutableList.of("SET sql_mode='ANSI_QUOTES'"));
this.dbi = new DBI(datasource);
}
@Override
protected String getPayloadType()
{
return PAYLOAD_TYPE;
}
@Override
protected String getSerialType()
{
return SERIAL_TYPE;
}
@Override
protected boolean tableExists(Handle handle, String tableName)
{
return !handle.createQuery("SHOW tables LIKE :tableName")
.bind("tableName", tableName)
.list()
.isEmpty();
}
@Override
protected boolean isTransientException(Throwable e)
{
return e instanceof MySQLTransientException
|| (e instanceof SQLException && ((SQLException) e).getErrorCode() == 1317)
;
}
@Override
public Void insertOrUpdate(
final String tableName,
final String keyColumn,
final String valueColumn,
final String key,
final byte[] value
) throws Exception
{
return getDBI().withHandle(
new HandleCallback<Void>()
{
@Override
public Void withHandle(Handle handle) throws Exception
{
handle.createStatement(
String.format(
"INSERT INTO %1$s (%2$s, %3$s) VALUES (:key, :value) ON DUPLICATE KEY UPDATE %3$s = :value",
tableName,
keyColumn,
valueColumn
)
)
.bind("key", key)
.bind("value", value)
.execute();
return null;
}
}
);
}
@Override
public DBI getDBI() { return dbi; }
}

View File

@ -0,0 +1,65 @@
/*
* 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.mysql;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import com.google.inject.Key;
import io.druid.guice.LazySingleton;
import io.druid.guice.PolyBind;
import io.druid.guice.SQLMetadataStorageDruidModule;
import io.druid.initialization.DruidModule;
import io.druid.metadata.MetadataStorageConnector;
import io.druid.metadata.SQLMetadataConnector;
import java.util.List;
public class MySQLMetadataStorageModule extends SQLMetadataStorageDruidModule implements DruidModule
{
public static final String TYPE = "mysql";
public MySQLMetadataStorageModule()
{
super(TYPE);
}
@Override
public List<? extends com.fasterxml.jackson.databind.Module> getJacksonModules()
{
return ImmutableList.of();
}
@Override
public void configure(Binder binder)
{
super.configure(binder);
PolyBind.optionBinder(binder, Key.get(MetadataStorageConnector.class))
.addBinding(TYPE)
.to(MySQLConnector.class)
.in(LazySingleton.class);
PolyBind.optionBinder(binder, Key.get(SQLMetadataConnector.class))
.addBinding(TYPE)
.to(MySQLConnector.class)
.in(LazySingleton.class);
}
}

View File

@ -0,0 +1 @@
io.druid.metadata.storage.mysql.MySQLMetadataStorageModule

View File

@ -42,7 +42,7 @@
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<metamx.java-util.version>0.26.9</metamx.java-util.version>
<apache.curator.version>2.6.0</apache.curator.version>
<druid.api.version>0.2.16</druid.api.version>
<druid.api.version>0.2.17</druid.api.version>
</properties>
<modules>
@ -61,6 +61,8 @@
<module>kafka-eight</module>
<module>rabbitmq</module>
<module>histogram</module>
<module>mysql-metadata-storage</module>
<module>postgresql-metadata-storage</module>
</modules>
<dependencyManagement>
@ -387,6 +389,11 @@
<artifactId>commons-cli</artifactId>
<version>1.2</version>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-dbcp2</artifactId>
<version>2.0.1</version>
</dependency>
<dependency>
<groupId>net.jpountz.lz4</groupId>
<artifactId>lz4</artifactId>

View File

@ -0,0 +1,78 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
~ 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.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<groupId>io.druid.extensions</groupId>
<artifactId>postgresql-metadata-storage</artifactId>
<name>postgresql-metadata-storage</name>
<description>postgresql-metadata-storage</description>
<parent>
<groupId>io.druid</groupId>
<artifactId>druid</artifactId>
<version>0.7.0-SNAPSHOT</version>
</parent>
<dependencies>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-api</artifactId>
</dependency>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-common</artifactId>
<version>${project.parent.version}</version>
</dependency>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-server</artifactId>
<version>${project.parent.version}</version>
</dependency>
<dependency>
<groupId>org.postgresql</groupId>
<artifactId>postgresql</artifactId>
<version>9.3-1102-jdbc41</version>
</dependency>
<dependency>
<groupId>org.jdbi</groupId>
<artifactId>jdbi</artifactId>
</dependency>
</dependencies>
<build>
<plugins>
<plugin>
<artifactId>maven-jar-plugin</artifactId>
<configuration>
<archive>
<manifest>
<addDefaultImplementationEntries>true</addDefaultImplementationEntries>
<addDefaultSpecificationEntries>true</addDefaultSpecificationEntries>
</manifest>
</archive>
</configuration>
</plugin>
</plugins>
</build>
</project>

View File

@ -0,0 +1,117 @@
/*
* 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.postgresql;
import com.google.common.base.Supplier;
import com.google.inject.Inject;
import com.metamx.common.logger.Logger;
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.HandleCallback;
import org.skife.jdbi.v2.util.StringMapper;
public class PostgreSQLConnector extends SQLMetadataConnector
{
private static final Logger log = new Logger(PostgreSQLConnector.class);
private static final String PAYLOAD_TYPE = "BYTEA";
private static final String SERIAL_TYPE = "BIGSERIAL";
private final DBI dbi;
@Inject
public PostgreSQLConnector(Supplier<MetadataStorageConnectorConfig> config, Supplier<MetadataStorageTablesConfig> dbTables)
{
super(config, dbTables);
final BasicDataSource datasource = getDatasource();
// PostgreSQL driver is classloader isolated as part of the extension
// so we need to help JDBC find the driver
datasource.setDriverClassLoader(getClass().getClassLoader());
datasource.setDriverClassName("org.postgresql.Driver");
this.dbi = new DBI(datasource);
}
@Override
protected String getPayloadType() {
return PAYLOAD_TYPE;
}
@Override
protected String getSerialType()
{
return SERIAL_TYPE;
}
@Override
protected 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"
)
.bind("tableName", tableName)
.map(StringMapper.FIRST)
.list()
.isEmpty();
}
@Override
public Void insertOrUpdate(
final String tableName,
final String keyColumn,
final String valueColumn,
final String key,
final byte[] value
) throws Exception
{
return getDBI().withHandle(
new HandleCallback<Void>()
{
@Override
public Void withHandle(Handle handle) throws Exception
{
handle.createStatement(
String.format(
"BEGIN;\n" +
"LOCK TABLE %1$s IN SHARE ROW EXCLUSIVE MODE;\n" +
"WITH upsert AS (UPDATE %1$s SET %3$s=:value WHERE %2$s=:key RETURNING *)\n" +
" INSERT INTO %1$s (%2$s, %3$s) SELECT :key, :value WHERE NOT EXISTS (SELECT * FROM upsert)\n;" +
"COMMIT;",
tableName,
keyColumn,
valueColumn
)
)
.bind("key", key)
.bind("value", value)
.execute();
return null;
}
}
);
}
@Override
public DBI getDBI() { return dbi; }
}

View File

@ -0,0 +1,65 @@
/*
* 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.postgresql;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import com.google.inject.Key;
import io.druid.guice.LazySingleton;
import io.druid.guice.PolyBind;
import io.druid.guice.SQLMetadataStorageDruidModule;
import io.druid.initialization.DruidModule;
import io.druid.metadata.MetadataStorageConnector;
import io.druid.metadata.SQLMetadataConnector;
import java.util.List;
public class PostgreSQLMetadataStorageModule extends SQLMetadataStorageDruidModule implements DruidModule
{
public static final String TYPE = "postgresql";
public PostgreSQLMetadataStorageModule()
{
super(TYPE);
}
@Override
public List<? extends com.fasterxml.jackson.databind.Module> getJacksonModules()
{
return ImmutableList.of();
}
@Override
public void configure(Binder binder)
{
super.configure(binder);
PolyBind.optionBinder(binder, Key.get(MetadataStorageConnector.class))
.addBinding(TYPE)
.to(PostgreSQLConnector.class)
.in(LazySingleton.class);
PolyBind.optionBinder(binder, Key.get(SQLMetadataConnector.class))
.addBinding(TYPE)
.to(PostgreSQLConnector.class)
.in(LazySingleton.class);
}
}

View File

@ -0,0 +1 @@
io.druid.metadata.storage.postgresql.PostgreSQLMetadataStorageModule

View File

@ -150,8 +150,28 @@
<groupId>com.maxmind.geoip2</groupId>
<artifactId>geoip2</artifactId>
</dependency>
<dependency>
<groupId>org.apache.derby</groupId>
<artifactId>derby</artifactId>
<version>10.11.1.1</version>
</dependency>
<dependency>
<groupId>org.apache.derby</groupId>
<artifactId>derbynet</artifactId>
<version>10.11.1.1</version>
</dependency>
<dependency>
<groupId>org.apache.derby</groupId>
<artifactId>derbyclient</artifactId>
<version>10.11.1.1</version>
</dependency>
<dependency>
<groupId>org.antlr</groupId>
<artifactId>antlr4-maven-plugin</artifactId>
<version>4.0</version>
</dependency>
<!-- Tests -->
<!-- Tests -->
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>

View File

@ -0,0 +1,55 @@
/*
* 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.guice;
import com.google.inject.Binder;
import com.google.inject.Key;
import com.google.inject.Provides;
import io.druid.metadata.DerbyConnector;
import io.druid.metadata.MetadataStorageConnector;
import io.druid.metadata.SQLMetadataConnector;
import org.skife.jdbi.v2.IDBI;
public class DerbyMetadataStorageDruidModule extends SQLMetadataStorageDruidModule
{
public DerbyMetadataStorageDruidModule()
{
super(TYPE);
}
public static final String TYPE = "derby";
@Override
public void configure(Binder binder)
{
createBindingChoices(binder, TYPE);
super.configure(binder);
PolyBind.optionBinder(binder, Key.get(MetadataStorageConnector.class))
.addBinding(TYPE)
.to(DerbyConnector.class)
.in(LazySingleton.class);
PolyBind.optionBinder(binder, Key.get(SQLMetadataConnector.class))
.addBinding(TYPE)
.to(DerbyConnector.class)
.in(LazySingleton.class);
}
}

View File

@ -21,28 +21,20 @@ package io.druid.guice;
import com.google.inject.Binder;
import com.google.inject.Module;
import com.google.inject.Provides;
import io.druid.db.DbConnector;
import io.druid.db.DbConnectorConfig;
import io.druid.db.DbTablesConfig;
import org.skife.jdbi.v2.IDBI;
import io.druid.metadata.MetadataStorageConnectorConfig;
import io.druid.metadata.MetadataRuleManagerConfig;
import io.druid.metadata.MetadataSegmentManagerConfig;
import io.druid.metadata.MetadataStorageTablesConfig;
/**
*/
public class DbConnectorModule implements Module
public class MetadataDbConfigModule implements Module
{
@Override
public void configure(Binder binder)
{
JsonConfigProvider.bind(binder, "druid.db.tables", DbTablesConfig.class);
JsonConfigProvider.bind(binder, "druid.db.connector", DbConnectorConfig.class);
JsonConfigProvider.bind(binder, "druid.metadata.storage.tables", MetadataStorageTablesConfig.class);
JsonConfigProvider.bind(binder, "druid.metadata.storage.connector", MetadataStorageConnectorConfig.class);
binder.bind(DbConnector.class);
}
@Provides @LazySingleton
public IDBI getDbi(final DbConnector dbConnector)
{
return dbConnector.getDBI();
JsonConfigProvider.bind(binder, "druid.manager.segments", MetadataSegmentManagerConfig.class);
JsonConfigProvider.bind(binder, "druid.manager.rules", MetadataRuleManagerConfig.class);
}
}

View File

@ -19,6 +19,7 @@
package io.druid.guice;
import io.druid.metadata.SegmentPublisherProvider;
import io.druid.segment.realtime.NoopSegmentPublisher;
import io.druid.segment.realtime.SegmentPublisher;

View File

@ -0,0 +1,181 @@
/*
* 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.guice;
import com.google.inject.Binder;
import com.google.inject.Key;
import com.google.inject.Module;
import io.druid.indexer.MetadataStorageUpdaterJobHandler;
import io.druid.indexer.SQLMetadataStorageUpdaterJobHandler;
import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
import io.druid.metadata.MetadataStorageActionHandlerFactory;
import io.druid.metadata.IndexerSQLMetadataStorageCoordinator;
import io.druid.metadata.MetadataRuleManager;
import io.druid.metadata.MetadataRuleManagerProvider;
import io.druid.metadata.MetadataSegmentManager;
import io.druid.metadata.MetadataSegmentManagerProvider;
import io.druid.metadata.MetadataSegmentPublisher;
import io.druid.metadata.MetadataSegmentPublisherProvider;
import io.druid.metadata.MetadataStorageConnector;
import io.druid.metadata.SQLMetadataConnector;
import io.druid.metadata.SQLMetadataRuleManager;
import io.druid.metadata.SQLMetadataRuleManagerProvider;
import io.druid.metadata.SQLMetadataSegmentManager;
import io.druid.metadata.SQLMetadataSegmentManagerProvider;
import io.druid.metadata.SQLMetadataSegmentPublisher;
import io.druid.metadata.SQLMetadataSegmentPublisherProvider;
import io.druid.metadata.SQLMetadataStorageActionHandlerFactory;
public class SQLMetadataStorageDruidModule implements Module
{
public static final String PROPERTY = "druid.metadata.storage.type";
final String type;
public SQLMetadataStorageDruidModule(String type)
{
this.type = type;
}
/**
* This function only needs to be called by the default SQL metadata storage module
* Other modules should default to calling super.configure(...) alone
*/
public void createBindingChoices(Binder binder, String defaultPropertyValue)
{
PolyBind.createChoiceWithDefault(
binder, PROPERTY, Key.get(MetadataStorageConnector.class), null, defaultPropertyValue
);
PolyBind.createChoiceWithDefault(
binder, PROPERTY, Key.get(SQLMetadataConnector.class), null, defaultPropertyValue
);
PolyBind.createChoiceWithDefault(
binder,
PROPERTY,
Key.get(MetadataSegmentManager.class),
Key.get(SQLMetadataSegmentManager.class),
defaultPropertyValue
);
PolyBind.createChoiceWithDefault(
binder,
PROPERTY,
Key.get(MetadataSegmentManagerProvider.class),
Key.get(SQLMetadataSegmentManagerProvider.class),
defaultPropertyValue
);
PolyBind.createChoiceWithDefault(
binder,
PROPERTY,
Key.get(MetadataRuleManager.class),
Key.get(SQLMetadataRuleManager.class),
defaultPropertyValue
);
PolyBind.createChoiceWithDefault(
binder,
PROPERTY,
Key.get(MetadataRuleManagerProvider.class),
Key.get(SQLMetadataRuleManagerProvider.class),
defaultPropertyValue
);
PolyBind.createChoiceWithDefault(
binder,
PROPERTY,
Key.get(MetadataSegmentPublisher.class),
Key.get(SQLMetadataSegmentPublisher.class),
defaultPropertyValue
);
PolyBind.createChoiceWithDefault(
binder,
PROPERTY,
Key.get(MetadataSegmentPublisherProvider.class),
Key.get(SQLMetadataSegmentPublisherProvider.class),
defaultPropertyValue
);
PolyBind.createChoiceWithDefault(
binder,
PROPERTY,
Key.get(IndexerMetadataStorageCoordinator.class),
Key.get(IndexerSQLMetadataStorageCoordinator.class),
defaultPropertyValue
);
PolyBind.createChoiceWithDefault(
binder,
PROPERTY,
Key.get(MetadataStorageActionHandlerFactory.class),
Key.get(SQLMetadataStorageActionHandlerFactory.class),
defaultPropertyValue
);
PolyBind.createChoiceWithDefault(
binder,
PROPERTY,
Key.get(MetadataStorageUpdaterJobHandler.class),
Key.get(SQLMetadataStorageUpdaterJobHandler.class),
defaultPropertyValue
);
}
@Override
public void configure(Binder binder)
{
PolyBind.optionBinder(binder, Key.get(MetadataSegmentManager.class))
.addBinding(type)
.to(SQLMetadataSegmentManager.class)
.in(LazySingleton.class);
PolyBind.optionBinder(binder, Key.get(MetadataSegmentManagerProvider.class))
.addBinding(type)
.to(SQLMetadataSegmentManagerProvider.class)
.in(LazySingleton.class);
PolyBind.optionBinder(binder, Key.get(MetadataRuleManager.class))
.addBinding(type)
.to(SQLMetadataRuleManager.class)
.in(LazySingleton.class);
PolyBind.optionBinder(binder, Key.get(MetadataRuleManagerProvider.class))
.addBinding(type)
.to(SQLMetadataRuleManagerProvider.class)
.in(LazySingleton.class);
PolyBind.optionBinder(binder, Key.get(MetadataSegmentPublisher.class))
.addBinding(type)
.to(SQLMetadataSegmentPublisher.class)
.in(LazySingleton.class);
PolyBind.optionBinder(binder, Key.get(MetadataSegmentPublisherProvider.class))
.addBinding(type)
.to(SQLMetadataSegmentPublisherProvider.class)
.in(LazySingleton.class);
PolyBind.optionBinder(binder, Key.get(MetadataStorageActionHandlerFactory.class))
.addBinding(type)
.to(SQLMetadataStorageActionHandlerFactory.class)
.in(LazySingleton.class);
PolyBind.optionBinder(binder, Key.get(IndexerMetadataStorageCoordinator.class))
.addBinding(type)
.to(IndexerSQLMetadataStorageCoordinator.class)
.in(LazySingleton.class);
PolyBind.optionBinder(binder, Key.get(MetadataStorageUpdaterJobHandler.class))
.addBinding(type)
.to(SQLMetadataStorageUpdaterJobHandler.class)
.in(LazySingleton.class);
}
}

View File

@ -19,9 +19,10 @@
package io.druid.indexer;
import com.google.common.collect.ImmutableMap;
import com.metamx.common.logger.Logger;
import io.druid.db.DbConnector;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Inject;
import io.druid.metadata.MetadataStorageConnector;
import io.druid.metadata.SQLMetadataConnector;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
import org.joda.time.DateTime;
@ -29,33 +30,24 @@ import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.IDBI;
import org.skife.jdbi.v2.PreparedBatch;
import org.skife.jdbi.v2.tweak.HandleCallback;
import com.metamx.common.logger.Logger;
import com.google.common.collect.ImmutableMap;
import java.util.List;
/**
*/
public class DbUpdaterJob implements Jobby
public class SQLMetadataStorageUpdaterJobHandler implements MetadataStorageUpdaterJobHandler
{
private static final Logger log = new Logger(DbUpdaterJob.class);
private final HadoopDruidIndexerConfig config;
private static final Logger log = new Logger(SQLMetadataStorageUpdaterJobHandler.class);
private final IDBI dbi;
private final DbConnector dbConnector;
public DbUpdaterJob(
HadoopDruidIndexerConfig config
)
@Inject
public SQLMetadataStorageUpdaterJobHandler(SQLMetadataConnector connector)
{
this.config = config;
this.dbConnector = new DbConnector(config.getSchema().getIOConfig().getMetadataUpdateSpec(), null);
this.dbi = this.dbConnector.getDBI();
this.dbi = connector.getDBI();
}
@Override
public boolean run()
public void publishSegments(final String tableName, final List<DataSegment> segments, final ObjectMapper mapper)
{
final List<DataSegment> segments = IndexGeneratorJob.getPublishedSegments(config);
dbi.withHandle(
new HandleCallback<Void>()
{
@ -64,12 +56,9 @@ public class DbUpdaterJob implements Jobby
{
final PreparedBatch batch = handle.prepareBatch(
String.format(
dbConnector.isPostgreSQL() ?
"INSERT INTO %s (id, dataSource, created_date, start, \"end\", partitioned, version, used, payload) "
+ "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)" :
"INSERT INTO %s (id, dataSource, created_date, start, end, partitioned, version, used, payload) "
+ "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)",
config.getSchema().getIOConfig().getMetadataUpdateSpec().getSegmentTable()
"INSERT INTO %s (id, dataSource, created_date, start, \"end\", partitioned, version, used, payload) "
+ "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)",
tableName
)
);
for (final DataSegment segment : segments) {
@ -84,7 +73,7 @@ public class DbUpdaterJob implements Jobby
.put("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? 0 : 1)
.put("version", segment.getVersion())
.put("used", true)
.put("payload", HadoopDruidIndexerConfig.jsonMapper.writeValueAsString(segment))
.put("payload", mapper.writeValueAsString(segment))
.build()
);
@ -97,7 +86,5 @@ public class DbUpdaterJob implements Jobby
}
}
);
return true;
}
}

View File

@ -0,0 +1,51 @@
/*
* 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.indexing.overlord;
import io.druid.timeline.DataSegment;
import org.joda.time.Interval;
import java.io.IOException;
import java.util.List;
import java.util.Set;
/**
*/
public interface IndexerMetadataStorageCoordinator
{
public List<DataSegment> getUsedSegmentsForInterval(final String dataSource, final Interval interval)
throws IOException;
/**
* Attempts to insert a set of segments to the metadata storage. Returns the set of segments actually added (segments
* with identifiers already in the metadata storage will not be added).
*
* @param segments set of segments to add
* @return set of segments actually added
*/
public Set<DataSegment> announceHistoricalSegments(final Set<DataSegment> segments) throws IOException;
public void updateSegmentMetadata(final Set<DataSegment> segments) throws IOException;
public void deleteSegments(final Set<DataSegment> segments) throws IOException;
public List<DataSegment> getUnusedSegmentsForInterval(final String dataSource, final Interval interval);
}

View File

@ -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.DbConnectorModule;
import io.druid.guice.DerbyMetadataStorageDruidModule;
import io.druid.guice.DruidProcessingModule;
import io.druid.guice.DruidSecondaryModule;
import io.druid.guice.ExtensionsConfig;
@ -45,6 +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.ParsersModule;
import io.druid.guice.QueryRunnerFactoryModule;
import io.druid.guice.QueryableModule;
@ -103,7 +104,7 @@ public class Initialization
/**
* @param clazz Module class
* @param <T> Module type
* @param <T>
*
* @return Returns the set of modules loaded.
*/
@ -332,7 +333,8 @@ public class Initialization
new QueryRunnerFactoryModule(),
new DiscoveryModule(),
new ServerViewModule(),
new DbConnectorModule(),
new MetadataDbConfigModule(),
new DerbyMetadataStorageDruidModule(),
new JacksonConfigManagerModule(),
new IndexingServiceDiscoveryModule(),
new LocalDataStorageDruidModule(),

View File

@ -0,0 +1,47 @@
/*
* 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;
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
{
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);
}
}

View File

@ -0,0 +1,77 @@
/*
* 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;
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 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
{
private static final String SERIAL_TYPE = "BIGINT GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1)";
private final DBI dbi;
@Inject
public DerbyConnector(Supplier<MetadataStorageConnectorConfig> config, Supplier<MetadataStorageTablesConfig> dbTables)
{
this(config, dbTables, new DBI(getConnectionFactory("druidDerbyDb")));
}
public DerbyConnector(Supplier<MetadataStorageConnectorConfig> config, Supplier<MetadataStorageTablesConfig> dbTables, DBI dbi)
{
super(config, dbTables);
this.dbi = dbi;
}
@Override
protected boolean tableExists(Handle handle, String tableName)
{
return !handle.createQuery("select * from SYS.SYSTABLES where tablename = :tableName")
.bind("tableName", tableName.toUpperCase())
.list()
.isEmpty();
}
@Override
protected String getSerialType()
{
return SERIAL_TYPE;
}
@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);
}
}

View File

@ -17,8 +17,7 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.indexing.overlord;
package io.druid.metadata;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
@ -28,8 +27,7 @@ import com.google.common.collect.Ordering;
import com.google.common.collect.Sets;
import com.google.inject.Inject;
import com.metamx.common.logger.Logger;
import io.druid.db.DbConnector;
import io.druid.db.DbTablesConfig;
import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
import io.druid.timeline.DataSegment;
import io.druid.timeline.TimelineObjectHolder;
import io.druid.timeline.VersionedIntervalTimeline;
@ -39,6 +37,7 @@ import org.joda.time.Interval;
import org.skife.jdbi.v2.FoldController;
import org.skife.jdbi.v2.Folder3;
import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.IDBI;
import org.skife.jdbi.v2.ResultIterator;
import org.skife.jdbi.v2.StatementContext;
import org.skife.jdbi.v2.TransactionCallback;
@ -53,30 +52,30 @@ import java.util.Set;
/**
*/
public class IndexerDBCoordinator
public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStorageCoordinator
{
private static final Logger log = new Logger(IndexerDBCoordinator.class);
private static final Logger log = new Logger(IndexerSQLMetadataStorageCoordinator.class);
private final ObjectMapper jsonMapper;
private final DbTablesConfig dbTables;
private final DbConnector dbConnector;
private final MetadataStorageTablesConfig dbTables;
private final SQLMetadataConnector connector;
@Inject
public IndexerDBCoordinator(
public IndexerSQLMetadataStorageCoordinator(
ObjectMapper jsonMapper,
DbTablesConfig dbTables,
DbConnector dbConnector
MetadataStorageTablesConfig dbTables,
SQLMetadataConnector connector
)
{
this.jsonMapper = jsonMapper;
this.dbTables = dbTables;
this.dbConnector = dbConnector;
this.connector = connector;
}
public List<DataSegment> getUsedSegmentsForInterval(final String dataSource, final Interval interval)
throws IOException
{
final VersionedIntervalTimeline<String, DataSegment> timeline = dbConnector.getDBI().withHandle(
final VersionedIntervalTimeline<String, DataSegment> timeline = connector.getDBI().withHandle(
new HandleCallback<VersionedIntervalTimeline<String, DataSegment>>()
{
@Override
@ -138,12 +137,10 @@ public class IndexerDBCoordinator
*
* @param segments set of segments to add
* @return set of segments actually added
*
* @throws java.io.IOException if a database error occurs
*/
public Set<DataSegment> announceHistoricalSegments(final Set<DataSegment> segments) throws IOException
{
return dbConnector.getDBI().inTransaction(
return connector.getDBI().inTransaction(
new TransactionCallback<Set<DataSegment>>()
{
@Override
@ -181,11 +178,8 @@ public class IndexerDBCoordinator
try {
handle.createStatement(
String.format(
dbConnector.isPostgreSQL() ?
"INSERT INTO %s (id, dataSource, created_date, start, \"end\", partitioned, version, used, payload) "
+ "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)":
"INSERT INTO %s (id, dataSource, created_date, start, end, partitioned, version, used, payload) "
+ "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)",
"INSERT INTO %s (id, dataSource, created_date, start, \"end\", partitioned, version, used, payload) "
+ "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)",
dbTables.getSegmentsTable()
)
)
@ -197,7 +191,7 @@ public class IndexerDBCoordinator
.bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? 0 : 1)
.bind("version", segment.getVersion())
.bind("used", true)
.bind("payload", jsonMapper.writeValueAsString(segment))
.bind("payload", jsonMapper.writeValueAsBytes(segment))
.execute();
log.info("Published segment [%s] to DB", segment.getIdentifier());
@ -233,7 +227,7 @@ public class IndexerDBCoordinator
public void updateSegmentMetadata(final Set<DataSegment> segments) throws IOException
{
dbConnector.getDBI().inTransaction(
connector.getDBI().inTransaction(
new TransactionCallback<Void>()
{
@Override
@ -251,7 +245,7 @@ public class IndexerDBCoordinator
public void deleteSegments(final Set<DataSegment> segments) throws IOException
{
dbConnector.getDBI().inTransaction(
connector.getDBI().inTransaction(
new TransactionCallback<Void>()
{
@Override
@ -283,7 +277,7 @@ public class IndexerDBCoordinator
String.format("UPDATE %s SET payload = :payload WHERE id = :id", dbTables.getSegmentsTable())
)
.bind("id", segment.getIdentifier())
.bind("payload", jsonMapper.writeValueAsString(segment))
.bind("payload", jsonMapper.writeValueAsBytes(segment))
.execute();
}
catch (IOException e) {
@ -294,7 +288,7 @@ public class IndexerDBCoordinator
public List<DataSegment> getUnusedSegmentsForInterval(final String dataSource, final Interval interval)
{
List<DataSegment> matchingSegments = dbConnector.getDBI().withHandle(
List<DataSegment> matchingSegments = connector.getDBI().withHandle(
new HandleCallback<List<DataSegment>>()
{
@Override
@ -302,9 +296,7 @@ public class IndexerDBCoordinator
{
return handle.createQuery(
String.format(
dbConnector.isPostgreSQL() ?
"SELECT payload FROM %s WHERE dataSource = :dataSource and start >= :start and \"end\" <= :end and used = false":
"SELECT payload FROM %s WHERE dataSource = :dataSource and start >= :start and end <= :end and used = false",
"SELECT payload FROM %s WHERE dataSource = :dataSource and start >= :start and \"end\" <= :end and used = false",
dbTables.getSegmentsTable()
)
)

View File

@ -0,0 +1,44 @@
/*
* 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;
import io.druid.server.coordinator.rules.Rule;
import java.util.List;
import java.util.Map;
/**
*/
public interface MetadataRuleManager
{
public void start();
public void stop();
public void poll();
public Map<String, List<Rule>> getAllRules();
public List<Rule> getRules(final String dataSource);
public List<Rule> getRulesWithDefault(final String dataSource);
public boolean overrideRule(final String dataSource, final List<Rule> newRules);
}

View File

@ -17,14 +17,14 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.db;
package io.druid.metadata;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.joda.time.Period;
/**
*/
public class DatabaseRuleManagerConfig
public class MetadataRuleManagerConfig
{
@JsonProperty
private String defaultRule = "_default";

View File

@ -0,0 +1,29 @@
/*
* 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;
import com.google.inject.Provider;
/**
*/
public interface MetadataRuleManagerProvider extends Provider<MetadataRuleManager>
{
public MetadataRuleManager get();
}

View File

@ -0,0 +1,52 @@
/*
* 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;
import io.druid.client.DruidDataSource;
import java.util.Collection;
/**
*/
public interface MetadataSegmentManager
{
public void start();
public void stop();
public boolean enableDatasource(final String ds);
public boolean enableSegment(final String segmentId);
public boolean removeDatasource(final String ds);
public boolean removeSegment(String ds, final String segmentID);
public boolean isStarted();
public DruidDataSource getInventoryValue(String key);
public Collection<DruidDataSource> getInventory();
public Collection<String> getAllDatasourceNames();
public void poll();
}

View File

@ -17,14 +17,14 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.db;
package io.druid.metadata;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.joda.time.Period;
/**
*/
public class DatabaseSegmentManagerConfig
public class MetadataSegmentManagerConfig
{
@JsonProperty
private Period pollDuration = new Period("PT1M");

View File

@ -0,0 +1,29 @@
/*
* 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;
import com.google.inject.Provider;
/**
*/
public interface MetadataSegmentManagerProvider extends Provider<MetadataSegmentManager>
{
public MetadataSegmentManager get();
}

View File

@ -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 io.druid.segment.realtime.SegmentPublisher;
public interface MetadataSegmentPublisher extends SegmentPublisher
{
}

View File

@ -0,0 +1,25 @@
/*
* 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;
public interface MetadataSegmentPublisherProvider extends SegmentPublisherProvider
{
public MetadataSegmentPublisher get();
}

View File

@ -0,0 +1,376 @@
/*
* 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;
import com.google.common.base.Supplier;
import com.google.common.collect.ImmutableList;
import com.metamx.common.ISE;
import com.metamx.common.logger.Logger;
import org.apache.commons.dbcp2.BasicDataSource;
import org.skife.jdbi.v2.Batch;
import org.skife.jdbi.v2.DBI;
import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.IDBI;
import org.skife.jdbi.v2.tweak.HandleCallback;
import org.skife.jdbi.v2.util.ByteArrayMapper;
import org.skife.jdbi.v2.util.IntegerMapper;
import java.sql.Connection;
import java.util.List;
public abstract class SQLMetadataConnector implements MetadataStorageConnector
{
private static final Logger log = new Logger(SQLMetadataConnector.class);
private static final String PAYLOAD_TYPE = "BLOB";
private final Supplier<MetadataStorageConnectorConfig> config;
private final Supplier<MetadataStorageTablesConfig> tablesConfigSupplier;
public SQLMetadataConnector(Supplier<MetadataStorageConnectorConfig> config,
Supplier<MetadataStorageTablesConfig> tablesConfigSupplier
)
{
this.config = config;
this.tablesConfigSupplier = tablesConfigSupplier;
}
/**
* SQL type to use for payload data (e.g. JSON blobs).
* Must be a binary type, which values can be accessed using ResultSet.getBytes()
*
* The resulting string will be interpolated into the table creation statement, e.g.
* <code>CREATE TABLE druid_table ( payload <type> NOT NULL, ... )</code>
*
* @return String representing the SQL type
*/
protected String getPayloadType() {
return PAYLOAD_TYPE;
}
/**
* Auto-incrementing SQL type to use for IDs
* Must be an integer type, which values will be automatically set by the database
*
* The resulting string will be interpolated into the table creation statement, e.g.
* <code>CREATE TABLE druid_table ( id <type> NOT NULL, ... )</code>
*
* @return String representing the SQL type and auto-increment statement
*/
protected abstract String getSerialType();
protected abstract boolean tableExists(Handle handle, final String tableName);
protected boolean isTransientException(Throwable e) {
return false;
}
public void createTable(final IDBI dbi, final String tableName, final Iterable<String> sql)
{
try {
dbi.withHandle(
new HandleCallback<Void>()
{
@Override
public Void withHandle(Handle handle) throws Exception
{
if (!tableExists(handle, tableName)) {
log.info("Creating table[%s]", tableName);
final Batch batch = handle.createBatch();
for(String s : sql) {
batch.add(s);
}
batch.execute();
} else {
log.info("Table[%s] already exists", tableName);
}
return null;
}
}
);
}
catch (Exception e) {
log.warn(e, "Exception creating table");
}
}
public void createSegmentTable(final IDBI dbi, final String tableName)
{
createTable(
dbi,
tableName,
ImmutableList.of(
String.format(
"CREATE TABLE %1$s (\n"
+ " id VARCHAR(255) NOT NULL,\n"
+ " dataSource VARCHAR(255) NOT NULL,\n"
+ " created_date VARCHAR(255) NOT NULL,\n"
+ " start VARCHAR(255) NOT NULL,\n"
+ " \"end\" VARCHAR(255) NOT NULL,\n"
+ " partitioned BOOLEAN NOT NULL,\n"
+ " version VARCHAR(255) NOT NULL,\n"
+ " used BOOLEAN NOT NULL,\n"
+ " payload %2$s NOT NULL,\n"
+ " PRIMARY KEY (id)\n"
+ ")",
tableName, getPayloadType()
),
String.format("CREATE INDEX idx_%1$s_datasource ON %1$s(dataSource)", tableName),
String.format("CREATE INDEX idx_%1$s_used ON %1$s(used)", tableName)
)
);
}
public void createRulesTable(final IDBI dbi, final String tableName)
{
createTable(
dbi,
tableName,
ImmutableList.of(
String.format(
"CREATE TABLE %1$s (\n"
+ " id VARCHAR(255) NOT NULL,\n"
+ " dataSource VARCHAR(255) NOT NULL,\n"
+ " version VARCHAR(255) NOT NULL,\n"
+ " payload %2$s NOT NULL,\n"
+ " PRIMARY KEY (id)\n"
+ ")",
tableName, getPayloadType()
),
String.format("CREATE INDEX idx_%1$s_datasource ON %1$s(dataSource)", tableName)
)
);
}
public void createConfigTable(final IDBI dbi, final String tableName)
{
createTable(
dbi,
tableName,
ImmutableList.of(
String.format(
"CREATE TABLE %1$s (\n"
+ " name VARCHAR(255) NOT NULL,\n"
+ " payload %2$s NOT NULL,\n"
+ " PRIMARY KEY(name)\n"
+ ")",
tableName, getPayloadType()
)
)
);
}
public void createEntryTable(final IDBI dbi, final String tableName)
{
createTable(
dbi,
tableName,
ImmutableList.of(
String.format(
"CREATE TABLE %1$s (\n"
+ " id VARCHAR(255) NOT NULL,\n"
+ " created_date VARCHAR(255) NOT NULL,\n"
+ " datasource VARCHAR(255) NOT NULL,\n"
+ " payload %2$s NOT NULL,\n"
+ " status_payload %2$s NOT NULL,\n"
+ " active BOOLEAN NOT NULL DEFAULT FALSE,\n"
+ " PRIMARY KEY (id)\n"
+ ")",
tableName, getPayloadType()
),
String.format("CREATE INDEX idx_%1$s_active_created_date ON %1$s(active, created_date)", tableName)
)
);
}
public void createLogTable(final IDBI dbi, final String tableName, final String entryTypeName)
{
createTable(
dbi,
tableName,
ImmutableList.of(
String.format(
"CREATE TABLE %1$s (\n"
+ " id %2$s NOT NULL,\n"
+ " %4$s_id VARCHAR(255) DEFAULT NULL,\n"
+ " log_payload %3$s,\n"
+ " PRIMARY KEY (id)\n"
+ ")",
tableName, getSerialType(), getPayloadType(), entryTypeName
),
String.format("CREATE INDEX idx_%1$s_%2$s_id ON %1$s(%2$s_id)", tableName, entryTypeName)
)
);
}
public void createLockTable(final IDBI dbi, final String tableName, final String entryTypeName)
{
createTable(
dbi,
tableName,
ImmutableList.of(
String.format(
"CREATE TABLE %1$s (\n"
+ " id %2$s NOT NULL,\n"
+ " %4$s_id VARCHAR(255) DEFAULT NULL,\n"
+ " lock_payload %3$s,\n"
+ " PRIMARY KEY (id)\n"
+ ")",
tableName, getSerialType(), getPayloadType(), entryTypeName
),
String.format("CREATE INDEX idx_%1$s_%2$s_id ON %1$s(%2$s_id)", tableName, entryTypeName)
)
);
}
@Override
public Void insertOrUpdate(
final String tableName,
final String keyColumn,
final String valueColumn,
final String key,
final byte[] value
) throws Exception
{
return getDBI().withHandle(
new HandleCallback<Void>()
{
@Override
public Void withHandle(Handle handle) throws Exception
{
Connection conn = getDBI().open().getConnection();
handle.begin();
conn.setAutoCommit(false);
int count = handle
.createQuery(
String.format("SELECT COUNT(*) FROM %1$s WHERE %2$s = :key", tableName, keyColumn)
)
.bind("key", key)
.map(IntegerMapper.FIRST)
.first();
if (count == 0) {
handle.createStatement(String.format("INSERT INTO %1$s (%2$s, %3$s) VALUES (:key, :value)",
tableName, keyColumn, valueColumn))
.bind("key", key)
.bind("value", value)
.execute();
} else {
handle.createStatement(String.format("UPDATE %1$s SET %3$s=:value WHERE %2$s=:key",
tableName, keyColumn, valueColumn))
.bind("key", key)
.bind("value", value)
.execute();
}
conn.setAutoCommit(true);
handle.commit();
return null;
}
}
);
}
public abstract DBI getDBI();
@Override
public void createSegmentTable() {
if (config.get().isCreateTables()) {
createSegmentTable(getDBI(), tablesConfigSupplier.get().getSegmentsTable());
}
}
@Override
public void createRulesTable() {
if (config.get().isCreateTables()) {
createRulesTable(getDBI(), tablesConfigSupplier.get().getRulesTable());
}
}
@Override
public void createConfigTable() {
if (config.get().isCreateTables()) {
createConfigTable(getDBI(), tablesConfigSupplier.get().getConfigTable());
}
}
@Override
public void createTaskTables() {
if (config.get().isCreateTables()) {
final MetadataStorageTablesConfig tablesConfig = tablesConfigSupplier.get();
final String entryType = tablesConfig.getTaskEntryType();
createEntryTable(getDBI(), tablesConfig.getEntryTable(entryType));
createLogTable(getDBI(), tablesConfig.getLogTable(entryType), entryType);
createLockTable(getDBI(), tablesConfig.getLockTable(entryType), entryType);
}
}
@Override
public byte[] lookup(
final String tableName,
final String keyColumn,
final String valueColumn,
final String key
)
{
final String selectStatement = String.format("SELECT %s FROM %s WHERE %s = :key", valueColumn,
tableName, keyColumn);
return getDBI().withHandle(
new HandleCallback<byte[]>()
{
@Override
public byte[] withHandle(Handle handle) throws Exception
{
List<byte[]> matched = handle.createQuery(selectStatement)
.bind("key", key)
.map(ByteArrayMapper.FIRST)
.list();
if (matched.isEmpty()) {
return null;
}
if (matched.size() > 1) {
throw new ISE("Error! More than one matching entry[%d] found for [%s]?!", matched.size(), key);
}
return matched.get(0);
}
}
);
}
public MetadataStorageConnectorConfig getConfig() { return config.get(); }
protected BasicDataSource getDatasource()
{
MetadataStorageConnectorConfig connectorConfig = getConfig();
BasicDataSource dataSource = new BasicDataSource();
dataSource.setUsername(connectorConfig.getUser());
dataSource.setPassword(connectorConfig.getPassword());
String uri = connectorConfig.getConnectURI();
dataSource.setUrl(uri);
dataSource.setValidationQuery(connectorConfig.getValidationQuery());
dataSource.setTestOnBorrow(true);
return dataSource;
}
}

View File

@ -1,3 +1,4 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
@ -17,7 +18,7 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.db;
package io.druid.metadata;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
@ -27,7 +28,7 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.inject.Inject;
import com.metamx.common.MapUtils;
import com.metamx.common.Pair;
import com.metamx.common.concurrent.ScheduledExecutors;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
@ -46,7 +47,10 @@ import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.IDBI;
import org.skife.jdbi.v2.StatementContext;
import org.skife.jdbi.v2.tweak.HandleCallback;
import org.skife.jdbi.v2.tweak.ResultSetMapper;
import java.io.IOException;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.util.Arrays;
import java.util.List;
@ -57,7 +61,7 @@ import java.util.concurrent.atomic.AtomicReference;
/**
*/
@ManageLifecycle
public class DatabaseRuleManager
public class SQLMetadataRuleManager implements MetadataRuleManager
{
public static void createDefaultRule(
final IDBI dbi,
@ -76,7 +80,7 @@ public class DatabaseRuleManager
List<Map<String, Object>> existing = handle
.createQuery(
String.format(
"SELECT id from %s where datasource=:dataSource;",
"SELECT id from %s where datasource=:dataSource",
ruleTable
)
)
@ -105,7 +109,7 @@ public class DatabaseRuleManager
.bind("id", String.format("%s_%s", defaultDatasourceName, version))
.bind("dataSource", defaultDatasourceName)
.bind("version", version)
.bind("payload", jsonMapper.writeValueAsString(defaultRules))
.bind("payload", jsonMapper.writeValueAsBytes(defaultRules))
.execute();
return null;
@ -118,11 +122,11 @@ public class DatabaseRuleManager
}
}
private static final Logger log = new Logger(DatabaseRuleManager.class);
private static final Logger log = new Logger(SQLMetadataRuleManager.class);
private final ObjectMapper jsonMapper;
private final Supplier<DatabaseRuleManagerConfig> config;
private final Supplier<DbTablesConfig> dbTables;
private final Supplier<MetadataRuleManagerConfig> config;
private final Supplier<MetadataStorageTablesConfig> dbTables;
private final IDBI dbi;
private final AtomicReference<ImmutableMap<String, List<Rule>>> rules;
@ -133,17 +137,17 @@ public class DatabaseRuleManager
private volatile boolean started = false;
@Inject
public DatabaseRuleManager(
public SQLMetadataRuleManager(
@Json ObjectMapper jsonMapper,
Supplier<DatabaseRuleManagerConfig> config,
Supplier<DbTablesConfig> dbTables,
IDBI dbi
Supplier<MetadataRuleManagerConfig> config,
Supplier<MetadataStorageTablesConfig> dbTables,
SQLMetadataConnector connector
)
{
this.jsonMapper = jsonMapper;
this.config = config;
this.dbTables = dbTables;
this.dbi = dbi;
this.dbi = connector.getDBI();
this.rules = new AtomicReference<>(
ImmutableMap.<String, List<Rule>>of()
@ -214,35 +218,50 @@ public class DatabaseRuleManager
+ "ON r.datasource = ds.datasource and r.version = ds.version",
getRulesTable()
)
).fold(
Maps.<String, List<Rule>>newHashMap(),
new Folder3<Map<String, List<Rule>>, Map<String, Object>>()
).map(
new ResultSetMapper<Pair<String, List<Rule>>>()
{
@Override
public Map<String, List<Rule>> fold(
Map<String, List<Rule>> retVal,
Map<String, Object> stringObjectMap,
FoldController foldController,
StatementContext statementContext
) throws SQLException
public Pair<String, List<Rule>> map(int index, ResultSet r, StatementContext ctx)
throws SQLException
{
try {
String dataSource = MapUtils.getString(stringObjectMap, "dataSource");
List<Rule> rules = jsonMapper.readValue(
MapUtils.getString(stringObjectMap, "payload"), new TypeReference<List<Rule>>()
{
}
return Pair.of(
r.getString("dataSource"),
jsonMapper.<List<Rule>>readValue(
r.getBytes("payload"), new TypeReference<List<Rule>>(){}
)
);
retVal.put(dataSource, rules);
return retVal;
}
catch (Exception e) {
catch (IOException e) {
throw Throwables.propagate(e);
}
}
}
);
)
.fold(
Maps.<String, List<Rule>>newHashMap(),
new Folder3<Map<String, List<Rule>>, Pair<String, List<Rule>>>()
{
@Override
public Map<String, List<Rule>> fold(
Map<String, List<Rule>> retVal,
Pair<String, List<Rule>> stringObjectMap,
FoldController foldController,
StatementContext statementContext
) throws SQLException
{
try {
String dataSource = stringObjectMap.lhs;
retVal.put(dataSource, stringObjectMap.rhs);
return retVal;
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
}
);
}
}
)
@ -301,7 +320,7 @@ public class DatabaseRuleManager
.bind("id", String.format("%s_%s", dataSource, version))
.bind("dataSource", dataSource)
.bind("version", version)
.bind("payload", jsonMapper.writeValueAsString(newRules))
.bind("payload", jsonMapper.writeValueAsBytes(newRules))
.execute();
return null;
@ -318,5 +337,8 @@ public class DatabaseRuleManager
return true;
}
private String getRulesTable() {return dbTables.get().getRulesTable();}
private String getRulesTable()
{
return dbTables.get().getRulesTable();
}
}

View File

@ -17,43 +17,45 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.db;
package io.druid.metadata;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Supplier;
import com.google.common.base.Throwables;
import com.google.inject.Inject;
import com.google.inject.Provider;
import com.metamx.common.lifecycle.Lifecycle;
import org.skife.jdbi.v2.IDBI;
/**
*/
public class DatabaseRuleManagerProvider implements Provider<DatabaseRuleManager>
public class SQLMetadataRuleManagerProvider implements MetadataRuleManagerProvider
{
private final ObjectMapper jsonMapper;
private final Supplier<DatabaseRuleManagerConfig> config;
private final Supplier<DbTablesConfig> dbTables;
private final DbConnector dbConnector;
private final Supplier<MetadataRuleManagerConfig> config;
private final Supplier<MetadataStorageTablesConfig> dbTables;
private final SQLMetadataConnector connector;
private final Lifecycle lifecycle;
private final IDBI dbi;
@Inject
public DatabaseRuleManagerProvider(
public SQLMetadataRuleManagerProvider(
ObjectMapper jsonMapper,
Supplier<DatabaseRuleManagerConfig> config,
Supplier<DbTablesConfig> dbTables,
DbConnector dbConnector,
Supplier<MetadataRuleManagerConfig> config,
Supplier<MetadataStorageTablesConfig> dbTables,
SQLMetadataConnector connector,
Lifecycle lifecycle
)
{
this.jsonMapper = jsonMapper;
this.config = config;
this.dbTables = dbTables;
this.dbConnector = dbConnector;
this.connector = connector;
this.dbi = connector.getDBI();
this.lifecycle = lifecycle;
}
@Override
public DatabaseRuleManager get()
public SQLMetadataRuleManager get()
{
try {
lifecycle.addMaybeStartHandler(
@ -62,9 +64,9 @@ public class DatabaseRuleManagerProvider implements Provider<DatabaseRuleManager
@Override
public void start() throws Exception
{
dbConnector.createRulesTable();
DatabaseRuleManager.createDefaultRule(
dbConnector.getDBI(), dbTables.get().getRulesTable(), config.get().getDefaultRule(), jsonMapper
connector.createRulesTable();
SQLMetadataRuleManager.createDefaultRule(
dbi, dbTables.get().getRulesTable(), config.get().getDefaultRule(), jsonMapper
);
}
@ -80,6 +82,6 @@ public class DatabaseRuleManagerProvider implements Provider<DatabaseRuleManager
throw Throwables.propagate(e);
}
return new DatabaseRuleManager(jsonMapper, config, dbTables, dbConnector.getDBI());
return new SQLMetadataRuleManager(jsonMapper, config, dbTables, connector);
}
}

View File

@ -17,7 +17,7 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.db;
package io.druid.metadata;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Supplier;
@ -47,7 +47,11 @@ import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.IDBI;
import org.skife.jdbi.v2.StatementContext;
import org.skife.jdbi.v2.tweak.HandleCallback;
import org.skife.jdbi.v2.tweak.ResultSetMapper;
import org.skife.jdbi.v2.util.ByteArrayMapper;
import java.io.IOException;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.util.ArrayList;
import java.util.Collection;
@ -60,15 +64,15 @@ import java.util.concurrent.atomic.AtomicReference;
/**
*/
@ManageLifecycle
public class DatabaseSegmentManager
public class SQLMetadataSegmentManager implements MetadataSegmentManager
{
private static final Logger log = new Logger(DatabaseSegmentManager.class);
private static final Logger log = new Logger(SQLMetadataSegmentManager.class);
private final Object lock = new Object();
private final ObjectMapper jsonMapper;
private final Supplier<DatabaseSegmentManagerConfig> config;
private final Supplier<DbTablesConfig> dbTables;
private final Supplier<MetadataSegmentManagerConfig> config;
private final Supplier<MetadataStorageTablesConfig> dbTables;
private final AtomicReference<ConcurrentHashMap<String, DruidDataSource>> dataSources;
private final IDBI dbi;
@ -77,20 +81,20 @@ public class DatabaseSegmentManager
private volatile boolean started = false;
@Inject
public DatabaseSegmentManager(
public SQLMetadataSegmentManager(
ObjectMapper jsonMapper,
Supplier<DatabaseSegmentManagerConfig> config,
Supplier<DbTablesConfig> dbTables,
IDBI dbi
Supplier<MetadataSegmentManagerConfig> config,
Supplier<MetadataStorageTablesConfig> dbTables,
SQLMetadataConnector connector
)
{
this.jsonMapper = jsonMapper;
this.config = config;
this.dbTables = dbTables;
this.dataSources = new AtomicReference<ConcurrentHashMap<String, DruidDataSource>>(
this.dataSources = new AtomicReference<>(
new ConcurrentHashMap<String, DruidDataSource>()
);
this.dbi = dbi;
this.dbi = connector.getDBI();
}
@LifecycleStart
@ -137,6 +141,7 @@ public class DatabaseSegmentManager
}
}
@Override
public boolean enableDatasource(final String ds)
{
try {
@ -150,21 +155,22 @@ public class DatabaseSegmentManager
String.format("SELECT payload FROM %s WHERE dataSource = :dataSource", getSegmentsTable())
)
.bind("dataSource", ds)
.map(ByteArrayMapper.FIRST)
.fold(
new VersionedIntervalTimeline<String, DataSegment>(Ordering.natural()),
new Folder3<VersionedIntervalTimeline<String, DataSegment>, Map<String, Object>>()
new Folder3<VersionedIntervalTimeline<String, DataSegment>, byte[]>()
{
@Override
public VersionedIntervalTimeline<String, DataSegment> fold(
VersionedIntervalTimeline<String, DataSegment> timeline,
Map<String, Object> stringObjectMap,
byte[] payload,
FoldController foldController,
StatementContext statementContext
) throws SQLException
{
try {
DataSegment segment = jsonMapper.readValue(
(String) stringObjectMap.get("payload"),
payload,
DataSegment.class
);
@ -234,6 +240,7 @@ public class DatabaseSegmentManager
return true;
}
@Override
public boolean enableSegment(final String segmentId)
{
try {
@ -261,6 +268,7 @@ public class DatabaseSegmentManager
return true;
}
@Override
public boolean removeDatasource(final String ds)
{
try {
@ -298,6 +306,7 @@ public class DatabaseSegmentManager
return true;
}
@Override
public boolean removeSegment(String ds, final String segmentID)
{
try {
@ -339,21 +348,25 @@ public class DatabaseSegmentManager
return true;
}
@Override
public boolean isStarted()
{
return started;
}
@Override
public DruidDataSource getInventoryValue(String key)
{
return dataSources.get().get(key);
}
@Override
public Collection<DruidDataSource> getInventory()
{
return dataSources.get().values();
}
@Override
public Collection<String> getAllDatasourceNames()
{
synchronized (lock) {
@ -392,6 +405,7 @@ public class DatabaseSegmentManager
}
}
@Override
public void poll()
{
try {
@ -401,29 +415,44 @@ public class DatabaseSegmentManager
ConcurrentHashMap<String, DruidDataSource> newDataSources = new ConcurrentHashMap<String, DruidDataSource>();
List<Map<String, Object>> segmentRows = dbi.withHandle(
new HandleCallback<List<Map<String, Object>>>()
List<DataSegment> segments = dbi.withHandle(
new HandleCallback<List<DataSegment>>()
{
@Override
public List<Map<String, Object>> withHandle(Handle handle) throws Exception
public List<DataSegment> withHandle(Handle handle) throws Exception
{
return handle.createQuery(
String.format("SELECT payload FROM %s WHERE used=true", getSegmentsTable())
).list();
)
.map(
new ResultSetMapper<DataSegment>()
{
@Override
public DataSegment map(int index, ResultSet r, StatementContext ctx)
throws SQLException
{
try {
return jsonMapper.readValue(r.getBytes("payload"), DataSegment.class);
}
catch (IOException e) {
throw new SQLException(e);
}
}
}
)
.list();
}
}
);
if (segmentRows == null || segmentRows.isEmpty()) {
if (segments == null || segments.isEmpty()) {
log.warn("No segments found in the database!");
return;
}
log.info("Polled and found %,d segments in the database", segmentRows.size());
for (Map<String, Object> segmentRow : segmentRows) {
DataSegment segment = jsonMapper.readValue((String) segmentRow.get("payload"), DataSegment.class);
log.info("Polled and found %,d segments in the database", segments.size());
for (final DataSegment segment : segments) {
String datasourceName = segment.getDataSource();
DruidDataSource dataSource = newDataSources.get(datasourceName);

View File

@ -17,42 +17,44 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.db;
package io.druid.metadata;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Supplier;
import com.google.inject.Inject;
import com.google.inject.Provider;
import com.metamx.common.lifecycle.Lifecycle;
import org.skife.jdbi.v2.IDBI;
/**
*/
public class DatabaseSegmentManagerProvider implements Provider<DatabaseSegmentManager>
public class SQLMetadataSegmentManagerProvider implements MetadataSegmentManagerProvider
{
private final ObjectMapper jsonMapper;
private final Supplier<DatabaseSegmentManagerConfig> config;
private final Supplier<DbTablesConfig> dbTables;
private final DbConnector dbConnector;
private final Supplier<MetadataSegmentManagerConfig> config;
private final Supplier<MetadataStorageTablesConfig> storageConfig;
private final SQLMetadataConnector connector;
private final IDBI dbi;
private final Lifecycle lifecycle;
@Inject
public DatabaseSegmentManagerProvider(
public SQLMetadataSegmentManagerProvider(
ObjectMapper jsonMapper,
Supplier<DatabaseSegmentManagerConfig> config,
Supplier<DbTablesConfig> dbTables,
DbConnector dbConnector,
Supplier<MetadataSegmentManagerConfig> config,
Supplier<MetadataStorageTablesConfig> storageConfig,
SQLMetadataConnector connector,
Lifecycle lifecycle
)
{
this.jsonMapper = jsonMapper;
this.config = config;
this.dbTables = dbTables;
this.dbConnector = dbConnector;
this.storageConfig = storageConfig;
this.connector = connector;
this.dbi = this.connector.getDBI();
this.lifecycle = lifecycle;
}
@Override
public DatabaseSegmentManager get()
public MetadataSegmentManager get()
{
lifecycle.addHandler(
new Lifecycle.Handler()
@ -60,7 +62,7 @@ public class DatabaseSegmentManagerProvider implements Provider<DatabaseSegmentM
@Override
public void start() throws Exception
{
dbConnector.createSegmentTable();
connector.createSegmentTable();
}
@Override
@ -71,11 +73,11 @@ public class DatabaseSegmentManagerProvider implements Provider<DatabaseSegmentM
}
);
return new DatabaseSegmentManager(
return new SQLMetadataSegmentManager(
jsonMapper,
config,
dbTables,
dbConnector.getDBI()
storageConfig,
connector
);
}
}

View File

@ -1,3 +1,5 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013 Metamarkets Group Inc.
@ -17,16 +19,16 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.segment.realtime;
package io.druid.metadata;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Inject;
import com.metamx.common.logger.Logger;
import io.druid.db.DbConnector;
import io.druid.db.DbTablesConfig;
import io.druid.segment.realtime.SegmentPublisher;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
import org.joda.time.DateTime;
import org.skife.jdbi.v2.DBI;
import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.IDBI;
import org.skife.jdbi.v2.tweak.HandleCallback;
@ -35,44 +37,37 @@ import java.io.IOException;
import java.util.List;
import java.util.Map;
public class DbSegmentPublisher implements SegmentPublisher
public class SQLMetadataSegmentPublisher implements MetadataSegmentPublisher
{
private static final Logger log = new Logger(DbSegmentPublisher.class);
private static final Logger log = new Logger(SQLMetadataSegmentPublisher.class);
private final ObjectMapper jsonMapper;
private final DbTablesConfig config;
private final IDBI dbi;
private final MetadataStorageTablesConfig config;
private final SQLMetadataConnector connector;
private final String statement;
@Inject
public DbSegmentPublisher(
public SQLMetadataSegmentPublisher(
ObjectMapper jsonMapper,
DbTablesConfig config,
IDBI dbi
MetadataStorageTablesConfig config,
SQLMetadataConnector connector
)
{
this.jsonMapper = jsonMapper;
this.config = config;
this.dbi = dbi;
if (DbConnector.isPostgreSQL(dbi)) {
this.statement = String.format(
"INSERT INTO %s (id, dataSource, created_date, start, \"end\", partitioned, version, used, payload) "
+ "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)",
config.getSegmentsTable()
);
} else {
this.statement = String.format(
"INSERT INTO %s (id, dataSource, created_date, start, end, partitioned, version, used, payload) "
+ "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)",
config.getSegmentsTable()
);
}
this.connector = connector;
this.statement = String.format(
"INSERT INTO %s (id, dataSource, created_date, start, \"end\", partitioned, version, used, payload) "
+ "VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)",
config.getSegmentsTable()
);
}
@Override
public void publishSegment(final DataSegment segment) throws IOException
{
try {
final DBI dbi = connector.getDBI();
List<Map<String, Object>> exists = dbi.withHandle(
new HandleCallback<List<Map<String, Object>>>()
{
@ -108,7 +103,7 @@ public class DbSegmentPublisher implements SegmentPublisher
.bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? 0 : 1)
.bind("version", segment.getVersion())
.bind("used", true)
.bind("payload", jsonMapper.writeValueAsString(segment))
.bind("payload", jsonMapper.writeValueAsBytes(segment))
.execute();
return null;

View File

@ -17,12 +17,10 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.guice;
package io.druid.metadata;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.databind.ObjectMapper;
import io.druid.db.DbTablesConfig;
import io.druid.segment.realtime.DbSegmentPublisher;
import io.druid.segment.realtime.SegmentPublisher;
import org.skife.jdbi.v2.IDBI;
@ -30,23 +28,23 @@ import javax.validation.constraints.NotNull;
/**
*/
public class DbSegmentPublisherProvider implements SegmentPublisherProvider
public class SQLMetadataSegmentPublisherProvider implements MetadataSegmentPublisherProvider
{
@JacksonInject
@NotNull
private IDBI idbi = null;
private SQLMetadataConnector connector = null;
@JacksonInject
@NotNull
private DbTablesConfig config = null;
private MetadataStorageTablesConfig config = null;
@JacksonInject
@NotNull
private ObjectMapper jsonMapper = null;
@Override
public SegmentPublisher get()
public MetadataSegmentPublisher get()
{
return new DbSegmentPublisher(jsonMapper, config, idbi);
return new SQLMetadataSegmentPublisher(jsonMapper, config, connector);
}
}

View File

@ -0,0 +1,516 @@
/*
* 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;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.api.client.repackaged.com.google.common.base.Preconditions;
import com.google.api.client.util.Maps;
import com.google.common.base.Charsets;
import com.google.common.base.Optional;
import com.google.common.base.Predicate;
import com.google.common.base.Throwables;
import com.google.common.collect.Lists;
import com.metamx.common.Pair;
import com.metamx.common.RetryUtils;
import com.metamx.emitter.EmittingLogger;
import org.joda.time.DateTime;
import org.skife.jdbi.v2.FoldController;
import org.skife.jdbi.v2.Folder3;
import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.StatementContext;
import org.skife.jdbi.v2.exceptions.CallbackFailedException;
import org.skife.jdbi.v2.exceptions.DBIException;
import org.skife.jdbi.v2.exceptions.StatementException;
import org.skife.jdbi.v2.exceptions.UnableToObtainConnectionException;
import org.skife.jdbi.v2.tweak.HandleCallback;
import org.skife.jdbi.v2.tweak.ResultSetMapper;
import org.skife.jdbi.v2.util.ByteArrayMapper;
import java.io.IOException;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.SQLRecoverableException;
import java.sql.SQLTransientException;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Callable;
public class SQLMetadataStorageActionHandler<EntryType, StatusType, LogType, LockType>
implements MetadataStorageActionHandler<EntryType, StatusType, LogType, LockType>
{
private static final EmittingLogger log = new EmittingLogger(SQLMetadataStorageActionHandler.class);
private final SQLMetadataConnector connector;
private final ObjectMapper jsonMapper;
private final TypeReference entryType;
private final TypeReference statusType;
private final TypeReference logType;
private final TypeReference lockType;
private final String entryTypeName;
private final String entryTable;
private final String logTable;
private final String lockTable;
public SQLMetadataStorageActionHandler(
final SQLMetadataConnector connector,
final ObjectMapper jsonMapper,
final MetadataStorageActionHandlerTypes<EntryType, StatusType, LogType, LockType> types,
final String entryTypeName,
final String entryTable,
final String logTable,
final String lockTable
)
{
this.connector = connector;
this.jsonMapper = jsonMapper;
this.entryType = types.getEntryType();
this.statusType = types.getStatusType();
this.logType = types.getLogType();
this.lockType = types.getLockType();
this.entryTypeName = entryTypeName;
this.entryTable = entryTable;
this.logTable = logTable;
this.lockTable = lockTable;
}
public void insert(
final String id,
final DateTime timestamp,
final String dataSource,
final EntryType entry,
final boolean active,
final StatusType status
) throws EntryExistsException
{
try {
retryingHandle(
new HandleCallback<Void>()
{
@Override
public Void withHandle(Handle handle) throws Exception
{
handle.createStatement(
String.format(
"INSERT INTO %s (id, created_date, datasource, payload, active, status_payload) VALUES (:id, :created_date, :datasource, :payload, :active, :status_payload)",
entryTable
)
)
.bind("id", id)
.bind("created_date", timestamp.toString())
.bind("datasource", dataSource)
.bind("payload", jsonMapper.writeValueAsBytes(entry))
.bind("active", active)
.bind("status_payload", jsonMapper.writeValueAsBytes(status))
.execute();
return null;
}
}
);
} catch(Exception e) {
final boolean isStatementException = e instanceof StatementException ||
(e instanceof CallbackFailedException
&& e.getCause() instanceof StatementException);
if (isStatementException && getEntry(id).isPresent()) {
throw new EntryExistsException(id, e);
} else {
throw Throwables.propagate(e);
}
}
}
public boolean setStatus(final String entryId, final boolean active, final StatusType status)
{
return retryingHandle(
new HandleCallback<Boolean>()
{
@Override
public Boolean withHandle(Handle handle) throws Exception
{
return handle.createStatement(
String.format(
"UPDATE %s SET active = :active, status_payload = :status_payload WHERE id = :id AND active = TRUE",
entryTable
)
)
.bind("id", entryId)
.bind("active", active)
.bind("status_payload", jsonMapper.writeValueAsBytes(status))
.execute() == 1;
}
}
);
}
public Optional<EntryType> getEntry(final String entryId)
{
return retryingHandle(
new HandleCallback<Optional<EntryType>>()
{
@Override
public Optional<EntryType> withHandle(Handle handle) throws Exception
{
return Optional.fromNullable(
jsonMapper.<EntryType>readValue(
handle.createQuery(
String.format("SELECT payload FROM %s WHERE id = :id", entryTable)
)
.bind("id", entryId)
.map(ByteArrayMapper.FIRST)
.first(),
entryType
)
);
}
}
);
}
public Optional<StatusType> getStatus(final String entryId)
{
return retryingHandle(
new HandleCallback<Optional<StatusType>>()
{
@Override
public Optional<StatusType> withHandle(Handle handle) throws Exception
{
return Optional.fromNullable(
jsonMapper.<StatusType>readValue(
handle.createQuery(
String.format("SELECT status_payload FROM %s WHERE id = :id", entryTable)
)
.bind("id", entryId)
.map(ByteArrayMapper.FIRST)
.first(),
statusType
)
);
}
}
);
}
public List<Pair<EntryType, StatusType>> getActiveEntriesWithStatus()
{
return retryingHandle(
new HandleCallback<List<Pair<EntryType, StatusType>>>()
{
@Override
public List<Pair<EntryType, StatusType>> withHandle(Handle handle) throws Exception
{
return handle
.createQuery(
String.format(
"SELECT id, payload, status_payload FROM %s WHERE active = TRUE ORDER BY created_date",
entryTable
)
)
.map(
new ResultSetMapper<Pair<EntryType, StatusType>>()
{
@Override
public Pair<EntryType, StatusType> map(int index, ResultSet r, StatementContext ctx)
throws SQLException
{
try {
return Pair.of(
jsonMapper.<EntryType>readValue(
r.getBytes("payload"),
entryType
),
jsonMapper.<StatusType>readValue(
r.getBytes("status_payload"),
statusType
)
);
}
catch (IOException e) {
log.makeAlert(e, "Failed to parse entry payload").addData("entry", r.getString("id")).emit();
throw new SQLException(e);
}
}
}
).list();
}
}
);
}
public List<StatusType> getInactiveStatusesSince(final DateTime timestamp)
{
return retryingHandle(
new HandleCallback<List<StatusType>>()
{
@Override
public List<StatusType> withHandle(Handle handle) throws Exception
{
return handle
.createQuery(
String.format(
"SELECT id, status_payload FROM %s WHERE active = FALSE AND created_date >= :start ORDER BY created_date DESC",
entryTable
)
).bind("start", timestamp.toString())
.map(
new ResultSetMapper<StatusType>()
{
@Override
public StatusType map(int index, ResultSet r, StatementContext ctx) throws SQLException
{
try {
return jsonMapper.readValue(
r.getBytes("status_payload"),
statusType
);
}
catch (IOException e) {
log.makeAlert(e, "Failed to parse status payload")
.addData("entry", r.getString("id"))
.emit();
throw new SQLException(e);
}
}
}
).list();
}
}
);
}
public boolean addLock(final String entryId, final LockType lock)
{
return retryingHandle(
new HandleCallback<Boolean>()
{
@Override
public Boolean withHandle(Handle handle) throws Exception
{
return handle.createStatement(
String.format(
"INSERT INTO %1$s (%2$s_id, lock_payload) VALUES (:entryId, :payload)",
lockTable, entryTypeName
)
)
.bind("entryId", entryId)
.bind("payload", jsonMapper.writeValueAsBytes(lock))
.execute() == 1;
}
}
);
}
public boolean removeLock(final long lockId)
{
return retryingHandle(
new HandleCallback<Boolean>()
{
@Override
public Boolean withHandle(Handle handle) throws Exception
{
return handle.createStatement(
String.format(
"DELETE FROM %s WHERE id = :id",
lockTable
)
)
.bind("id", lockId)
.execute() == 1;
}
}
);
}
public boolean addLog(final String entryId, final LogType log)
{
return retryingHandle(
new HandleCallback<Boolean>()
{
@Override
public Boolean withHandle(Handle handle) throws Exception
{
return handle.createStatement(
String.format(
"INSERT INTO %1$s (%2$s_id, log_payload) VALUES (:entryId, :payload)",
logTable, entryTypeName
)
)
.bind("entryId", entryId)
.bind("payload", jsonMapper.writeValueAsBytes(log))
.execute() == 1;
}
}
);
}
public List<LogType> getLogs(final String entryId)
{
return retryingHandle(
new HandleCallback<List<LogType>>()
{
@Override
public List<LogType> withHandle(Handle handle) throws Exception
{
return handle
.createQuery(
String.format(
"SELECT log_payload FROM %1$s WHERE %2$s_id = :entryId",
logTable, entryTypeName
)
)
.bind("entryId", entryId)
.map(ByteArrayMapper.FIRST)
.fold(
Lists.<LogType>newLinkedList(),
new Folder3<List<LogType>, byte[]>()
{
@Override
public List<LogType> fold(
List<LogType> list, byte[] bytes, FoldController control, StatementContext ctx
) throws SQLException
{
try {
list.add(
jsonMapper.<LogType>readValue(
bytes, logType
)
);
return list;
}
catch (IOException e) {
log.makeAlert(e, "Failed to deserialize log")
.addData("entryId", entryId)
.addData("payload", new String(bytes, Charsets.UTF_8))
.emit();
throw new SQLException(e);
}
}
}
);
}
}
);
}
public Map<Long, LockType> getLocks(final String entryId)
{
return retryingHandle(
new HandleCallback<Map<Long, LockType>>()
{
@Override
public Map<Long, LockType> withHandle(Handle handle) throws Exception
{
return handle.createQuery(
String.format(
"SELECT id, lock_payload FROM %1$s WHERE %2$s_id = :entryId",
lockTable, entryTypeName
)
)
.bind("entryId", entryId)
.map(
new ResultSetMapper<Pair<Long, LockType>>()
{
@Override
public Pair<Long, LockType> map(int index, ResultSet r, StatementContext ctx)
throws SQLException
{
try {
return Pair.of(
r.getLong("id"),
jsonMapper.<LockType>readValue(
r.getBytes("lock_payload"),
lockType
)
);
}
catch (IOException e) {
log.makeAlert(e, "Failed to deserialize " + lockType.getType())
.addData("id", r.getLong("id"))
.addData(
"lockPayload", new String(r.getBytes("lock_payload"), Charsets.UTF_8)
)
.emit();
throw new SQLException(e);
}
}
}
)
.fold(
Maps.<Long, LockType>newLinkedHashMap(),
new Folder3<Map<Long, LockType>, Pair<Long, LockType>>()
{
@Override
public Map<Long, LockType> fold(
Map<Long, LockType> accumulator,
Pair<Long, LockType> lock,
FoldController control,
StatementContext ctx
) throws SQLException
{
accumulator.put(lock.lhs, lock.rhs);
return accumulator;
}
}
);
}
}
);
}
private <T> T retryingHandle(final HandleCallback<T> callback)
{
final Callable<T> call = new Callable<T>()
{
@Override
public T call() throws Exception
{
return connector.getDBI().withHandle(callback);
}
};
final Predicate<Throwable> shouldRetry = new Predicate<Throwable>()
{
@Override
public boolean apply(Throwable e)
{
return shouldRetryException(e);
}
};
final int maxTries = 10;
try {
return RetryUtils.retry(call, shouldRetry, maxTries);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
protected boolean shouldRetryException(final Throwable e)
{
return e != null && (e instanceof SQLTransientException
|| connector.isTransientException(e)
|| e instanceof SQLRecoverableException
|| e instanceof UnableToObtainConnectionException
|| (e instanceof SQLException && shouldRetryException(e.getCause()))
|| (e instanceof DBIException && shouldRetryException(e.getCause())));
}
}

View File

@ -0,0 +1,58 @@
/*
* 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.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Inject;
public class SQLMetadataStorageActionHandlerFactory implements MetadataStorageActionHandlerFactory
{
private final SQLMetadataConnector connector;
private final MetadataStorageTablesConfig config;
private final ObjectMapper jsonMapper;
@Inject
public SQLMetadataStorageActionHandlerFactory(
SQLMetadataConnector connector,
MetadataStorageTablesConfig config,
ObjectMapper jsonMapper
)
{
this.connector = connector;
this.config = config;
this.jsonMapper = jsonMapper;
}
public <A,B,C,D> MetadataStorageActionHandler<A,B,C,D> create(
final String entryType,
MetadataStorageActionHandlerTypes<A,B,C,D> payloadTypes
)
{
return new SQLMetadataStorageActionHandler<>(
connector,
jsonMapper,
payloadTypes,
entryType,
config.getEntryTable(entryType),
config.getLogTable(entryType),
config.getLockTable(entryType)
);
}
}

View File

@ -17,18 +17,19 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.guice;
package io.druid.metadata;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.google.inject.Provider;
import io.druid.guice.NoopSegmentPublisherProvider;
import io.druid.segment.realtime.SegmentPublisher;
/**
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = NoopSegmentPublisherProvider.class)
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "db", value = DbSegmentPublisherProvider.class)
@JsonSubTypes.Type(name = "metadata", value = MetadataSegmentPublisherProvider.class)
})
public interface SegmentPublisherProvider extends Provider<SegmentPublisher>
{

View File

@ -23,6 +23,6 @@ import org.skife.config.Config;
public abstract class DbSegmentPublisherConfig
{
@Config("druid.db.tables.segmentTable")
@Config("druid.metadata.storage.tables.segmentTable")
public abstract String getSegmentTable();
}

View File

@ -25,9 +25,9 @@ import com.google.inject.Inject;
import com.metamx.common.logger.Logger;
import io.druid.client.ServerView;
import io.druid.concurrent.Execs;
import io.druid.db.DatabaseSegmentManager;
import io.druid.metadata.MetadataSegmentManager;
import io.druid.segment.loading.SegmentLoaderConfig;
import io.druid.segment.realtime.DbSegmentPublisher;
import io.druid.segment.realtime.SegmentPublisher;
import io.druid.server.coordination.BaseZkCoordinator;
import io.druid.server.coordination.DataSegmentChangeCallback;
import io.druid.server.coordination.DataSegmentChangeHandler;
@ -44,8 +44,8 @@ public class BridgeZkCoordinator extends BaseZkCoordinator
{
private static final Logger log = new Logger(BaseZkCoordinator.class);
private final DbSegmentPublisher dbSegmentPublisher;
private final DatabaseSegmentManager databaseSegmentManager;
private final SegmentPublisher dbSegmentPublisher;
private final MetadataSegmentManager databaseSegmentManager;
private final ServerView serverView;
private final ExecutorService exec = Execs.singleThreaded("BridgeZkCoordinatorServerView-%s");
@ -57,8 +57,8 @@ public class BridgeZkCoordinator extends BaseZkCoordinator
SegmentLoaderConfig config,
DruidServerMetadata me,
@Bridge CuratorFramework curator,
DbSegmentPublisher dbSegmentPublisher,
DatabaseSegmentManager databaseSegmentManager,
SegmentPublisher dbSegmentPublisher,
MetadataSegmentManager databaseSegmentManager,
ServerView serverView
)
{

View File

@ -50,8 +50,8 @@ import io.druid.collections.CountingMap;
import io.druid.common.config.JacksonConfigManager;
import io.druid.concurrent.Execs;
import io.druid.curator.discovery.ServiceAnnouncer;
import io.druid.db.DatabaseRuleManager;
import io.druid.db.DatabaseSegmentManager;
import io.druid.metadata.MetadataRuleManager;
import io.druid.metadata.MetadataSegmentManager;
import io.druid.guice.ManageLifecycle;
import io.druid.guice.annotations.Self;
import io.druid.segment.IndexIO;
@ -97,9 +97,9 @@ public class DruidCoordinator
private final DruidCoordinatorConfig config;
private final ZkPathsConfig zkPaths;
private final JacksonConfigManager configManager;
private final DatabaseSegmentManager databaseSegmentManager;
private final MetadataSegmentManager metadataSegmentManager;
private final ServerInventoryView<Object> serverInventoryView;
private final DatabaseRuleManager databaseRuleManager;
private final MetadataRuleManager metadataRuleManager;
private final CuratorFramework curator;
private final ServiceEmitter emitter;
private final IndexingServiceClient indexingServiceClient;
@ -120,9 +120,9 @@ public class DruidCoordinator
DruidCoordinatorConfig config,
ZkPathsConfig zkPaths,
JacksonConfigManager configManager,
DatabaseSegmentManager databaseSegmentManager,
MetadataSegmentManager metadataSegmentManager,
ServerInventoryView serverInventoryView,
DatabaseRuleManager databaseRuleManager,
MetadataRuleManager metadataRuleManager,
CuratorFramework curator,
ServiceEmitter emitter,
ScheduledExecutorFactory scheduledExecutorFactory,
@ -136,9 +136,9 @@ public class DruidCoordinator
config,
zkPaths,
configManager,
databaseSegmentManager,
metadataSegmentManager,
serverInventoryView,
databaseRuleManager,
metadataRuleManager,
curator,
emitter,
scheduledExecutorFactory,
@ -154,9 +154,9 @@ public class DruidCoordinator
DruidCoordinatorConfig config,
ZkPathsConfig zkPaths,
JacksonConfigManager configManager,
DatabaseSegmentManager databaseSegmentManager,
MetadataSegmentManager metadataSegmentManager,
ServerInventoryView serverInventoryView,
DatabaseRuleManager databaseRuleManager,
MetadataRuleManager metadataRuleManager,
CuratorFramework curator,
ServiceEmitter emitter,
ScheduledExecutorFactory scheduledExecutorFactory,
@ -171,9 +171,9 @@ public class DruidCoordinator
this.zkPaths = zkPaths;
this.configManager = configManager;
this.databaseSegmentManager = databaseSegmentManager;
this.metadataSegmentManager = metadataSegmentManager;
this.serverInventoryView = serverInventoryView;
this.databaseRuleManager = databaseRuleManager;
this.metadataRuleManager = metadataRuleManager;
this.curator = curator;
this.emitter = emitter;
this.indexingServiceClient = indexingServiceClient;
@ -207,7 +207,7 @@ public class DruidCoordinator
final DateTime now = new DateTime();
for (DataSegment segment : getAvailableDataSegments()) {
List<Rule> rules = databaseRuleManager.getRulesWithDefault(segment.getDataSource());
List<Rule> rules = metadataRuleManager.getRulesWithDefault(segment.getDataSource());
for (Rule rule : rules) {
if (rule instanceof LoadRule && rule.appliesTo(segment, now)) {
for (Map.Entry<String, Integer> entry : ((LoadRule) rule).getTieredReplicants().entrySet()) {
@ -250,7 +250,7 @@ public class DruidCoordinator
public Map<String, Double> getLoadStatus()
{
Map<String, Double> loadStatus = Maps.newHashMap();
for (DruidDataSource dataSource : databaseSegmentManager.getInventory()) {
for (DruidDataSource dataSource : metadataSegmentManager.getInventory()) {
final Set<DataSegment> segments = Sets.newHashSet(dataSource.getSegments());
final int availableSegmentSize = segments.size();
@ -283,17 +283,17 @@ public class DruidCoordinator
public void removeSegment(DataSegment segment)
{
log.info("Removing Segment[%s]", segment);
databaseSegmentManager.removeSegment(segment.getDataSource(), segment.getIdentifier());
metadataSegmentManager.removeSegment(segment.getDataSource(), segment.getIdentifier());
}
public void removeDatasource(String ds)
{
databaseSegmentManager.removeDatasource(ds);
metadataSegmentManager.removeDatasource(ds);
}
public void enableDatasource(String ds)
{
databaseSegmentManager.enableDatasource(ds);
metadataSegmentManager.enableDatasource(ds);
}
public String getCurrentLeader()
@ -419,7 +419,7 @@ public class DruidCoordinator
{
return Iterables.concat(
Iterables.transform(
databaseSegmentManager.getInventory(),
metadataSegmentManager.getInventory(),
new Function<DruidDataSource, Iterable<DataSegment>>()
{
@Override
@ -512,8 +512,8 @@ public class DruidCoordinator
try {
leaderCounter++;
leader = true;
databaseSegmentManager.start();
databaseRuleManager.start();
metadataSegmentManager.start();
metadataRuleManager.start();
serverInventoryView.start();
serviceAnnouncer.announce(self);
final int startingLeaderCounter = leaderCounter;
@ -602,8 +602,8 @@ public class DruidCoordinator
serviceAnnouncer.unannounce(self);
serverInventoryView.stop();
databaseRuleManager.stop();
databaseSegmentManager.stop();
metadataRuleManager.stop();
metadataSegmentManager.stop();
leader = false;
}
catch (Exception e) {
@ -707,7 +707,7 @@ public class DruidCoordinator
}
List<Boolean> allStarted = Arrays.asList(
databaseSegmentManager.isStarted(),
metadataSegmentManager.isStarted(),
serverInventoryView.isStarted()
);
for (Boolean aBoolean : allStarted) {
@ -725,7 +725,7 @@ public class DruidCoordinator
DruidCoordinatorRuntimeParams params =
DruidCoordinatorRuntimeParams.newBuilder()
.withStartTime(startTime)
.withDatasources(databaseSegmentManager.getInventory())
.withDatasources(metadataSegmentManager.getInventory())
.withDynamicConfigs(getDynamicConfigs())
.withEmitter(emitter)
.withBalancerStrategyFactory(factory)
@ -821,7 +821,7 @@ public class DruidCoordinator
return params.buildFromExisting()
.withDruidCluster(cluster)
.withDatabaseRuleManager(databaseRuleManager)
.withDatabaseRuleManager(metadataRuleManager)
.withLoadManagementPeons(loadManagementPeons)
.withSegmentReplicantLookup(segmentReplicantLookup)
.withBalancerReferenceTimestamp(DateTime.now())

View File

@ -24,7 +24,7 @@ import com.google.common.collect.Sets;
import com.metamx.common.guava.Comparators;
import com.metamx.emitter.service.ServiceEmitter;
import io.druid.client.DruidDataSource;
import io.druid.db.DatabaseRuleManager;
import io.druid.metadata.MetadataRuleManager;
import io.druid.timeline.DataSegment;
import org.joda.time.DateTime;
@ -39,7 +39,7 @@ public class DruidCoordinatorRuntimeParams
{
private final long startTime;
private final DruidCluster druidCluster;
private final DatabaseRuleManager databaseRuleManager;
private final MetadataRuleManager databaseRuleManager;
private final SegmentReplicantLookup segmentReplicantLookup;
private final Set<DruidDataSource> dataSources;
private final Set<DataSegment> availableSegments;
@ -54,7 +54,7 @@ public class DruidCoordinatorRuntimeParams
public DruidCoordinatorRuntimeParams(
long startTime,
DruidCluster druidCluster,
DatabaseRuleManager databaseRuleManager,
MetadataRuleManager databaseRuleManager,
SegmentReplicantLookup segmentReplicantLookup,
Set<DruidDataSource> dataSources,
Set<DataSegment> availableSegments,
@ -92,7 +92,7 @@ public class DruidCoordinatorRuntimeParams
return druidCluster;
}
public DatabaseRuleManager getDatabaseRuleManager()
public MetadataRuleManager getDatabaseRuleManager()
{
return databaseRuleManager;
}
@ -180,7 +180,7 @@ public class DruidCoordinatorRuntimeParams
{
private long startTime;
private DruidCluster druidCluster;
private DatabaseRuleManager databaseRuleManager;
private MetadataRuleManager databaseRuleManager;
private SegmentReplicantLookup segmentReplicantLookup;
private final Set<DruidDataSource> dataSources;
private final Set<DataSegment> availableSegments;
@ -212,7 +212,7 @@ public class DruidCoordinatorRuntimeParams
Builder(
long startTime,
DruidCluster cluster,
DatabaseRuleManager databaseRuleManager,
MetadataRuleManager databaseRuleManager,
SegmentReplicantLookup segmentReplicantLookup,
Set<DruidDataSource> dataSources,
Set<DataSegment> availableSegments,
@ -271,7 +271,7 @@ public class DruidCoordinatorRuntimeParams
return this;
}
public Builder withDatabaseRuleManager(DatabaseRuleManager databaseRuleManager)
public Builder withDatabaseRuleManager(MetadataRuleManager databaseRuleManager)
{
this.databaseRuleManager = databaseRuleManager;
return this;

View File

@ -20,7 +20,7 @@
package io.druid.server.coordinator.helper;
import com.metamx.emitter.EmittingLogger;
import io.druid.db.DatabaseRuleManager;
import io.druid.metadata.MetadataRuleManager;
import io.druid.server.coordinator.CoordinatorStats;
import io.druid.server.coordinator.DruidCluster;
import io.druid.server.coordinator.DruidCoordinator;
@ -86,7 +86,7 @@ public class DruidCoordinatorRuleRunner implements DruidCoordinatorHelper
// Run through all matched rules for available segments
DateTime now = new DateTime();
DatabaseRuleManager databaseRuleManager = paramsWithReplicationManager.getDatabaseRuleManager();
MetadataRuleManager databaseRuleManager = paramsWithReplicationManager.getDatabaseRuleManager();
for (DataSegment segment : paramsWithReplicationManager.getAvailableSegments()) {
List<Rule> rules = databaseRuleManager.getRulesWithDefault(segment.getDataSource());
boolean foundMatchingRule = false;

View File

@ -23,8 +23,8 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Inject;
import io.druid.client.InventoryView;
import io.druid.client.indexing.IndexingServiceClient;
import io.druid.db.DatabaseRuleManager;
import io.druid.db.DatabaseSegmentManager;
import io.druid.metadata.MetadataRuleManager;
import io.druid.metadata.MetadataSegmentManager;
import io.druid.server.coordinator.DruidCoordinator;
import javax.annotation.Nullable;
@ -40,8 +40,8 @@ public class BackwardsCompatibleInfoResource extends InfoResource
public BackwardsCompatibleInfoResource(
DruidCoordinator coordinator,
InventoryView serverInventoryView,
DatabaseSegmentManager databaseSegmentManager,
DatabaseRuleManager databaseRuleManager,
MetadataSegmentManager databaseSegmentManager,
MetadataRuleManager databaseRuleManager,
@Nullable IndexingServiceClient indexingServiceClient,
ObjectMapper jsonMapper
)

View File

@ -33,7 +33,7 @@ import io.druid.client.DruidDataSource;
import io.druid.client.DruidServer;
import io.druid.client.InventoryView;
import io.druid.client.indexing.IndexingServiceClient;
import io.druid.db.DatabaseSegmentManager;
import io.druid.metadata.MetadataSegmentManager;
import io.druid.timeline.DataSegment;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@ -60,13 +60,13 @@ import java.util.TreeSet;
public class DatasourcesResource
{
private final InventoryView serverInventoryView;
private final DatabaseSegmentManager databaseSegmentManager;
private final MetadataSegmentManager databaseSegmentManager;
private final IndexingServiceClient indexingServiceClient;
@Inject
public DatasourcesResource(
InventoryView serverInventoryView,
DatabaseSegmentManager databaseSegmentManager,
MetadataSegmentManager databaseSegmentManager,
@Nullable IndexingServiceClient indexingServiceClient
)
{

View File

@ -33,8 +33,8 @@ import io.druid.client.DruidDataSource;
import io.druid.client.DruidServer;
import io.druid.client.InventoryView;
import io.druid.client.indexing.IndexingServiceClient;
import io.druid.db.DatabaseRuleManager;
import io.druid.db.DatabaseSegmentManager;
import io.druid.metadata.MetadataRuleManager;
import io.druid.metadata.MetadataSegmentManager;
import io.druid.server.coordinator.DruidCoordinator;
import io.druid.server.coordinator.rules.LoadRule;
import io.druid.server.coordinator.rules.Rule;
@ -102,8 +102,8 @@ public class InfoResource
private final DruidCoordinator coordinator;
private final InventoryView serverInventoryView;
private final DatabaseSegmentManager databaseSegmentManager;
private final DatabaseRuleManager databaseRuleManager;
private final MetadataSegmentManager metadataSegmentManager;
private final MetadataRuleManager metadataRuleManager;
private final IndexingServiceClient indexingServiceClient;
private final ObjectMapper jsonMapper;
@ -112,8 +112,8 @@ public class InfoResource
public InfoResource(
DruidCoordinator coordinator,
InventoryView serverInventoryView,
DatabaseSegmentManager databaseSegmentManager,
DatabaseRuleManager databaseRuleManager,
MetadataSegmentManager metadataSegmentManager,
MetadataRuleManager metadataRuleManager,
@Nullable
IndexingServiceClient indexingServiceClient,
ObjectMapper jsonMapper
@ -121,8 +121,8 @@ public class InfoResource
{
this.coordinator = coordinator;
this.serverInventoryView = serverInventoryView;
this.databaseSegmentManager = databaseSegmentManager;
this.databaseRuleManager = databaseRuleManager;
this.metadataSegmentManager = metadataSegmentManager;
this.metadataRuleManager = metadataRuleManager;
this.indexingServiceClient = indexingServiceClient;
this.jsonMapper = jsonMapper;
}
@ -358,7 +358,7 @@ public class InfoResource
// This will def. be removed as part of the next release
return Response.ok().entity(
Maps.transformValues(
databaseRuleManager.getAllRules(),
metadataRuleManager.getAllRules(),
new Function<List<Rule>, Object>()
{
@Override
@ -374,8 +374,8 @@ public class InfoResource
if (rule instanceof LoadRule) {
Map<String, Object> newRule = jsonMapper.convertValue(
rule, new TypeReference<Map<String, Object>>()
{
}
{
}
);
Set<String> tiers = Sets.newHashSet(((LoadRule) rule).getTieredReplicants().keySet());
tiers.remove(DruidServer.DEFAULT_TIER);
@ -409,10 +409,10 @@ public class InfoResource
)
{
if (full != null) {
return Response.ok(databaseRuleManager.getRulesWithDefault(dataSourceName))
return Response.ok(metadataRuleManager.getRulesWithDefault(dataSourceName))
.build();
}
return Response.ok(databaseRuleManager.getRules(dataSourceName))
return Response.ok(metadataRuleManager.getRules(dataSourceName))
.build();
}
@ -424,7 +424,7 @@ public class InfoResource
final List<Rule> rules
)
{
if (databaseRuleManager.overrideRule(dataSourceName, rules)) {
if (metadataRuleManager.overrideRule(dataSourceName, rules)) {
return Response.status(Response.Status.OK).build();
}
return Response.status(Response.Status.INTERNAL_SERVER_ERROR).build();
@ -491,7 +491,7 @@ public class InfoResource
if (kill != null && Boolean.valueOf(kill)) {
indexingServiceClient.killSegments(dataSourceName, new Interval(interval));
} else {
if (!databaseSegmentManager.removeDatasource(dataSourceName)) {
if (!metadataSegmentManager.removeDatasource(dataSourceName)) {
return Response.status(Response.Status.NOT_FOUND).build();
}
}
@ -506,7 +506,7 @@ public class InfoResource
@PathParam("dataSourceName") final String dataSourceName
)
{
if (!databaseSegmentManager.enableDatasource(dataSourceName)) {
if (!metadataSegmentManager.enableDatasource(dataSourceName)) {
return Response.status(Response.Status.NOT_FOUND).build();
}
@ -574,7 +574,7 @@ public class InfoResource
@PathParam("segmentId") String segmentId
)
{
if (!databaseSegmentManager.removeSegment(dataSourceName, segmentId)) {
if (!metadataSegmentManager.removeSegment(dataSourceName, segmentId)) {
return Response.status(Response.Status.NOT_FOUND).build();
}
@ -589,7 +589,7 @@ public class InfoResource
@PathParam("segmentId") String segmentId
)
{
if (!databaseSegmentManager.enableSegment(segmentId)) {
if (!metadataSegmentManager.enableSegment(segmentId)) {
return Response.status(Response.Status.NOT_FOUND).build();
}
@ -672,7 +672,7 @@ public class InfoResource
}
@GET
@Path("/db/datasources")
@Path("/metadata/datasources")
@Produces("application/json")
public Response getDatabaseDataSources(
@QueryParam("full") String full,
@ -681,15 +681,15 @@ public class InfoResource
{
Response.ResponseBuilder builder = Response.status(Response.Status.OK);
if (includeDisabled != null) {
return builder.entity(databaseSegmentManager.getAllDatasourceNames()).build();
return builder.entity(metadataSegmentManager.getAllDatasourceNames()).build();
}
if (full != null) {
return builder.entity(databaseSegmentManager.getInventory()).build();
return builder.entity(metadataSegmentManager.getInventory()).build();
}
List<String> dataSourceNames = Lists.newArrayList(
Iterables.transform(
databaseSegmentManager.getInventory(),
metadataSegmentManager.getInventory(),
new Function<DruidDataSource, String>()
{
@Override
@ -707,13 +707,13 @@ public class InfoResource
}
@GET
@Path("/db/datasources/{dataSourceName}")
@Path("/metadata/datasources/{dataSourceName}")
@Produces("application/json")
public Response getDatabaseSegmentDataSource(
@PathParam("dataSourceName") final String dataSourceName
)
{
DruidDataSource dataSource = databaseSegmentManager.getInventoryValue(dataSourceName);
DruidDataSource dataSource = metadataSegmentManager.getInventoryValue(dataSourceName);
if (dataSource == null) {
return Response.status(Response.Status.NOT_FOUND).build();
}
@ -722,14 +722,14 @@ public class InfoResource
}
@GET
@Path("/db/datasources/{dataSourceName}/segments")
@Path("/metadata/datasources/{dataSourceName}/segments")
@Produces("application/json")
public Response getDatabaseSegmentDataSourceSegments(
@PathParam("dataSourceName") String dataSourceName,
@QueryParam("full") String full
)
{
DruidDataSource dataSource = databaseSegmentManager.getInventoryValue(dataSourceName);
DruidDataSource dataSource = metadataSegmentManager.getInventoryValue(dataSourceName);
if (dataSource == null) {
return Response.status(Response.Status.NOT_FOUND).build();
}
@ -755,14 +755,14 @@ public class InfoResource
}
@GET
@Path("/db/datasources/{dataSourceName}/segments/{segmentId}")
@Path("/metadata/datasources/{dataSourceName}/segments/{segmentId}")
@Produces("application/json")
public Response getDatabaseSegmentDataSourceSegment(
@PathParam("dataSourceName") String dataSourceName,
@PathParam("segmentId") String segmentId
)
{
DruidDataSource dataSource = databaseSegmentManager.getInventoryValue(dataSourceName);
DruidDataSource dataSource = metadataSegmentManager.getInventoryValue(dataSourceName);
if (dataSource == null) {
return Response.status(Response.Status.NOT_FOUND).build();
}

View File

@ -24,10 +24,9 @@ import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.inject.Inject;
import io.druid.client.DruidDataSource;
import io.druid.db.DatabaseSegmentManager;
import io.druid.metadata.MetadataSegmentManager;
import io.druid.timeline.DataSegment;
import javax.annotation.Nullable;
import javax.ws.rs.GET;
import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
@ -39,17 +38,17 @@ import java.util.List;
/**
*/
@Path("/druid/coordinator/v1/db")
public class DBResource
@Path("/druid/coordinator/v1/metadata")
public class MetadataResource
{
private final DatabaseSegmentManager databaseSegmentManager;
private final MetadataSegmentManager metadataSegmentManager;
@Inject
public DBResource(
DatabaseSegmentManager databaseSegmentManager
public MetadataResource(
MetadataSegmentManager metadataSegmentManager
)
{
this.databaseSegmentManager = databaseSegmentManager;
this.metadataSegmentManager = metadataSegmentManager;
}
@GET
@ -62,15 +61,15 @@ public class DBResource
{
Response.ResponseBuilder builder = Response.status(Response.Status.OK);
if (includeDisabled != null) {
return builder.entity(databaseSegmentManager.getAllDatasourceNames()).build();
return builder.entity(metadataSegmentManager.getAllDatasourceNames()).build();
}
if (full != null) {
return builder.entity(databaseSegmentManager.getInventory()).build();
return builder.entity(metadataSegmentManager.getInventory()).build();
}
List<String> dataSourceNames = Lists.newArrayList(
Iterables.transform(
databaseSegmentManager.getInventory(),
metadataSegmentManager.getInventory(),
new Function<DruidDataSource, String>()
{
@Override
@ -94,7 +93,7 @@ public class DBResource
@PathParam("dataSourceName") final String dataSourceName
)
{
DruidDataSource dataSource = databaseSegmentManager.getInventoryValue(dataSourceName);
DruidDataSource dataSource = metadataSegmentManager.getInventoryValue(dataSourceName);
if (dataSource == null) {
return Response.status(Response.Status.NOT_FOUND).build();
}
@ -110,7 +109,7 @@ public class DBResource
@QueryParam("full") String full
)
{
DruidDataSource dataSource = databaseSegmentManager.getInventoryValue(dataSourceName);
DruidDataSource dataSource = metadataSegmentManager.getInventoryValue(dataSourceName);
if (dataSource == null) {
return Response.status(Response.Status.NOT_FOUND).build();
}
@ -143,7 +142,7 @@ public class DBResource
@PathParam("segmentId") String segmentId
)
{
DruidDataSource dataSource = databaseSegmentManager.getInventoryValue(dataSourceName);
DruidDataSource dataSource = metadataSegmentManager.getInventoryValue(dataSourceName);
if (dataSource == null) {
return Response.status(Response.Status.NOT_FOUND).build();
}

View File

@ -20,7 +20,7 @@
package io.druid.server.http;
import com.google.inject.Inject;
import io.druid.db.DatabaseRuleManager;
import io.druid.metadata.MetadataRuleManager;
import io.druid.server.coordinator.rules.Rule;
import javax.ws.rs.Consumes;
@ -38,11 +38,11 @@ import java.util.List;
@Path("/druid/coordinator/v1/rules")
public class RulesResource
{
private final DatabaseRuleManager databaseRuleManager;
private final MetadataRuleManager databaseRuleManager;
@Inject
public RulesResource(
DatabaseRuleManager databaseRuleManager
MetadataRuleManager databaseRuleManager
)
{
this.databaseRuleManager = databaseRuleManager;

View File

@ -11375,10 +11375,10 @@ app.factory('$druid', function($q, $http, $utils, $window) {
return this.getAndProcess("/servers?simple", $utils.processServerTiers);
},
getDataSources: function() {
return this.getAndProcess("/db/datasources", $utils.processDataSources);
return this.getAndProcess("/metadata/datasources", $utils.processDataSources);
},
getAllDataSources: function() {
return this.getAndProcess("/db/datasources?includeDisabled", function(dataSources) {
return this.getAndProcess("/metadata/datasources?includeDisabled", function(dataSources) {
return dataSources;
});
},
@ -12758,4 +12758,4 @@ ClusterConfigInstanceCtrl = function($scope, $modalInstance, $druid) {
};
},{"../../bower_components/angular-bootstrap/ui-bootstrap-tpls.min.js":1,"../../bower_components/angular-sanitize/angular-sanitize.min.js":2,"../../bower_components/angular-ui-router/release/angular-ui-router.min.js":3,"../../bower_components/angular/angular.min.js":4,"../../bower_components/d3/d3.js":5,"../../bower_components/moment/min/moment.min.js":7,"../../bower_components/ng-clip/dest/ng-clip.min.js":8,"../../bower_components/ng-csv/build/ng-csv.min.js":9,"../../bower_components/underscore/underscore.js":10,"../../bower_components/zeroclipboard/ZeroClipboard.min.js":11,"../../lib/moment-interval.js":12,"jquery":6}]},{},[13])
},{"../../bower_components/angular-bootstrap/ui-bootstrap-tpls.min.js":1,"../../bower_components/angular-sanitize/angular-sanitize.min.js":2,"../../bower_components/angular-ui-router/release/angular-ui-router.min.js":3,"../../bower_components/angular/angular.min.js":4,"../../bower_components/d3/d3.js":5,"../../bower_components/moment/min/moment.min.js":7,"../../bower_components/ng-clip/dest/ng-clip.min.js":8,"../../bower_components/ng-csv/build/ng-csv.min.js":9,"../../bower_components/underscore/underscore.js":10,"../../bower_components/zeroclipboard/ZeroClipboard.min.js":11,"../../lib/moment-interval.js":12,"jquery":6}]},{},[13])

View File

@ -70,12 +70,12 @@ $(document).ready(function() {
}
});
$.getJSON("/druid/coordinator/v1/db/datasources", function(enabled_datasources) {
$.getJSON("/druid/coordinator/v1/metadata/datasources", function(enabled_datasources) {
$.each(enabled_datasources, function(index, datasource) {
$('#enabled_datasources').append($('<li>' + datasource + '</li>'));
});
$.getJSON("/druid/coordinator/v1/db/datasources?includeDisabled", function(db_datasources) {
$.getJSON("/druid/coordinator/v1/metadata/datasources?includeDisabled", function(db_datasources) {
var disabled_datasources = _.difference(db_datasources, enabled_datasources);
$.each(disabled_datasources, function(index, datasource) {
$('#disabled_datasources').append($('<li>' + datasource + '</li>'));
@ -93,4 +93,4 @@ $(document).ready(function() {
$('#disable').click(function (){
$("#disable_dialog").dialog("open")
});
});
});

View File

@ -41,7 +41,7 @@ $(document).ready(function() {
}
});
$.getJSON("/druid/coordinator/v1/db/datasources?includeDisabled", function(data) {
$.getJSON("/druid/coordinator/v1/metadata/datasources?includeDisabled", function(data) {
$.each(data, function(index, datasource) {
$('#datasources').append($('<option></option>').val(datasource).text(datasource));
});
@ -50,4 +50,4 @@ $(document).ready(function() {
$("#confirm").click(function() {
$("#confirm_dialog").dialog("open");
});
});
});

View File

@ -268,7 +268,7 @@ $(document).ready(function() {
tiers = theTiers;
});
$.getJSON("/druid/coordinator/v1/db/datasources", function(data) {
$.getJSON("/druid/coordinator/v1/metadata/datasources", function(data) {
$.each(data, function(index, datasource) {
$('#datasources').append($('<option></option>').val(datasource).text(datasource));
});
@ -307,4 +307,4 @@ $(document).ready(function() {
$('#update').click(function (){
$("#update_dialog").dialog("open")
});
});
});

Some files were not shown because too many files have changed in this diff Show More