Merge pull request #97 from metamx/convert_task

Index Conversion Task and adjustment to TaskAction API
This commit is contained in:
cheddar 2013-03-07 16:13:00 -08:00
commit c3a871b942
88 changed files with 2144 additions and 1459 deletions

View File

@ -261,6 +261,7 @@ public class DataSegment implements Comparable<DataSegment>
", loadSpec=" + loadSpec +
", interval=" + interval +
", dataSource='" + dataSource + '\'' +
", binaryVersion='" + binaryVersion + '\'' +
'}';
}

View File

@ -17,7 +17,7 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.merge;
package com.metamx.druid.client.indexing;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
@ -29,7 +29,7 @@ import java.util.List;
/**
*/
public class ClientAppendQuery implements ClientMergeQuery
public class ClientAppendQuery
{
private final String dataSource;
private final List<DataSegment> segments;
@ -45,14 +45,18 @@ public class ClientAppendQuery implements ClientMergeQuery
}
@JsonProperty
@Override
public String getType()
{
return "append";
}
@JsonProperty
public String getDataSource()
{
return dataSource;
}
@JsonProperty
@Override
public List<DataSegment> getSegments()
{
return segments;

View File

@ -0,0 +1,57 @@
package com.metamx.druid.client.indexing;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.metamx.druid.client.DataSegment;
import org.joda.time.Interval;
/**
*/
public class ClientConversionQuery
{
private final String dataSource;
private final Interval interval;
private final DataSegment segment;
public ClientConversionQuery(
DataSegment segment
)
{
this.dataSource = segment.getDataSource();
this.interval = segment.getInterval();
this.segment = segment;
}
public ClientConversionQuery(
String dataSource,
Interval interval
)
{
this.dataSource = dataSource;
this.interval = interval;
this.segment = null;
}
@JsonProperty
public String getType()
{
return "version_converter";
}
@JsonProperty
public String getDataSource()
{
return dataSource;
}
@JsonProperty
public Interval getInterval()
{
return interval;
}
@JsonProperty
public DataSegment getSegment()
{
return segment;
}
}

View File

@ -17,21 +17,21 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.merge;
package com.metamx.druid.client.indexing;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.joda.time.Interval;
public class ClientDeleteQuery
/**
*/
public class ClientKillQuery
{
private final String dataSource;
private final Interval interval;
@JsonCreator
public ClientDeleteQuery(
public ClientKillQuery(
@JsonProperty("dataSource") String dataSource,
@JsonProperty("interval") Interval interval
)
@ -40,22 +40,21 @@ public class ClientDeleteQuery
this.interval = interval;
}
@JsonProperty
public String getType()
{
return "kill";
}
@JsonProperty
public String getDataSource()
{
return dataSource;
}
@JsonProperty
public Interval getInterval()
{
return interval;
}
@Override
public String toString()
{
return "ClientDeleteQuery{" +
"dataSource='" + dataSource + '\'' +
", interval=" + interval +
'}';
}
}

View File

@ -17,7 +17,7 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.merge;
package com.metamx.druid.client.indexing;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
@ -30,14 +30,14 @@ import java.util.List;
/**
*/
public class ClientDefaultMergeQuery implements ClientMergeQuery
public class ClientMergeQuery
{
private final String dataSource;
private final List<DataSegment> segments;
private final List<AggregatorFactory> aggregators;
@JsonCreator
public ClientDefaultMergeQuery(
public ClientMergeQuery(
@JsonProperty("dataSource") String dataSource,
@JsonProperty("segments") List<DataSegment> segments,
@JsonProperty("aggregations") List<AggregatorFactory> aggregators
@ -50,14 +50,18 @@ public class ClientDefaultMergeQuery implements ClientMergeQuery
}
@JsonProperty
@Override
public String getType()
{
return "append";
}
@JsonProperty
public String getDataSource()
{
return dataSource;
}
@JsonProperty
@Override
public List<DataSegment> getSegments()
{
return segments;
@ -72,7 +76,7 @@ public class ClientDefaultMergeQuery implements ClientMergeQuery
@Override
public String toString()
{
return "ClientDefaultMergeQuery{" +
return "ClientMergeQuery{" +
"dataSource='" + dataSource + '\'' +
", segments=" + segments +
", aggregators=" + aggregators +

View File

@ -0,0 +1,115 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.client.indexing;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.metamx.common.IAE;
import com.metamx.druid.client.DataSegment;
import com.metamx.http.client.HttpClient;
import com.metamx.http.client.response.InputStreamResponseHandler;
import com.netflix.curator.x.discovery.ServiceInstance;
import com.netflix.curator.x.discovery.ServiceProvider;
import org.joda.time.Interval;
import java.io.InputStream;
import java.net.URL;
import java.util.Iterator;
import java.util.List;
public class IndexingServiceClient
{
private static final InputStreamResponseHandler RESPONSE_HANDLER = new InputStreamResponseHandler();
private final HttpClient client;
private final ObjectMapper jsonMapper;
private final ServiceProvider serviceProvider;
public IndexingServiceClient(
HttpClient client,
ObjectMapper jsonMapper,
ServiceProvider serviceProvider
)
{
this.client = client;
this.jsonMapper = jsonMapper;
this.serviceProvider = serviceProvider;
}
public void mergeSegments(List<DataSegment> segments)
{
final Iterator<DataSegment> segmentsIter = segments.iterator();
if (!segmentsIter.hasNext()) {
return;
}
final String dataSource = segmentsIter.next().getDataSource();
while (segmentsIter.hasNext()) {
DataSegment next = segmentsIter.next();
if (!dataSource.equals(next.getDataSource())) {
throw new IAE("Cannot merge segments of different dataSources[%s] and [%s]", dataSource, next.getDataSource());
}
}
runQuery("merge", new ClientAppendQuery(dataSource, segments));
}
public void killSegments(String dataSource, Interval interval)
{
runQuery("index", new ClientKillQuery(dataSource, interval));
}
public void upgradeSegment(DataSegment dataSegment)
{
runQuery("task", new ClientConversionQuery(dataSegment));
}
public void upgradeSegments(String dataSource, Interval interval)
{
runQuery("task", new ClientConversionQuery(dataSource, interval));
}
private InputStream runQuery(String endpoint, Object queryObject)
{
try {
return client.post(new URL(String.format("%s/%s", baseUrl(), endpoint)))
.setContent("application/json", jsonMapper.writeValueAsBytes(queryObject))
.go(RESPONSE_HANDLER)
.get();
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
private String baseUrl()
{
try {
final ServiceInstance instance = serviceProvider.getInstance();
return String.format("http://%s:%s/mmx/merger/v1", instance.getAddress(), instance.getPort());
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
}

View File

@ -1,41 +0,0 @@
package com.metamx.druid.merge;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.joda.time.Interval;
/**
*/
public class ClientKillQuery
{
private final String dataSource;
private final Interval interval;
@JsonCreator
public ClientKillQuery(
@JsonProperty("dataSource") String dataSource,
@JsonProperty("interval") Interval interval
)
{
this.dataSource = dataSource;
this.interval = interval;
}
@JsonProperty
public String getType()
{
return "kill";
}
@JsonProperty
public String getDataSource()
{
return dataSource;
}
@JsonProperty
public Interval getInterval()
{
return interval;
}
}

View File

@ -0,0 +1,274 @@
package com.metamx.druid.config;
import com.google.common.base.Throwables;
import com.google.common.collect.Maps;
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 org.joda.time.Duration;
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.sql.ResultSet;
import java.sql.SQLException;
import java.util.Arrays;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicReference;
/**
*/
public class ConfigManager
{
private static final Logger log = new Logger(ConfigManager.class);
private final Object lock = new Object();
private boolean started = false;
private final IDBI dbi;
private final ConfigManagerConfig config;
private final ScheduledExecutorService exec;
private final ConcurrentMap<String, ConfigHolder> watchedConfigs;
private final String selectStatement;
private volatile ConfigManager.PollingCallable poller;
public ConfigManager(IDBI dbi, ConfigManagerConfig config)
{
this.dbi = dbi;
this.config = config;
this.exec = ScheduledExecutors.fixed(1, "config-manager-%s");
this.watchedConfigs = Maps.newConcurrentMap();
this.selectStatement = String.format("SELECT payload FROM %s WHERE name = :name", config.getConfigTable());
}
@LifecycleStart
public void start()
{
synchronized (lock) {
if (started) {
return;
}
poller = new PollingCallable();
ScheduledExecutors.scheduleWithFixedDelay(exec, new Duration(0), config.getPollDuration(), poller);
started = true;
}
}
@LifecycleStop
public void stop()
{
synchronized (lock) {
if (!started) {
return;
}
poller.stop();
poller = null;
started = false;
}
}
private void poll()
{
for (Map.Entry<String, ConfigHolder> entry : watchedConfigs.entrySet()) {
try {
if (entry.getValue().swapIfNew(lookup(entry.getKey()))) {
log.info("New value for key[%s] seen.", entry.getKey());
}
}
catch (Exception e) {
log.warn(e, "Exception when checking property[%s]", entry.getKey());
}
}
}
@SuppressWarnings("unchecked")
public <T> AtomicReference<T> watchConfig(final String key, final ConfigSerde<T> serde)
{
ConfigHolder<T> holder = watchedConfigs.get(key);
if (holder == null) {
try {
log.info("Creating watch for key[%s]", key);
holder = exec.submit(
new Callable<ConfigHolder<T>>()
{
@Override
@SuppressWarnings("unchecked")
public ConfigHolder<T> call() throws Exception
{
if (!started) {
watchedConfigs.put(key, new ConfigHolder<T>(null, serde));
} else {
try {
// Multiple of these callables can be submitted at the same time, but the callables themselves
// are executed serially, so double check that it hasn't already been populated.
if (!watchedConfigs.containsKey(key)) {
byte[] value = lookup(key);
ConfigHolder<T> holder = new ConfigHolder<T>(value, serde);
watchedConfigs.put(key, holder);
}
}
catch (Exception e) {
log.warn(e, "Failed loading config for key[%s]", key);
watchedConfigs.put(key, new ConfigHolder<T>(null, serde));
}
}
return watchedConfigs.get(key);
}
}
).get();
}
catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw Throwables.propagate(e);
}
catch (ExecutionException e) {
throw Throwables.propagate(e);
}
}
return holder.getReference();
}
public byte[] lookup(final String key)
{
return dbi.withHandle(
new HandleCallback<byte[]>()
{
@Override
public byte[] withHandle(Handle handle) throws Exception
{
return handle.createQuery(selectStatement)
.bind("name", key)
.map(
new ResultSetMapper<byte[]>()
{
@Override
public byte[] map(int index, ResultSet r, StatementContext ctx) throws SQLException
{
return r.getBytes("payload");
}
}
)
.first();
}
}
);
}
public <T> boolean set(final String key, final ConfigSerde<T> serde, final T obj)
{
if (obj == null) {
return false;
}
final byte[] newBytes = serde.serialize(obj);
try {
return exec.submit(
new Callable<Boolean>()
{
@Override
public Boolean call() throws Exception
{
dbi.withHandle(
new HandleCallback<Void>()
{
@Override
public Void withHandle(Handle handle) throws Exception
{
handle.createStatement(
"INSERT INTO %s (name, payload) VALUES (:name, :payload) ON DUPLICATE KEY UPDATE payload = :payload"
)
.bind("name", key)
.bind("payload", newBytes)
.execute();
return null;
}
}
);
final ConfigHolder configHolder = watchedConfigs.get(key);
if (configHolder != null) {
configHolder.swapIfNew(newBytes);
}
return true;
}
}
).get();
}
catch (Exception e) {
log.warn(e, "Failed to set[%s]", key);
return false;
}
}
private static class ConfigHolder<T>
{
private final AtomicReference<byte[]> rawBytes;
private final ConfigSerde<T> serde;
private final AtomicReference<T> reference;
ConfigHolder(
byte[] rawBytes,
ConfigSerde<T> serde
)
{
this.rawBytes = new AtomicReference<byte[]>(rawBytes);
this.serde = serde;
this.reference = new AtomicReference<T>(serde.deserialize(rawBytes));
}
public AtomicReference<T> getReference()
{
return reference;
}
public boolean swapIfNew(byte[] newBytes)
{
if (!Arrays.equals(newBytes, rawBytes.get())) {
reference.set(serde.deserialize(newBytes));
rawBytes.set(newBytes);
return true;
}
return false;
}
}
private class PollingCallable implements Callable<ScheduledExecutors.Signal>
{
private volatile boolean stop = false;
void stop()
{
stop = true;
}
@Override
public ScheduledExecutors.Signal call() throws Exception
{
if (stop) {
return ScheduledExecutors.Signal.STOP;
}
poll();
return ScheduledExecutors.Signal.REPEAT;
}
}
}

View File

@ -0,0 +1,18 @@
package com.metamx.druid.config;
import org.joda.time.Duration;
import org.skife.config.Config;
import org.skife.config.Default;
/**
*/
public abstract class ConfigManagerConfig
{
@Config("druid.indexer.configTable")
public abstract String getConfigTable();
@Config("druid.indexer.poll.duration")
@Default("PT1M")
public abstract Duration getPollDuration();
}

View File

@ -0,0 +1,9 @@
package com.metamx.druid.config;
/**
*/
public interface ConfigSerde<T>
{
public byte[] serialize(T obj);
public T deserialize(byte[] bytes);
}

View File

@ -0,0 +1,134 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.config;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicReference;
/**
*/
public class JacksonConfigManager
{
private final ConfigManager configManager;
private final ObjectMapper jsonMapper;
public JacksonConfigManager(
ConfigManager configManager,
ObjectMapper jsonMapper
)
{
this.configManager = configManager;
this.jsonMapper = jsonMapper;
}
public <T> AtomicReference<T> watch(String key, Class<? extends T> clazz)
{
return watch(key, clazz, null);
}
public <T> AtomicReference<T> watch(String key, Class<? extends T> clazz, T defaultVal)
{
return configManager.watchConfig(key, create(clazz, defaultVal));
}
public <T> AtomicReference<T> watch(String key, TypeReference<T> clazz)
{
return watch(key, clazz, null);
}
public <T> AtomicReference<T> watch(String key, TypeReference<T> clazz, T defaultVal)
{
return configManager.watchConfig(key, create(clazz, defaultVal));
}
public <T> boolean set(String key, T val)
{
return configManager.set(key, create(val.getClass(), null), val);
}
private <T> ConfigSerde<T> create(final Class<? extends T> clazz, final T defaultVal)
{
return new ConfigSerde<T>()
{
@Override
public byte[] serialize(T obj)
{
try {
return jsonMapper.writeValueAsBytes(obj);
}
catch (JsonProcessingException e) {
throw Throwables.propagate(e);
}
}
@Override
public T deserialize(byte[] bytes)
{
if (bytes == null) {
return defaultVal;
}
try {
return jsonMapper.readValue(bytes, clazz);
}
catch (IOException e) {
throw Throwables.propagate(e);
}
}
};
}
private <T> ConfigSerde<T> create(final TypeReference<? extends T> clazz, final T defaultVal)
{
return new ConfigSerde<T>()
{
@Override
public byte[] serialize(T obj)
{
try {
return jsonMapper.writeValueAsBytes(obj);
}
catch (JsonProcessingException e) {
throw Throwables.propagate(e);
}
}
@Override
public T deserialize(byte[] bytes)
{
if (bytes == null) {
return defaultVal;
}
try {
return jsonMapper.readValue(bytes, clazz);
}
catch (IOException e) {
throw Throwables.propagate(e);
}
}
};
}
}

View File

@ -65,7 +65,7 @@ public class DbConnector
dbi,
configTableName,
String.format(
"CREATE table %s (name VARCHAR(255) NOT NULL, payload LONGTEXT NOT NULL, PRIMARY KEY(name))",
"CREATE table %s (name VARCHAR(255) NOT NULL, payload BLOB NOT NULL, PRIMARY KEY(name))",
configTableName
)
);
@ -84,12 +84,7 @@ public class DbConnector
@Override
public Void withHandle(Handle handle) throws Exception
{
List<Map<String, Object>> table = handle.select(
String.format(
"SHOW tables LIKE '%s'",
tableName
)
);
List<Map<String, Object>> table = handle.select(String.format("SHOW tables LIKE '%s'", tableName));
if (table.isEmpty()) {
log.info("Creating table[%s]", tableName);

View File

@ -199,6 +199,19 @@ public class IndexIO
}
}
public static boolean convertSegment(File toConvert, File converted) throws IOException
{
final int version = getVersionFromDir(toConvert);
switch (version) {
case 8:
DefaultIndexIOHandler.convertV8toV9(toConvert, converted);
return true;
default:
return false;
}
}
public static interface IndexIOHandler
{
/**
@ -229,7 +242,7 @@ public class IndexIO
public void storeLatest(Index index, File file) throws IOException;
}
static class DefaultIndexIOHandler implements IndexIOHandler
public static class DefaultIndexIOHandler implements IndexIOHandler
{
private static final Logger log = new Logger(DefaultIndexIOHandler.class);
@Override

View File

@ -25,11 +25,12 @@ import com.metamx.druid.client.DataSegment;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.loading.MMappedQueryableIndexFactory;
import com.metamx.druid.loading.S3DataSegmentPuller;
import com.metamx.druid.loading.SegmentKiller;
import com.metamx.druid.loading.DataSegmentKiller;
import com.metamx.druid.loading.SegmentLoaderConfig;
import com.metamx.druid.loading.SegmentLoadingException;
import com.metamx.druid.loading.SingleSegmentLoader;
import com.metamx.druid.merger.common.actions.TaskActionClient;
import com.metamx.druid.merger.common.actions.TaskActionClientFactory;
import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.emitter.service.ServiceEmitter;
@ -45,29 +46,32 @@ import java.util.Map;
public class TaskToolbox
{
private final TaskConfig config;
private final TaskActionClient taskActionClient;
private final Task task;
private final TaskActionClientFactory taskActionClientFactory;
private final ServiceEmitter emitter;
private final RestS3Service s3Client;
private final DataSegmentPusher segmentPusher;
private final SegmentKiller segmentKiller;
private final DataSegmentKiller dataSegmentKiller;
private final ObjectMapper objectMapper;
public TaskToolbox(
TaskConfig config,
TaskActionClient taskActionClient,
Task task,
TaskActionClientFactory taskActionClientFactory,
ServiceEmitter emitter,
RestS3Service s3Client,
DataSegmentPusher segmentPusher,
SegmentKiller segmentKiller,
DataSegmentKiller dataSegmentKiller,
ObjectMapper objectMapper
)
{
this.config = config;
this.taskActionClient = taskActionClient;
this.task = task;
this.taskActionClientFactory = taskActionClientFactory;
this.emitter = emitter;
this.s3Client = s3Client;
this.segmentPusher = segmentPusher;
this.segmentKiller = segmentKiller;
this.dataSegmentKiller = dataSegmentKiller;
this.objectMapper = objectMapper;
}
@ -76,9 +80,9 @@ public class TaskToolbox
return config;
}
public TaskActionClient getTaskActionClient()
public TaskActionClient getTaskActionClientFactory()
{
return taskActionClient;
return taskActionClientFactory.create(task);
}
public ServiceEmitter getEmitter()
@ -91,9 +95,9 @@ public class TaskToolbox
return segmentPusher;
}
public SegmentKiller getSegmentKiller()
public DataSegmentKiller getDataSegmentKiller()
{
return segmentKiller;
return dataSegmentKiller;
}
public ObjectMapper getObjectMapper()
@ -101,7 +105,7 @@ public class TaskToolbox
return objectMapper;
}
public Map<DataSegment, File> getSegments(final Task task, List<DataSegment> segments)
public Map<DataSegment, File> getSegments(List<DataSegment> segments)
throws SegmentLoadingException
{
final SingleSegmentLoader loader = new SingleSegmentLoader(
@ -112,7 +116,7 @@ public class TaskToolbox
@Override
public File getCacheDirectory()
{
return new File(config.getTaskDir(task), "fetched_segments");
return new File(getTaskDir(), "fetched_segments");
}
}
);
@ -124,4 +128,9 @@ public class TaskToolbox
return retVal;
}
public File getTaskDir() {
return new File(config.getBaseTaskDir(), task.getId());
}
}

View File

@ -0,0 +1,81 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.merger.common;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.loading.DataSegmentKiller;
import com.metamx.druid.merger.common.actions.TaskActionClientFactory;
import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.emitter.service.ServiceEmitter;
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
/**
* Stuff that may be needed by a Task in order to conduct its business.
*/
public class TaskToolboxFactory
{
private final TaskConfig config;
private final TaskActionClientFactory taskActionClientFactory;
private final ServiceEmitter emitter;
private final RestS3Service s3Client;
private final DataSegmentPusher segmentPusher;
private final DataSegmentKiller dataSegmentKiller;
private final ObjectMapper objectMapper;
public TaskToolboxFactory(
TaskConfig config,
TaskActionClientFactory taskActionClientFactory,
ServiceEmitter emitter,
RestS3Service s3Client,
DataSegmentPusher segmentPusher,
DataSegmentKiller dataSegmentKiller,
ObjectMapper objectMapper
)
{
this.config = config;
this.taskActionClientFactory = taskActionClientFactory;
this.emitter = emitter;
this.s3Client = s3Client;
this.segmentPusher = segmentPusher;
this.dataSegmentKiller = dataSegmentKiller;
this.objectMapper = objectMapper;
}
public ObjectMapper getObjectMapper()
{
return objectMapper;
}
public TaskToolbox build(Task task)
{
return new TaskToolbox(
config,
task,
taskActionClientFactory,
emitter,
s3Client,
segmentPusher,
dataSegmentKiller,
objectMapper
);
}
}

View File

@ -1,17 +1,20 @@
package com.metamx.druid.merger.common.actions;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.TaskStorage;
import com.metamx.emitter.EmittingLogger;
public class LocalTaskActionClient implements TaskActionClient
{
private final Task task;
private final TaskStorage storage;
private final TaskActionToolbox toolbox;
private static final EmittingLogger log = new EmittingLogger(LocalTaskActionClient.class);
public LocalTaskActionClient(TaskStorage storage, TaskActionToolbox toolbox)
public LocalTaskActionClient(Task task, TaskStorage storage, TaskActionToolbox toolbox)
{
this.task = task;
this.storage = storage;
this.toolbox = toolbox;
}
@ -19,15 +22,15 @@ public class LocalTaskActionClient implements TaskActionClient
@Override
public <RetType> RetType submit(TaskAction<RetType> taskAction)
{
final RetType ret = taskAction.perform(toolbox);
final RetType ret = taskAction.perform(task, toolbox);
// Add audit log
try {
storage.addAuditLog(taskAction);
storage.addAuditLog(task, taskAction);
}
catch (Exception e) {
log.makeAlert(e, "Failed to record action in audit log")
.addData("task", taskAction.getTask().getId())
.addData("task", task.getId())
.addData("actionClass", taskAction.getClass().getName())
.emit();
}

View File

@ -17,25 +17,27 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.merge;
package com.metamx.druid.merger.common.actions;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.metamx.druid.client.DataSegment;
import java.util.List;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.TaskStorage;
/**
*/
@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type", defaultImpl = ClientDefaultMergeQuery.class)
@JsonSubTypes(value={
@JsonSubTypes.Type(name="append", value=ClientAppendQuery.class)
})
public interface ClientMergeQuery
public class LocalTaskActionClientFactory implements TaskActionClientFactory
{
public String getDataSource();
private final TaskStorage storage;
private final TaskActionToolbox toolbox;
public List<DataSegment> getSegments();
public LocalTaskActionClientFactory(TaskStorage storage, TaskActionToolbox toolbox)
{
this.storage = storage;
this.toolbox = toolbox;
}
@Override
public TaskActionClient create(Task task)
{
return new LocalTaskActionClient(task, storage, toolbox);
}
}

View File

@ -11,25 +11,16 @@ import org.joda.time.Interval;
public class LockAcquireAction implements TaskAction<Optional<TaskLock>>
{
private final Task task;
private final Interval interval;
@JsonCreator
public LockAcquireAction(
@JsonProperty("task") Task task,
@JsonProperty("interval") Interval interval
)
{
this.task = task;
this.interval = interval;
}
@JsonProperty
public Task getTask()
{
return task;
}
@JsonProperty
public Interval getInterval()
{
@ -42,7 +33,7 @@ public class LockAcquireAction implements TaskAction<Optional<TaskLock>>
}
@Override
public Optional<TaskLock> perform(TaskActionToolbox toolbox)
public Optional<TaskLock> perform(Task task, TaskActionToolbox toolbox)
{
try {
return toolbox.getTaskLockbox().tryLock(task, interval);

View File

@ -12,29 +12,13 @@ import java.util.List;
public class LockListAction implements TaskAction<List<TaskLock>>
{
private final Task task;
@JsonCreator
public LockListAction(
@JsonProperty("task") Task task
)
{
this.task = task;
}
@JsonProperty
public Task getTask()
{
return task;
}
public TypeReference<List<TaskLock>> getReturnTypeReference()
{
return new TypeReference<List<TaskLock>>() {};
}
@Override
public List<TaskLock> perform(TaskActionToolbox toolbox)
public List<TaskLock> perform(Task task, TaskActionToolbox toolbox)
{
try {
return toolbox.getTaskLockbox().findLocksForTask(task);

View File

@ -12,25 +12,16 @@ import java.util.List;
public class LockReleaseAction implements TaskAction<Void>
{
private final Task task;
private final Interval interval;
@JsonCreator
public LockReleaseAction(
@JsonProperty("task") Task task,
@JsonProperty("interval") Interval interval
)
{
this.task = task;
this.interval = interval;
}
@JsonProperty
public Task getTask()
{
return task;
}
@JsonProperty
public Interval getInterval()
{
@ -43,7 +34,7 @@ public class LockReleaseAction implements TaskAction<Void>
}
@Override
public Void perform(TaskActionToolbox toolbox)
public Void perform(Task task, TaskActionToolbox toolbox)
{
try {
toolbox.getTaskLockbox().unlock(task, interval);

View File

@ -4,6 +4,7 @@ import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.base.Charsets;
import com.google.common.base.Throwables;
import com.metamx.common.logger.Logger;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.http.client.HttpClient;
import com.metamx.http.client.response.ToStringResponseHandler;
import com.fasterxml.jackson.databind.ObjectMapper;
@ -16,14 +17,16 @@ import java.util.Map;
public class RemoteTaskActionClient implements TaskActionClient
{
private final Task task;
private final HttpClient httpClient;
private final ServiceProvider serviceProvider;
private final ObjectMapper jsonMapper;
private static final Logger log = new Logger(RemoteTaskActionClient.class);
public RemoteTaskActionClient(HttpClient httpClient, ServiceProvider serviceProvider, ObjectMapper jsonMapper)
public RemoteTaskActionClient(Task task, HttpClient httpClient, ServiceProvider serviceProvider, ObjectMapper jsonMapper)
{
this.task = task;
this.httpClient = httpClient;
this.serviceProvider = serviceProvider;
this.jsonMapper = jsonMapper;
@ -33,7 +36,7 @@ public class RemoteTaskActionClient implements TaskActionClient
public <RetType> RetType submit(TaskAction<RetType> taskAction)
{
try {
byte[] dataToSend = jsonMapper.writeValueAsBytes(taskAction);
byte[] dataToSend = jsonMapper.writeValueAsBytes(new TaskActionHolder(task, taskAction));
final String response = httpClient.post(getServiceUri().toURL())
.setContent("application/json", dataToSend)

View File

@ -0,0 +1,47 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.merger.common.actions;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.http.client.HttpClient;
import com.netflix.curator.x.discovery.ServiceProvider;
/**
*/
public class RemoteTaskActionClientFactory implements TaskActionClientFactory
{
private final HttpClient httpClient;
private final ServiceProvider serviceProvider;
private final ObjectMapper jsonMapper;
public RemoteTaskActionClientFactory(HttpClient httpClient, ServiceProvider serviceProvider, ObjectMapper jsonMapper)
{
this.httpClient = httpClient;
this.serviceProvider = serviceProvider;
this.jsonMapper = jsonMapper;
}
@Override
public TaskActionClient create(Task task)
{
return new RemoteTaskActionClient(task, httpClient, serviceProvider, jsonMapper);
}
}

View File

@ -18,25 +18,16 @@ import java.util.Set;
public class SegmentInsertAction implements TaskAction<Void>
{
private final Task task;
private final Set<DataSegment> segments;
@JsonCreator
public SegmentInsertAction(
@JsonProperty("task") Task task,
@JsonProperty("segments") Set<DataSegment> segments
)
{
this.task = task;
this.segments = ImmutableSet.copyOf(segments);
}
@JsonProperty
public Task getTask()
{
return task;
}
@JsonProperty
public Set<DataSegment> getSegments()
{
@ -49,7 +40,7 @@ public class SegmentInsertAction implements TaskAction<Void>
}
@Override
public Void perform(TaskActionToolbox toolbox)
public Void perform(Task task, TaskActionToolbox toolbox)
{
if(!toolbox.taskLockCoversSegments(task, segments, false)) {
throw new ISE("Segments not covered by locks for task: %s", task.getId());

View File

@ -12,28 +12,19 @@ import java.util.List;
public class SegmentListUnusedAction implements TaskAction<List<DataSegment>>
{
private final Task task;
private final String dataSource;
private final Interval interval;
@JsonCreator
public SegmentListUnusedAction(
@JsonProperty("task") Task task,
@JsonProperty("dataSource") String dataSource,
@JsonProperty("interval") Interval interval
)
{
this.task = task;
this.dataSource = dataSource;
this.interval = interval;
}
@JsonProperty
public Task getTask()
{
return task;
}
@JsonProperty
public String getDataSource()
{
@ -52,7 +43,7 @@ public class SegmentListUnusedAction implements TaskAction<List<DataSegment>>
}
@Override
public List<DataSegment> perform(TaskActionToolbox toolbox)
public List<DataSegment> perform(Task task, TaskActionToolbox toolbox)
{
try {
return toolbox.getMergerDBCoordinator().getUnusedSegmentsForInterval(dataSource, interval);

View File

@ -12,28 +12,19 @@ import java.util.List;
public class SegmentListUsedAction implements TaskAction<List<DataSegment>>
{
private final Task task;
private final String dataSource;
private final Interval interval;
@JsonCreator
public SegmentListUsedAction(
@JsonProperty("task") Task task,
@JsonProperty("dataSource") String dataSource,
@JsonProperty("interval") Interval interval
)
{
this.task = task;
this.dataSource = dataSource;
this.interval = interval;
}
@JsonProperty
public Task getTask()
{
return task;
}
@JsonProperty
public String getDataSource()
{
@ -52,7 +43,7 @@ public class SegmentListUsedAction implements TaskAction<List<DataSegment>>
}
@Override
public List<DataSegment> perform(TaskActionToolbox toolbox)
public List<DataSegment> perform(Task task, TaskActionToolbox toolbox)
{
try {
return toolbox.getMergerDBCoordinator().getUsedSegmentsForInterval(dataSource, interval);

View File

@ -18,25 +18,16 @@ import java.util.Set;
public class SegmentNukeAction implements TaskAction<Void>
{
private final Task task;
private final Set<DataSegment> segments;
@JsonCreator
public SegmentNukeAction(
@JsonProperty("task") Task task,
@JsonProperty("segments") Set<DataSegment> segments
)
{
this.task = task;
this.segments = ImmutableSet.copyOf(segments);
}
@JsonProperty
public Task getTask()
{
return task;
}
@JsonProperty
public Set<DataSegment> getSegments()
{
@ -49,7 +40,7 @@ public class SegmentNukeAction implements TaskAction<Void>
}
@Override
public Void perform(TaskActionToolbox toolbox)
public Void perform(Task task, TaskActionToolbox toolbox)
{
if(!toolbox.taskLockCoversSegments(task, segments, true)) {
throw new ISE("Segments not covered by locks for task: %s", task.getId());

View File

@ -11,25 +11,16 @@ import java.util.List;
public class SpawnTasksAction implements TaskAction<Void>
{
private final Task task;
private final List<Task> newTasks;
@JsonCreator
public SpawnTasksAction(
@JsonProperty("task") Task task,
@JsonProperty("newTasks") List<Task> newTasks
)
{
this.task = task;
this.newTasks = ImmutableList.copyOf(newTasks);
}
@JsonProperty
public Task getTask()
{
return task;
}
@JsonProperty
public List<Task> getNewTasks()
{
@ -42,7 +33,7 @@ public class SpawnTasksAction implements TaskAction<Void>
}
@Override
public Void perform(TaskActionToolbox toolbox)
public Void perform(Task task, TaskActionToolbox toolbox)
{
try {
for(final Task newTask : newTasks) {

View File

@ -18,7 +18,6 @@ import com.fasterxml.jackson.core.type.TypeReference;
})
public interface TaskAction<RetType>
{
public Task getTask(); // TODO Look into replacing this with task ID so stuff serializes smaller
public TypeReference<RetType> getReturnTypeReference(); // T_T
public RetType perform(TaskActionToolbox toolbox);
public RetType perform(Task task, TaskActionToolbox toolbox);
}

View File

@ -17,23 +17,13 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.merger.coordinator.config;
package com.metamx.druid.merger.common.actions;
import org.joda.time.Duration;
import org.skife.config.Config;
import org.skife.config.Default;
import com.metamx.druid.merger.common.task.Task;
/**
*/
public abstract class WorkerSetupManagerConfig
public interface TaskActionClientFactory
{
@Config("druid.indexer.configTable")
public abstract String getConfigTable();
@Config("druid.indexer.workerSetupConfigName")
public abstract String getWorkerSetupConfigName();
@Config("druid.indexer.poll.duration")
@Default("PT1M")
public abstract Duration getPollDuration();
public TaskActionClient create(Task task);
}

View File

@ -0,0 +1,54 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.merger.common.actions;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.metamx.druid.merger.common.task.Task;
/**
*/
public class TaskActionHolder<T>
{
private final Task task;
private final TaskAction<T> action;
@JsonCreator
public TaskActionHolder(
@JsonProperty("task") Task task,
@JsonProperty("action") TaskAction action
)
{
this.task = task;
this.action = action;
}
@JsonProperty
public Task getTask()
{
return task;
}
@JsonProperty
public TaskAction<T> getAction()
{
return action;
}
}

View File

@ -1,3 +1,22 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.merger.common.config;
import com.metamx.druid.merger.common.task.Task;
@ -17,8 +36,4 @@ public abstract class TaskConfig
@Config("druid.merger.hadoopWorkingPath")
public abstract String getHadoopWorkingPath();
public File getTaskDir(final Task task) {
return new File(getBaseTaskDir(), task.getId());
}
}

View File

@ -20,15 +20,19 @@
package com.metamx.druid.merger.common.task;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Joiner;
import com.google.common.base.Objects;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.actions.SegmentListUsedAction;
import org.joda.time.Interval;
public abstract class AbstractTask implements Task
{
private static final Joiner ID_JOINER = Joiner.on("_");
private final String id;
private final String groupId;
private final String dataSource;
@ -91,4 +95,20 @@ public abstract class AbstractTask implements Task
.add("interval", getImplicitLockInterval())
.toString();
}
/** Start helper methods **/
public static String joinId(Object... objects)
{
return ID_JOINER.join(objects);
}
public SegmentListUsedAction defaultListUsedAction()
{
return new SegmentListUsedAction(getDataSource(), getImplicitLockInterval().get());
}
public TaskStatus success()
{
return TaskStatus.success(getId());
}
}

View File

@ -44,7 +44,7 @@ import java.util.Map;
/**
*/
public class AppendTask extends MergeTask
public class AppendTask extends MergeTaskBase
{
@JsonCreator
public AppendTask(

View File

@ -1,87 +0,0 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.merger.common.task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.metamx.druid.aggregation.AggregatorFactory;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.index.QueryableIndex;
import com.metamx.druid.index.v1.IndexIO;
import com.metamx.druid.index.v1.IndexMerger;
import javax.annotation.Nullable;
import java.io.File;
import java.util.List;
import java.util.Map;
/**
*/
public class DefaultMergeTask extends MergeTask
{
private final List<AggregatorFactory> aggregators;
@JsonCreator
public DefaultMergeTask(
@JsonProperty("dataSource") String dataSource,
@JsonProperty("segments") List<DataSegment> segments,
@JsonProperty("aggregations") List<AggregatorFactory> aggregators
)
{
super(dataSource, segments);
this.aggregators = aggregators;
}
@Override
public File merge(final Map<DataSegment, File> segments, final File outDir)
throws Exception
{
return IndexMerger.mergeQueryableIndex(
Lists.transform(
ImmutableList.copyOf(segments.values()),
new Function<File, QueryableIndex>()
{
@Override
public QueryableIndex apply(@Nullable File input)
{
try {
return IndexIO.loadIndex(input);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
}
),
aggregators.toArray(new AggregatorFactory[aggregators.size()]),
outDir
);
}
@Override
public String getType()
{
return "merge";
}
}

View File

@ -77,7 +77,7 @@ public class DeleteTask extends AbstractTask
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
// Strategy: Create an empty segment covering the interval to be deleted
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this)));
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction()));
final Interval interval = this.getImplicitLockInterval().get();
final IncrementalIndex empty = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]);
final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(interval, empty);
@ -91,7 +91,7 @@ public class DeleteTask extends AbstractTask
.shardSpec(new NoneShardSpec())
.build();
final File outDir = new File(toolbox.getConfig().getTaskDir(this), segment.getIdentifier());
final File outDir = new File(toolbox.getTaskDir(), segment.getIdentifier());
final File fileToUpload = IndexMerger.merge(Lists.newArrayList(emptyAdapter), new AggregatorFactory[0], outDir);
// Upload the segment
@ -104,7 +104,7 @@ public class DeleteTask extends AbstractTask
segment.getVersion()
);
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(uploadedSegment)));
toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment)));
return TaskStatus.success(getId());
}

View File

@ -1,3 +1,22 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.merger.common.task;
import com.fasterxml.jackson.annotation.JsonCreator;
@ -74,7 +93,7 @@ public class HadoopIndexTask extends AbstractTask
);
// We should have a lock from before we started running
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this)));
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction()));
log.info("Setting version to: %s", myLock.getVersion());
configCopy.setVersion(myLock.getVersion());
@ -105,7 +124,7 @@ public class HadoopIndexTask extends AbstractTask
List<DataSegment> publishedSegments = job.getPublishedSegments();
// Request segment pushes
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.copyOf(publishedSegments)));
toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.copyOf(publishedSegments)));
// Done
return TaskStatus.success(getId());

View File

@ -258,7 +258,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask
}
);
toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, nextTasks));
toolbox.getTaskActionClientFactory().submit(new SpawnTasksAction(nextTasks));
return TaskStatus.success(getId());
}

View File

@ -100,24 +100,21 @@ public class IndexGeneratorTask extends AbstractTask
public TaskStatus run(final TaskToolbox toolbox) throws Exception
{
// We should have a lock from before we started running
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this)));
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction()));
// We know this exists
final Interval interval = getImplicitLockInterval().get();
// Set up temporary directory for indexing
final File tmpDir = new File(
toolbox.getTaskDir(),
String.format(
"%s/%s",
toolbox.getConfig().getTaskDir(this).toString(),
String.format(
"%s_%s_%s_%s_%s",
this.getDataSource(),
interval.getStart(),
interval.getEnd(),
myLock.getVersion(),
schema.getShardSpec().getPartitionNum()
)
"%s_%s_%s_%s_%s",
this.getDataSource(),
interval.getStart(),
interval.getEnd(),
myLock.getVersion(),
schema.getShardSpec().getPartitionNum()
)
);
@ -193,7 +190,7 @@ public class IndexGeneratorTask extends AbstractTask
);
// Request segment pushes
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.copyOf(pushedSegments)));
toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.copyOf(pushedSegments)));
// Done
return TaskStatus.success(getId());

View File

@ -142,7 +142,7 @@ public class IndexTask extends AbstractTask
@Override
public TaskStatus preflight(TaskToolbox toolbox) throws Exception
{
toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, toSubtasks()));
toolbox.getTaskActionClientFactory().submit(new SpawnTasksAction(toSubtasks()));
return TaskStatus.success(getId());
}

View File

@ -1,3 +1,22 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.merger.common.task;
import com.fasterxml.jackson.annotation.JsonCreator;
@ -53,7 +72,7 @@ public class KillTask extends AbstractTask
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
// Confirm we have a lock (will throw if there isn't exactly one element)
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this)));
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction()));
if(!myLock.getDataSource().equals(getDataSource())) {
throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource());
@ -64,14 +83,9 @@ public class KillTask extends AbstractTask
}
// List unused segments
final List<DataSegment> unusedSegments = toolbox.getTaskActionClient()
.submit(
new SegmentListUnusedAction(
this,
myLock.getDataSource(),
myLock.getInterval()
)
);
final List<DataSegment> unusedSegments = toolbox
.getTaskActionClientFactory()
.submit(new SegmentListUnusedAction(myLock.getDataSource(), myLock.getInterval()));
// Verify none of these segments have versions > lock version
for(final DataSegment unusedSegment : unusedSegments) {
@ -88,10 +102,12 @@ public class KillTask extends AbstractTask
}
// Kill segments
toolbox.getSegmentKiller().kill(unusedSegments);
for (DataSegment segment : unusedSegments) {
toolbox.getDataSegmentKiller().kill(segment);
}
// Remove metadata for these segments
toolbox.getTaskActionClient().submit(new SegmentNukeAction(this, ImmutableSet.copyOf(unusedSegments)));
toolbox.getTaskActionClientFactory().submit(new SegmentNukeAction(ImmutableSet.copyOf(unusedSegments)));
return TaskStatus.success(getId());
}

View File

@ -19,318 +19,71 @@
package com.metamx.druid.merger.common.task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.google.common.base.Charsets;
import com.google.common.base.Function;
import com.google.common.base.Joiner;
import com.google.common.base.Objects;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.collect.Ordering;
import com.google.common.collect.Sets;
import com.google.common.hash.Hashing;
import com.metamx.common.ISE;
import com.metamx.common.logger.Logger;
import com.metamx.druid.aggregation.AggregatorFactory;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.actions.LockListAction;
import com.metamx.druid.merger.common.actions.SegmentInsertAction;
import com.metamx.druid.merger.common.actions.SegmentListUsedAction;
import com.metamx.druid.shard.NoneShardSpec;
import com.metamx.emitter.service.AlertEvent;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import com.metamx.druid.index.QueryableIndex;
import com.metamx.druid.index.v1.IndexIO;
import com.metamx.druid.index.v1.IndexMerger;
import javax.annotation.Nullable;
import java.io.File;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultMergeTask.class)
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "append", value = AppendTask.class)
})
public abstract class MergeTask extends AbstractTask
public class MergeTask extends MergeTaskBase
{
private final List<DataSegment> segments;
private final List<AggregatorFactory> aggregators;
private static final Logger log = new Logger(MergeTask.class);
protected MergeTask(final String dataSource, final List<DataSegment> segments)
{
super(
// _not_ the version, just something uniqueish
String.format("merge_%s_%s", computeProcessingID(dataSource, segments), new DateTime().toString()),
dataSource,
computeMergedInterval(segments)
);
// Verify segment list is nonempty
Preconditions.checkArgument(segments.size() > 0, "segments nonempty");
// Verify segments are all in the correct datasource
Preconditions.checkArgument(
Iterables.size(
Iterables.filter(
segments,
new Predicate<DataSegment>()
{
@Override
public boolean apply(@Nullable DataSegment segment)
{
return segment == null || !segment.getDataSource().equalsIgnoreCase(dataSource);
}
}
)
) == 0, "segments in the wrong datasource"
);
// Verify segments are all unsharded
Preconditions.checkArgument(
Iterables.size(
Iterables.filter(
segments,
new Predicate<DataSegment>()
{
@Override
public boolean apply(@Nullable DataSegment segment)
{
return segment == null || !(segment.getShardSpec() instanceof NoneShardSpec);
}
}
)
) == 0, "segments without NoneShardSpec"
);
this.segments = segments;
}
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this)));
final ServiceEmitter emitter = toolbox.getEmitter();
final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder();
final DataSegment mergedSegment = computeMergedSegment(getDataSource(), myLock.getVersion(), segments);
final File taskDir = toolbox.getConfig().getTaskDir(this);
try {
final long startTime = System.currentTimeMillis();
log.info(
"Starting merge of id[%s], segments: %s",
getId(),
Lists.transform(
segments,
new Function<DataSegment, String>()
{
@Override
public String apply(@Nullable DataSegment input)
{
return input.getIdentifier();
}
}
)
);
// download segments to merge
final Map<DataSegment, File> gettedSegments = toolbox.getSegments(this, segments);
// merge files together
final File fileToUpload = merge(gettedSegments, new File(taskDir, "merged"));
emitter.emit(builder.build("merger/numMerged", segments.size()));
emitter.emit(builder.build("merger/mergeTime", System.currentTimeMillis() - startTime));
log.info(
"[%s] : Merged %d segments in %,d millis",
mergedSegment.getDataSource(),
segments.size(),
System.currentTimeMillis() - startTime
);
long uploadStart = System.currentTimeMillis();
// Upload file
final DataSegment uploadedSegment = toolbox.getSegmentPusher().push(fileToUpload, mergedSegment);
emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart));
emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize()));
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(uploadedSegment)));
return TaskStatus.success(getId());
}
catch (Exception e) {
log.error(
e,
String.format(
"Exception merging %s[%s] segments",
mergedSegment.getDataSource(),
mergedSegment.getInterval()
)
);
emitter.emit(
new AlertEvent.Builder().build(
"Exception merging",
ImmutableMap.<String, Object>builder()
.put("exception", e.toString())
.build()
)
);
return TaskStatus.failure(getId());
}
}
/**
* Checks pre-existing segments in "context" to confirm that this merge query is valid. Specifically, confirm that
* we are operating on every segment that overlaps the chosen interval.
*/
@Override
public TaskStatus preflight(TaskToolbox toolbox)
{
final Function<DataSegment, String> toIdentifier = new Function<DataSegment, String>()
{
@Override
public String apply(DataSegment dataSegment)
{
return dataSegment.getIdentifier();
}
};
final Set<String> current = ImmutableSet.copyOf(
Iterables.transform(
toolbox.getTaskActionClient()
.submit(new SegmentListUsedAction(this, getDataSource(), getImplicitLockInterval().get())),
toIdentifier
)
);
final Set<String> requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier));
final Set<String> missingFromRequested = Sets.difference(current, requested);
if (!missingFromRequested.isEmpty()) {
throw new ISE(
"Merge is invalid: current segment(s) are not in the requested set: %s",
Joiner.on(", ").join(missingFromRequested)
);
}
final Set<String> missingFromCurrent = Sets.difference(requested, current);
if (!missingFromCurrent.isEmpty()) {
throw new ISE(
"Merge is invalid: requested segment(s) are not in the current set: %s",
Joiner.on(", ").join(missingFromCurrent)
);
}
return TaskStatus.running(getId());
}
protected abstract File merge(Map<DataSegment, File> segments, File outDir)
throws Exception;
@JsonProperty
public List<DataSegment> getSegments()
{
return segments;
}
@Override
public String toString()
{
return Objects.toStringHelper(this)
.add("id", getId())
.add("dataSource", getDataSource())
.add("interval", getImplicitLockInterval())
.add("segments", segments)
.toString();
}
private static String computeProcessingID(final String dataSource, final List<DataSegment> segments)
{
final String segmentIDs = Joiner.on("_").join(
Iterables.transform(
Ordering.natural().sortedCopy(segments), new Function<DataSegment, String>()
{
@Override
public String apply(DataSegment x)
{
return String.format(
"%s_%s_%s_%s",
x.getInterval().getStart(),
x.getInterval().getEnd(),
x.getVersion(),
x.getShardSpec().getPartitionNum()
);
}
}
)
);
return String.format(
"%s_%s",
dataSource,
Hashing.sha1().hashString(segmentIDs, Charsets.UTF_8).toString().toLowerCase()
);
}
private static Interval computeMergedInterval(final List<DataSegment> segments)
{
Preconditions.checkArgument(segments.size() > 0, "segments.size() > 0");
DateTime start = null;
DateTime end = null;
for(final DataSegment segment : segments) {
if(start == null || segment.getInterval().getStart().isBefore(start)) {
start = segment.getInterval().getStart();
}
if(end == null || segment.getInterval().getEnd().isAfter(end)) {
end = segment.getInterval().getEnd();
}
}
return new Interval(start, end);
}
private static DataSegment computeMergedSegment(
final String dataSource,
final String version,
final List<DataSegment> segments
@JsonCreator
public MergeTask(
@JsonProperty("dataSource") String dataSource,
@JsonProperty("segments") List<DataSegment> segments,
@JsonProperty("aggregations") List<AggregatorFactory> aggregators
)
{
final Interval mergedInterval = computeMergedInterval(segments);
final Set<String> mergedDimensions = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER);
final Set<String> mergedMetrics = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER);
super(dataSource, segments);
this.aggregators = aggregators;
}
for (DataSegment segment : segments) {
mergedDimensions.addAll(segment.getDimensions());
mergedMetrics.addAll(segment.getMetrics());
}
@Override
public File merge(final Map<DataSegment, File> segments, final File outDir)
throws Exception
{
return IndexMerger.mergeQueryableIndex(
Lists.transform(
ImmutableList.copyOf(segments.values()),
new Function<File, QueryableIndex>()
{
@Override
public QueryableIndex apply(@Nullable File input)
{
try {
return IndexIO.loadIndex(input);
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
}
),
aggregators.toArray(new AggregatorFactory[aggregators.size()]),
outDir
);
}
return DataSegment.builder()
.dataSource(dataSource)
.interval(mergedInterval)
.version(version)
.shardSpec(new NoneShardSpec())
.dimensions(Lists.newArrayList(mergedDimensions))
.metrics(Lists.newArrayList(mergedMetrics))
.build();
@Override
public String getType()
{
return "merge";
}
}

View File

@ -0,0 +1,315 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.merger.common.task;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.google.common.base.Charsets;
import com.google.common.base.Function;
import com.google.common.base.Joiner;
import com.google.common.base.Objects;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Ordering;
import com.google.common.collect.Sets;
import com.google.common.hash.Hashing;
import com.metamx.common.ISE;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.actions.LockListAction;
import com.metamx.druid.merger.common.actions.SegmentInsertAction;
import com.metamx.druid.shard.NoneShardSpec;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.AlertEvent;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.io.File;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
*/
public abstract class MergeTaskBase extends AbstractTask
{
private final List<DataSegment> segments;
private static final EmittingLogger log = new EmittingLogger(MergeTaskBase.class);
protected MergeTaskBase(final String dataSource, final List<DataSegment> segments)
{
super(
// _not_ the version, just something uniqueish
String.format("merge_%s_%s", computeProcessingID(dataSource, segments), new DateTime().toString()),
dataSource,
computeMergedInterval(segments)
);
// Verify segment list is nonempty
Preconditions.checkArgument(segments.size() > 0, "segments nonempty");
// Verify segments are all in the correct datasource
Preconditions.checkArgument(
Iterables.size(
Iterables.filter(
segments,
new Predicate<DataSegment>()
{
@Override
public boolean apply(@Nullable DataSegment segment)
{
return segment == null || !segment.getDataSource().equalsIgnoreCase(dataSource);
}
}
)
) == 0, "segments in the wrong datasource"
);
// Verify segments are all unsharded
Preconditions.checkArgument(
Iterables.size(
Iterables.filter(
segments,
new Predicate<DataSegment>()
{
@Override
public boolean apply(@Nullable DataSegment segment)
{
return segment == null || !(segment.getShardSpec() instanceof NoneShardSpec);
}
}
)
) == 0, "segments without NoneShardSpec"
);
this.segments = segments;
}
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction()));
final ServiceEmitter emitter = toolbox.getEmitter();
final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder();
final DataSegment mergedSegment = computeMergedSegment(getDataSource(), myLock.getVersion(), segments);
final File taskDir = toolbox.getTaskDir();
try {
final long startTime = System.currentTimeMillis();
log.info(
"Starting merge of id[%s], segments: %s",
getId(),
Lists.transform(
segments,
new Function<DataSegment, String>()
{
@Override
public String apply(@Nullable DataSegment input)
{
return input.getIdentifier();
}
}
)
);
// download segments to merge
final Map<DataSegment, File> gettedSegments = toolbox.getSegments(segments);
// merge files together
final File fileToUpload = merge(gettedSegments, new File(taskDir, "merged"));
emitter.emit(builder.build("merger/numMerged", segments.size()));
emitter.emit(builder.build("merger/mergeTime", System.currentTimeMillis() - startTime));
log.info(
"[%s] : Merged %d segments in %,d millis",
mergedSegment.getDataSource(),
segments.size(),
System.currentTimeMillis() - startTime
);
long uploadStart = System.currentTimeMillis();
// Upload file
final DataSegment uploadedSegment = toolbox.getSegmentPusher().push(fileToUpload, mergedSegment);
emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart));
emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize()));
toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment)));
return TaskStatus.success(getId());
}
catch (Exception e) {
log.makeAlert(e, "Exception merging[%s]", mergedSegment.getDataSource())
.addData("interval", mergedSegment.getInterval())
.emit();
return TaskStatus.failure(getId());
}
}
/**
* Checks pre-existing segments in "context" to confirm that this merge query is valid. Specifically, confirm that
* we are operating on every segment that overlaps the chosen interval.
*/
@Override
public TaskStatus preflight(TaskToolbox toolbox)
{
final Function<DataSegment, String> toIdentifier = new Function<DataSegment, String>()
{
@Override
public String apply(DataSegment dataSegment)
{
return dataSegment.getIdentifier();
}
};
final Set<String> current = ImmutableSet.copyOf(
Iterables.transform(toolbox.getTaskActionClientFactory().submit(defaultListUsedAction()), toIdentifier)
);
final Set<String> requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier));
final Set<String> missingFromRequested = Sets.difference(current, requested);
if (!missingFromRequested.isEmpty()) {
throw new ISE(
"Merge is invalid: current segment(s) are not in the requested set: %s",
Joiner.on(", ").join(missingFromRequested)
);
}
final Set<String> missingFromCurrent = Sets.difference(requested, current);
if (!missingFromCurrent.isEmpty()) {
throw new ISE(
"Merge is invalid: requested segment(s) are not in the current set: %s",
Joiner.on(", ").join(missingFromCurrent)
);
}
return TaskStatus.running(getId());
}
protected abstract File merge(Map<DataSegment, File> segments, File outDir)
throws Exception;
@JsonProperty
public List<DataSegment> getSegments()
{
return segments;
}
@Override
public String toString()
{
return Objects.toStringHelper(this)
.add("id", getId())
.add("dataSource", getDataSource())
.add("interval", getImplicitLockInterval())
.add("segments", segments)
.toString();
}
private static String computeProcessingID(final String dataSource, final List<DataSegment> segments)
{
final String segmentIDs = Joiner.on("_").join(
Iterables.transform(
Ordering.natural().sortedCopy(segments), new Function<DataSegment, String>()
{
@Override
public String apply(DataSegment x)
{
return String.format(
"%s_%s_%s_%s",
x.getInterval().getStart(),
x.getInterval().getEnd(),
x.getVersion(),
x.getShardSpec().getPartitionNum()
);
}
}
)
);
return String.format(
"%s_%s",
dataSource,
Hashing.sha1().hashString(segmentIDs, Charsets.UTF_8).toString().toLowerCase()
);
}
private static Interval computeMergedInterval(final List<DataSegment> segments)
{
Preconditions.checkArgument(segments.size() > 0, "segments.size() > 0");
DateTime start = null;
DateTime end = null;
for(final DataSegment segment : segments) {
if(start == null || segment.getInterval().getStart().isBefore(start)) {
start = segment.getInterval().getStart();
}
if(end == null || segment.getInterval().getEnd().isAfter(end)) {
end = segment.getInterval().getEnd();
}
}
return new Interval(start, end);
}
private static DataSegment computeMergedSegment(
final String dataSource,
final String version,
final List<DataSegment> segments
)
{
final Interval mergedInterval = computeMergedInterval(segments);
final Set<String> mergedDimensions = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER);
final Set<String> mergedMetrics = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER);
for (DataSegment segment : segments) {
mergedDimensions.addAll(segment.getDimensions());
mergedMetrics.addAll(segment.getMetrics());
}
return DataSegment.builder()
.dataSource(dataSource)
.interval(mergedInterval)
.version(version)
.shardSpec(new NoneShardSpec())
.dimensions(Lists.newArrayList(mergedDimensions))
.metrics(Lists.newArrayList(mergedMetrics))
.build();
}
}

View File

@ -41,16 +41,18 @@ import org.joda.time.Interval;
* to release locks early if they desire.</li>
* </ul>
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultMergeTask.class)
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "append", value = AppendTask.class),
@JsonSubTypes.Type(name = "merge", value = DefaultMergeTask.class),
@JsonSubTypes.Type(name = "merge", value = MergeTask.class),
@JsonSubTypes.Type(name = "delete", value = DeleteTask.class),
@JsonSubTypes.Type(name = "kill", value = KillTask.class),
@JsonSubTypes.Type(name = "index", value = IndexTask.class),
@JsonSubTypes.Type(name = "index_partitions", value = IndexDeterminePartitionsTask.class),
@JsonSubTypes.Type(name = "index_generator", value = IndexGeneratorTask.class),
@JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class)
@JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class),
@JsonSubTypes.Type(name = "version_converter", value = VersionConverterTask.class),
@JsonSubTypes.Type(name = "version_converter_sub", value = VersionConverterTask.SubTask.class)
})
public interface Task
{

View File

@ -1,36 +0,0 @@
package com.metamx.druid.merger.common.task;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import org.joda.time.DateTime;
import org.joda.time.Interval;
/**
*/
public class V8toV9UpgradeTask extends AbstractTask
{
public V8toV9UpgradeTask(
@JsonProperty("dataSource") String dataSource,
@JsonProperty("interval") Interval interval
)
{
super(
String.format("v8tov9_%s_%s_%s", dataSource, interval.toString().replace("/", "_"), new DateTime()),
dataSource,
interval
);
}
@Override
public String getType()
{
return "8to9";
}
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
throw new UnsupportedOperationException();
}
}

View File

@ -0,0 +1,184 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.merger.common.task;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Function;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.metamx.common.ISE;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.index.v1.IndexIO;
import com.metamx.druid.loading.SegmentLoadingException;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.actions.SegmentInsertAction;
import com.metamx.druid.merger.common.actions.SpawnTasksAction;
import com.metamx.druid.merger.common.actions.TaskActionClient;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import java.io.File;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
/**
*/
public class VersionConverterTask extends AbstractTask
{
private static final String TYPE = "version_converter";
private static final Integer CURR_VERSION_INTEGER = new Integer(IndexIO.CURRENT_VERSION_ID);
private static final Logger log = new Logger(VersionConverterTask.class);
private final DataSegment segment;
public VersionConverterTask(
@JsonProperty("dataSource") String dataSource,
@JsonProperty("interval") Interval interval,
@JsonProperty("segment") DataSegment segment
)
{
super(
joinId(TYPE, dataSource, interval.getStart(), interval.getEnd(), new DateTime()),
dataSource,
interval
);
this.segment = segment;
}
@Override
public String getType()
{
return TYPE;
}
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
if (segment == null) {
throw new ISE("Segment was null, this should never run.", this.getClass().getSimpleName());
}
log.info("I'm in a subless mood.");
convertSegment(toolbox, segment);
return success();
}
@Override
public TaskStatus preflight(TaskToolbox toolbox) throws Exception
{
if (segment != null) {
return super.preflight(toolbox);
}
final TaskActionClient taskClient = toolbox.getTaskActionClientFactory();
List<DataSegment> segments = taskClient.submit(defaultListUsedAction());
final FunctionalIterable<Task> tasks = FunctionalIterable
.create(segments)
.keep(
new Function<DataSegment, Task>()
{
@Override
public Task apply(DataSegment segment)
{
final Integer segmentVersion = segment.getBinaryVersion();
if (!CURR_VERSION_INTEGER.equals(segmentVersion)) {
return new SubTask(getGroupId(), segment);
}
log.info("Skipping[%s], already version[%s]", segment.getIdentifier(), segmentVersion);
return null;
}
}
);
taskClient.submit(new SpawnTasksAction(Lists.newArrayList(tasks)));
return TaskStatus.success(getId());
}
public static class SubTask extends AbstractTask
{
private final DataSegment segment;
protected SubTask(
@JsonProperty("groupId") String groupId,
@JsonProperty("segment") DataSegment segment
)
{
super(
joinId(
groupId,
"sub",
segment.getInterval().getStart(),
segment.getInterval().getEnd(),
segment.getShardSpec().getPartitionNum()
),
groupId,
segment.getDataSource(),
segment.getInterval()
);
this.segment = segment;
}
@Override
public String getType()
{
return "version_converter_sub";
}
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
log.info("Subs are good! Italian BMT and Meatball are probably my favorite.");
convertSegment(toolbox, segment);
return success();
}
}
private static void convertSegment(TaskToolbox toolbox, final DataSegment segment)
throws SegmentLoadingException, IOException
{
log.info("Converting segment[%s]", segment);
final Map<DataSegment, File> localSegments = toolbox.getSegments(Arrays.asList(segment));
final File location = localSegments.get(segment);
final File outLocation = new File(location, "v9_out");
if (IndexIO.convertSegment(location, outLocation)) {
final int outVersion = IndexIO.getVersionFromDir(outLocation);
// Appending to the version makes a new version that inherits most comparability parameters of the original
// version, but is "newer" than said original version.
DataSegment updatedSegment = segment.withVersion(String.format("%s_v%s", segment.getVersion(), outVersion));
updatedSegment = toolbox.getSegmentPusher().push(outLocation, updatedSegment);
toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(Sets.newHashSet(updatedSegment)));
} else {
log.info("Conversion failed.");
}
}
}

View File

@ -327,11 +327,11 @@ public class DbTaskStorage implements TaskStorage
}
@Override
public <T> void addAuditLog(final TaskAction<T> taskAction)
public <T> void addAuditLog(final Task task, final TaskAction<T> taskAction)
{
Preconditions.checkNotNull(taskAction, "taskAction");
log.info("Logging action for task[%s]: %s", taskAction.getTask().getId(), taskAction);
log.info("Logging action for task[%s]: %s", task.getId(), taskAction);
dbi.withHandle(
new HandleCallback<Integer>()
@ -345,7 +345,7 @@ public class DbTaskStorage implements TaskStorage
dbConnectorConfig.getTaskLogTable()
)
)
.bind("task_id", taskAction.getTask().getId())
.bind("task_id", task.getId())
.bind("log_payload", jsonMapper.writeValueAsString(taskAction))
.execute();
}

View File

@ -40,14 +40,14 @@ import java.util.concurrent.locks.ReentrantLock;
* Implements an in-heap TaskStorage facility, with no persistence across restarts. This class is not
* thread safe.
*/
public class LocalTaskStorage implements TaskStorage
public class HeapMemoryTaskStorage implements TaskStorage
{
private final ReentrantLock giant = new ReentrantLock();
private final Map<String, TaskStuff> tasks = Maps.newHashMap();
private final Multimap<String, TaskLock> taskLocks = HashMultimap.create();
private final Multimap<String, TaskAction> taskActions = ArrayListMultimap.create();
private static final Logger log = new Logger(LocalTaskStorage.class);
private static final Logger log = new Logger(HeapMemoryTaskStorage.class);
@Override
public void insert(Task task, TaskStatus status)
@ -185,12 +185,12 @@ public class LocalTaskStorage implements TaskStorage
}
@Override
public <T> void addAuditLog(TaskAction<T> taskAction)
public <T> void addAuditLog(Task task, TaskAction<T> taskAction)
{
giant.lock();
try {
taskActions.put(taskAction.getTask().getId(), taskAction);
taskActions.put(task.getId(), taskAction);
} finally {
giant.unlock();
}

View File

@ -26,6 +26,7 @@ import com.metamx.common.logger.Logger;
import com.metamx.druid.merger.common.TaskCallback;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.task.Task;
import org.apache.commons.io.FileUtils;
@ -38,17 +39,17 @@ import java.util.concurrent.ExecutorService;
*/
public class LocalTaskRunner implements TaskRunner
{
private final TaskToolbox toolbox;
private final TaskToolboxFactory toolboxFactory;
private final ExecutorService exec;
private static final Logger log = new Logger(LocalTaskRunner.class);
public LocalTaskRunner(
TaskToolbox toolbox,
TaskToolboxFactory toolboxFactory,
ExecutorService exec
)
{
this.toolbox = toolbox;
this.toolboxFactory = toolboxFactory;
this.exec = exec;
}
@ -61,6 +62,8 @@ public class LocalTaskRunner implements TaskRunner
@Override
public void run(final Task task, final TaskCallback callback)
{
final TaskToolbox toolbox = toolboxFactory.build(task);
exec.submit(
new Runnable()
{
@ -89,7 +92,7 @@ public class LocalTaskRunner implements TaskRunner
}
try {
final File taskDir = toolbox.getConfig().getTaskDir(task);
final File taskDir = toolbox.getTaskDir();
if (taskDir.exists()) {
log.info("Removing task directory: %s", taskDir);

View File

@ -35,7 +35,7 @@ import com.metamx.druid.merger.common.TaskCallback;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
import com.metamx.druid.merger.worker.Worker;
import com.metamx.emitter.EmittingLogger;
import com.netflix.curator.framework.CuratorFramework;
@ -57,6 +57,7 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
/**
* The RemoteTaskRunner's primary responsibility is to assign tasks to worker nodes and manage retries in failure
@ -82,7 +83,7 @@ public class RemoteTaskRunner implements TaskRunner
private final PathChildrenCache workerPathCache;
private final ScheduledExecutorService scheduledExec;
private final RetryPolicyFactory retryPolicyFactory;
private final WorkerSetupManager workerSetupManager;
private final AtomicReference<WorkerSetupData> workerSetupData;
// all workers that exist in ZK
private final Map<String, ZkWorker> zkWorkers = new ConcurrentHashMap<String, ZkWorker>();
@ -104,7 +105,7 @@ public class RemoteTaskRunner implements TaskRunner
PathChildrenCache workerPathCache,
ScheduledExecutorService scheduledExec,
RetryPolicyFactory retryPolicyFactory,
WorkerSetupManager workerSetupManager
AtomicReference<WorkerSetupData> workerSetupData
)
{
this.jsonMapper = jsonMapper;
@ -113,7 +114,7 @@ public class RemoteTaskRunner implements TaskRunner
this.workerPathCache = workerPathCache;
this.scheduledExec = scheduledExec;
this.retryPolicyFactory = retryPolicyFactory;
this.workerSetupManager = workerSetupManager;
this.workerSetupData = workerSetupData;
}
@LifecycleStart
@ -548,7 +549,7 @@ public class RemoteTaskRunner implements TaskRunner
return (!input.isAtCapacity() &&
input.getWorker()
.getVersion()
.compareTo(workerSetupManager.getWorkerSetupData().getMinVersion()) >= 0);
.compareTo(workerSetupData.get().getMinVersion()) >= 0);
}
}
)

View File

@ -1,3 +1,22 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.merger.coordinator;
import com.google.common.base.Function;

View File

@ -26,6 +26,8 @@ import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.initialization.Initialization;
import com.metamx.druid.initialization.ServiceDiscoveryConfig;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
import com.metamx.druid.merger.coordinator.exec.TaskConsumer;
import com.metamx.druid.merger.coordinator.scaling.ResourceManagementScheduler;
@ -49,7 +51,7 @@ public class TaskMasterLifecycle
private final ReentrantLock giant = new ReentrantLock();
private final Condition mayBeStopped = giant.newCondition();
private final TaskQueue taskQueue;
private final TaskToolbox taskToolbox;
private final TaskToolboxFactory taskToolboxFactory;
private volatile boolean leading = false;
private volatile TaskRunner taskRunner;
@ -59,7 +61,7 @@ public class TaskMasterLifecycle
public TaskMasterLifecycle(
final TaskQueue taskQueue,
final TaskToolbox taskToolbox,
final TaskToolboxFactory taskToolboxFactory,
final IndexerCoordinatorConfig indexerCoordinatorConfig,
final ServiceDiscoveryConfig serviceDiscoveryConfig,
final TaskRunnerFactory runnerFactory,
@ -69,7 +71,7 @@ public class TaskMasterLifecycle
)
{
this.taskQueue = taskQueue;
this.taskToolbox = taskToolbox;
this.taskToolboxFactory = taskToolboxFactory;
this.leaderSelector = new LeaderSelector(
curator, indexerCoordinatorConfig.getLeaderLatchPath(), new LeaderSelectorListener()
@ -87,7 +89,7 @@ public class TaskMasterLifecycle
final TaskConsumer taskConsumer = new TaskConsumer(
taskQueue,
taskRunner,
taskToolbox,
taskToolboxFactory,
emitter
);
@ -221,9 +223,4 @@ public class TaskMasterLifecycle
{
return taskToolbox;
}
public ResourceManagementScheduler getResourceManagementScheduler()
{
return resourceManagementScheduler;
}
}

View File

@ -69,7 +69,7 @@ public interface TaskStorage
/**
* Add an action taken by a task to the audit log.
*/
public <T> void addAuditLog(TaskAction<T> taskAction);
public <T> void addAuditLog(Task task, TaskAction<T> taskAction);
/**
* Returns all actions taken by a task.

View File

@ -24,7 +24,7 @@ import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.merger.common.TaskCallback;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.TaskQueue;
import com.metamx.druid.merger.coordinator.TaskRunner;
@ -36,7 +36,7 @@ public class TaskConsumer implements Runnable
{
private final TaskQueue queue;
private final TaskRunner runner;
private final TaskToolbox toolbox;
private final TaskToolboxFactory toolboxFactory;
private final ServiceEmitter emitter;
private final Thread thready;
@ -47,13 +47,13 @@ public class TaskConsumer implements Runnable
public TaskConsumer(
TaskQueue queue,
TaskRunner runner,
TaskToolbox toolbox,
TaskToolboxFactory toolboxFactory,
ServiceEmitter emitter
)
{
this.queue = queue;
this.runner = runner;
this.toolbox = toolbox;
this.toolboxFactory = toolboxFactory;
this.emitter = emitter;
this.thready = new Thread(this);
}
@ -123,7 +123,7 @@ public class TaskConsumer implements Runnable
// Run preflight checks
TaskStatus preflightStatus;
try {
preflightStatus = task.preflight(toolbox);
preflightStatus = task.preflight(toolboxFactory.build(task));
log.info("Preflight done for task: %s", task.getId());
}
catch (Exception e) {

View File

@ -39,6 +39,9 @@ import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import com.metamx.druid.RegisteringNode;
import com.metamx.druid.config.ConfigManager;
import com.metamx.druid.config.ConfigManagerConfig;
import com.metamx.druid.config.JacksonConfigManager;
import com.metamx.druid.db.DbConnector;
import com.metamx.druid.db.DbConnectorConfig;
import com.metamx.druid.http.GuiceServletConfig;
@ -48,22 +51,21 @@ import com.metamx.druid.http.RedirectInfo;
import com.metamx.druid.http.StatusServlet;
import com.metamx.druid.initialization.Initialization;
import com.metamx.druid.initialization.ServerConfig;
import com.metamx.druid.initialization.ServerInit;
import com.metamx.druid.initialization.ServiceDiscoveryConfig;
import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.loading.DataSegmentKiller;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.loading.S3DataSegmentPusher;
import com.metamx.druid.loading.S3DataSegmentPusherConfig;
import com.metamx.druid.loading.S3SegmentKiller;
import com.metamx.druid.loading.SegmentKiller;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.actions.LocalTaskActionClient;
import com.metamx.druid.loading.S3DataSegmentKiller;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory;
import com.metamx.druid.merger.common.actions.TaskActionToolbox;
import com.metamx.druid.merger.common.config.IndexerZkConfig;
import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory;
import com.metamx.druid.merger.coordinator.DbTaskStorage;
import com.metamx.druid.merger.coordinator.HeapMemoryTaskStorage;
import com.metamx.druid.merger.coordinator.LocalTaskRunner;
import com.metamx.druid.merger.coordinator.LocalTaskStorage;
import com.metamx.druid.merger.coordinator.MergerDBCoordinator;
import com.metamx.druid.merger.coordinator.RemoteTaskRunner;
import com.metamx.druid.merger.coordinator.RetryPolicyFactory;
@ -79,7 +81,6 @@ import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
import com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig;
import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig;
import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig;
import com.metamx.druid.merger.coordinator.config.WorkerSetupManagerConfig;
import com.metamx.druid.merger.coordinator.scaling.AutoScalingStrategy;
import com.metamx.druid.merger.coordinator.scaling.EC2AutoScalingStrategy;
import com.metamx.druid.merger.coordinator.scaling.NoopAutoScalingStrategy;
@ -88,7 +89,7 @@ import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerCo
import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerFactory;
import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagementStrategy;
import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagmentConfig;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
import com.metamx.druid.utils.PropUtils;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.core.Emitters;
@ -123,6 +124,7 @@ import java.util.Properties;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicReference;
/**
*/
@ -147,7 +149,8 @@ public class IndexerCoordinatorNode extends RegisteringNode
private RestS3Service s3Service = null;
private IndexerCoordinatorConfig config = null;
private TaskConfig taskConfig = null;
private TaskToolbox taskToolbox = null;
private DataSegmentPusher segmentPusher = null;
private TaskToolboxFactory taskToolboxFactory = null;
private MergerDBCoordinator mergerDBCoordinator = null;
private TaskStorage taskStorage = null;
private TaskQueue taskQueue = null;
@ -155,7 +158,6 @@ public class IndexerCoordinatorNode extends RegisteringNode
private CuratorFramework curatorFramework = null;
private ScheduledExecutorFactory scheduledExecutorFactory = null;
private IndexerZkConfig indexerZkConfig;
private WorkerSetupManager workerSetupManager = null;
private TaskRunnerFactory taskRunnerFactory = null;
private ResourceManagementSchedulerFactory resourceManagementSchedulerFactory = null;
private TaskMasterLifecycle taskMasterLifecycle = null;
@ -208,6 +210,12 @@ public class IndexerCoordinatorNode extends RegisteringNode
return this;
}
public IndexerCoordinatorNode setSegmentPusher(DataSegmentPusher segmentPusher)
{
this.segmentPusher = segmentPusher;
return this;
}
public IndexerCoordinatorNode setMergeDbCoordinator(MergerDBCoordinator mergeDbCoordinator)
{
this.mergerDBCoordinator = mergeDbCoordinator;
@ -220,12 +228,6 @@ public class IndexerCoordinatorNode extends RegisteringNode
return this;
}
public IndexerCoordinatorNode setWorkerSetupManager(WorkerSetupManager workerSetupManager)
{
this.workerSetupManager = workerSetupManager;
return this;
}
public IndexerCoordinatorNode setTaskRunnerFactory(TaskRunnerFactory taskRunnerFactory)
{
this.taskRunnerFactory = taskRunnerFactory;
@ -242,6 +244,10 @@ public class IndexerCoordinatorNode extends RegisteringNode
{
scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle);
final ConfigManagerConfig managerConfig = configFactory.build(ConfigManagerConfig.class);
DbConnector.createConfigTable(dbi, managerConfig.getConfigTable());
JacksonConfigManager configManager = new JacksonConfigManager(new ConfigManager(dbi, managerConfig), jsonMapper);
initializeEmitter();
initializeMonitors();
initializeDB();
@ -252,14 +258,14 @@ public class IndexerCoordinatorNode extends RegisteringNode
initializeTaskStorage();
initializeTaskLockbox();
initializeTaskQueue();
initializeDataSegmentPusher();
initializeTaskToolbox();
initializeJacksonInjections();
initializeJacksonSubtypes();
initializeCurator();
initializeIndexerZkConfig();
initializeWorkerSetupManager();
initializeTaskRunnerFactory();
initializeResourceManagement();
initializeTaskRunnerFactory(configManager);
initializeResourceManagement(configManager);
initializeTaskMasterLifecycle();
initializeServer();
@ -279,7 +285,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
emitter,
taskMasterLifecycle,
new TaskStorageQueryAdapter(taskStorage),
workerSetupManager
configManager
)
);
@ -339,7 +345,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class);
taskMasterLifecycle = new TaskMasterLifecycle(
taskQueue,
taskToolbox,
taskToolboxFactory,
config,
serviceDiscoveryConfig,
taskRunnerFactory,
@ -403,7 +409,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
InjectableValues.Std injectables = new InjectableValues.Std();
injectables.addValue("s3Client", s3Service)
.addValue("segmentPusher", taskToolbox.getSegmentPusher());
.addValue("segmentPusher", segmentPusher);
jsonMapper.setInjectableValues(injectables);
}
@ -472,27 +478,27 @@ public class IndexerCoordinatorNode extends RegisteringNode
);
}
public void initializeDataSegmentPusher()
{
if (segmentPusher == null) {
segmentPusher = ServerInit.getSegmentPusher(props, configFactory, jsonMapper);
}
}
public void initializeTaskToolbox()
{
if (taskToolbox == null) {
final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher(
s3Service,
configFactory.build(S3DataSegmentPusherConfig.class),
jsonMapper
);
final SegmentKiller segmentKiller = new S3SegmentKiller(
s3Service
);
taskToolbox = new TaskToolbox(
if (taskToolboxFactory == null) {
final DataSegmentKiller dataSegmentKiller = new S3DataSegmentKiller(s3Service);
taskToolboxFactory = new TaskToolboxFactory(
taskConfig,
new LocalTaskActionClient(
new LocalTaskActionClientFactory(
taskStorage,
new TaskActionToolbox(taskQueue, taskLockbox, mergerDBCoordinator, emitter)
),
emitter,
s3Service,
dataSegmentPusher,
segmentKiller,
segmentPusher,
dataSegmentKiller,
jsonMapper
);
}
@ -546,7 +552,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
{
if (taskStorage == null) {
if (config.getStorageImpl().equals("local")) {
taskStorage = new LocalTaskStorage();
taskStorage = new HeapMemoryTaskStorage();
} else if (config.getStorageImpl().equals("db")) {
final IndexerDbConnectorConfig dbConnectorConfig = configFactory.build(IndexerDbConnectorConfig.class);
taskStorage = new DbTaskStorage(jsonMapper, dbConnectorConfig, new DbConnector(dbConnectorConfig).getDBI());
@ -556,26 +562,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
}
}
public void initializeWorkerSetupManager()
{
if (workerSetupManager == null) {
final WorkerSetupManagerConfig workerSetupManagerConfig = configFactory.build(WorkerSetupManagerConfig.class);
DbConnector.createConfigTable(dbi, workerSetupManagerConfig.getConfigTable());
workerSetupManager = new WorkerSetupManager(
dbi, Executors.newScheduledThreadPool(
1,
new ThreadFactoryBuilder()
.setDaemon(true)
.setNameFormat("WorkerSetupManagerExec--%d")
.build()
), jsonMapper, workerSetupManagerConfig
);
}
lifecycle.addManagedInstance(workerSetupManager);
}
public void initializeTaskRunnerFactory()
private void initializeTaskRunnerFactory(final JacksonConfigManager configManager)
{
if (taskRunnerFactory == null) {
if (config.getRunnerImpl().equals("remote")) {
@ -601,7 +588,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
new PathChildrenCache(curatorFramework, indexerZkConfig.getAnnouncementPath(), true),
retryScheduledExec,
new RetryPolicyFactory(configFactory.build(RetryPolicyConfig.class)),
workerSetupManager
configManager.watch(WorkerSetupData.CONFIG_KEY, WorkerSetupData.class)
);
return remoteTaskRunner;
@ -615,7 +602,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
public TaskRunner build()
{
final ExecutorService runnerExec = Executors.newFixedThreadPool(config.getNumLocalThreads());
return new LocalTaskRunner(taskToolbox, runnerExec);
return new LocalTaskRunner(taskToolboxFactory, runnerExec);
}
};
} else {
@ -624,7 +611,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
}
}
private void initializeResourceManagement()
private void initializeResourceManagement(final JacksonConfigManager configManager)
{
if (resourceManagementSchedulerFactory == null) {
resourceManagementSchedulerFactory = new ResourceManagementSchedulerFactory()
@ -639,6 +626,9 @@ public class IndexerCoordinatorNode extends RegisteringNode
.setNameFormat("ScalingExec--%d")
.build()
);
final AtomicReference<WorkerSetupData> workerSetupData = configManager.watch(
WorkerSetupData.CONFIG_KEY, WorkerSetupData.class
);
AutoScalingStrategy strategy;
if (config.getStrategyImpl().equalsIgnoreCase("ec2")) {
@ -651,7 +641,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
)
),
configFactory.build(EC2AutoScalingStrategyConfig.class),
workerSetupManager
workerSetupData
);
} else if (config.getStrategyImpl().equalsIgnoreCase("noop")) {
strategy = new NoopAutoScalingStrategy();
@ -664,7 +654,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
new SimpleResourceManagementStrategy(
strategy,
configFactory.build(SimpleResourceManagmentConfig.class),
workerSetupManager
workerSetupData
),
configFactory.build(ResourceManagementSchedulerConfig.class),
scalingScheduledExec

View File

@ -19,6 +19,8 @@
package com.metamx.druid.merger.coordinator.http;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Optional;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
@ -26,8 +28,9 @@ import com.google.common.collect.Maps;
import com.google.inject.Inject;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.config.JacksonConfigManager;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.actions.TaskAction;
import com.metamx.druid.merger.common.actions.TaskActionHolder;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.TaskMasterLifecycle;
import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter;
@ -35,10 +38,7 @@ import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
import com.metamx.druid.merger.coordinator.scaling.AutoScalingData;
import com.metamx.druid.merger.coordinator.scaling.ScalingStats;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
import com.metamx.emitter.service.ServiceEmitter;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.core.type.TypeReference;
import javax.ws.rs.Consumes;
import javax.ws.rs.GET;
@ -49,6 +49,7 @@ import javax.ws.rs.Produces;
import javax.ws.rs.core.Response;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
/**
*/
@ -61,16 +62,18 @@ public class IndexerCoordinatorResource
private final ServiceEmitter emitter;
private final TaskMasterLifecycle taskMasterLifecycle;
private final TaskStorageQueryAdapter taskStorageQueryAdapter;
private final WorkerSetupManager workerSetupManager;
private final JacksonConfigManager configManager;
private final ObjectMapper jsonMapper;
private AtomicReference<WorkerSetupData> workerSetupDataRef = null;
@Inject
public IndexerCoordinatorResource(
IndexerCoordinatorConfig config,
ServiceEmitter emitter,
TaskMasterLifecycle taskMasterLifecycle,
TaskStorageQueryAdapter taskStorageQueryAdapter,
WorkerSetupManager workerSetupManager,
JacksonConfigManager configManager,
ObjectMapper jsonMapper
) throws Exception
{
@ -78,7 +81,7 @@ public class IndexerCoordinatorResource
this.emitter = emitter;
this.taskMasterLifecycle = taskMasterLifecycle;
this.taskStorageQueryAdapter = taskStorageQueryAdapter;
this.workerSetupManager = workerSetupManager;
this.configManager = configManager;
this.jsonMapper = jsonMapper;
}
@ -98,18 +101,15 @@ public class IndexerCoordinatorResource
@Produces("application/json")
public Response doIndex(final Task task)
{
// verify against whitelist
if (config.isWhitelistEnabled() && !config.getWhitelistDatasources().contains(task.getDataSource())) {
return Response.status(Response.Status.BAD_REQUEST)
.entity(
ImmutableMap.of(
"error",
String.format("dataSource[%s] is not whitelisted", task.getDataSource())
)
)
.build();
}
return taskPost(task);
}
@POST
@Path("/task")
@Consumes("application/json")
@Produces("application/json")
public Response taskPost(final Task task)
{
taskMasterLifecycle.getTaskQueue().add(task);
return Response.ok(ImmutableMap.of("task", task.getId())).build();
}
@ -162,7 +162,11 @@ public class IndexerCoordinatorResource
@Produces("application/json")
public Response getWorkerSetupData()
{
return Response.ok(workerSetupManager.getWorkerSetupData()).build();
if (workerSetupDataRef == null) {
workerSetupDataRef = configManager.watch(WorkerSetupData.CONFIG_KEY, WorkerSetupData.class);
}
return Response.ok(workerSetupDataRef.get()).build();
}
@POST
@ -172,7 +176,7 @@ public class IndexerCoordinatorResource
final WorkerSetupData workerSetupData
)
{
if (!workerSetupManager.setWorkerSetupData(workerSetupData)) {
if (!configManager.set(WorkerSetupData.CONFIG_KEY, workerSetupData)) {
return Response.status(Response.Status.BAD_REQUEST).build();
}
return Response.ok().build();
@ -181,9 +185,12 @@ public class IndexerCoordinatorResource
@POST
@Path("/action")
@Produces("application/json")
public <T> Response doAction(final TaskAction<T> action)
public <T> Response doAction(final TaskActionHolder<T> holder)
{
final T ret = taskMasterLifecycle.getTaskToolbox().getTaskActionClient().submit(action);
final T ret = taskMasterLifecycle.getTaskToolbox(holder.getTask())
.getTaskActionClientFactory()
.submit(holder.getAction());
final Map<String, Object> retMap = Maps.newHashMap();
retMap.put("result", ret);

View File

@ -22,10 +22,10 @@ package com.metamx.druid.merger.coordinator.http;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider;
import com.google.inject.Provides;
import com.metamx.druid.config.JacksonConfigManager;
import com.metamx.druid.merger.coordinator.TaskMasterLifecycle;
import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter;
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
import com.metamx.emitter.service.ServiceEmitter;
import com.sun.jersey.guice.JerseyServletModule;
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
@ -41,7 +41,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
private final ServiceEmitter emitter;
private final TaskMasterLifecycle taskMasterLifecycle;
private final TaskStorageQueryAdapter taskStorageQueryAdapter;
private final WorkerSetupManager workerSetupManager;
private final JacksonConfigManager configManager;
public IndexerCoordinatorServletModule(
ObjectMapper jsonMapper,
@ -49,7 +49,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
ServiceEmitter emitter,
TaskMasterLifecycle taskMasterLifecycle,
TaskStorageQueryAdapter taskStorageQueryAdapter,
WorkerSetupManager workerSetupManager
JacksonConfigManager configManager
)
{
this.jsonMapper = jsonMapper;
@ -57,7 +57,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
this.emitter = emitter;
this.taskMasterLifecycle = taskMasterLifecycle;
this.taskStorageQueryAdapter = taskStorageQueryAdapter;
this.workerSetupManager = workerSetupManager;
this.configManager = configManager;
}
@Override
@ -69,7 +69,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
bind(ServiceEmitter.class).toInstance(emitter);
bind(TaskMasterLifecycle.class).toInstance(taskMasterLifecycle);
bind(TaskStorageQueryAdapter.class).toInstance(taskStorageQueryAdapter);
bind(WorkerSetupManager.class).toInstance(workerSetupManager);
bind(JacksonConfigManager.class).toInstance(configManager);
serve("/*").with(GuiceContainer.class);
}

View File

@ -34,13 +34,13 @@ import com.google.common.collect.Lists;
import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig;
import com.metamx.druid.merger.coordinator.setup.EC2NodeData;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
import com.metamx.emitter.EmittingLogger;
import org.apache.commons.codec.binary.Base64;
import javax.annotation.Nullable;
import java.util.List;
import java.util.concurrent.atomic.AtomicReference;
/**
*/
@ -51,26 +51,26 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy<Instance>
private final ObjectMapper jsonMapper;
private final AmazonEC2Client amazonEC2Client;
private final EC2AutoScalingStrategyConfig config;
private final WorkerSetupManager workerSetupManager;
private final AtomicReference<WorkerSetupData> workerSetupDataRef;
public EC2AutoScalingStrategy(
ObjectMapper jsonMapper,
AmazonEC2Client amazonEC2Client,
EC2AutoScalingStrategyConfig config,
WorkerSetupManager workerSetupManager
AtomicReference<WorkerSetupData> workerSetupDataRef
)
{
this.jsonMapper = jsonMapper;
this.amazonEC2Client = amazonEC2Client;
this.config = config;
this.workerSetupManager = workerSetupManager;
this.workerSetupDataRef = workerSetupDataRef;
}
@Override
public AutoScalingData<Instance> provision()
{
try {
WorkerSetupData setupData = workerSetupManager.getWorkerSetupData();
WorkerSetupData setupData = workerSetupDataRef.get();
EC2NodeData workerConfig = setupData.getNodeData();
RunInstancesResult result = amazonEC2Client.runInstances(

View File

@ -1,3 +1,22 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.merger.coordinator.scaling;
import com.fasterxml.jackson.annotation.JsonProperty;

View File

@ -27,16 +27,16 @@ import com.google.common.collect.Sets;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem;
import com.metamx.druid.merger.coordinator.ZkWorker;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
import com.metamx.emitter.EmittingLogger;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.Set;
import java.util.concurrent.ConcurrentSkipListSet;
import java.util.concurrent.atomic.AtomicReference;
/**
*/
@ -46,7 +46,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat
private final AutoScalingStrategy autoScalingStrategy;
private final SimpleResourceManagmentConfig config;
private final WorkerSetupManager workerSetupManager;
private final AtomicReference<WorkerSetupData> workerSetupdDataRef;
private final ScalingStats scalingStats;
private final ConcurrentSkipListSet<String> currentlyProvisioning = new ConcurrentSkipListSet<String>();
@ -58,12 +58,12 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat
public SimpleResourceManagementStrategy(
AutoScalingStrategy autoScalingStrategy,
SimpleResourceManagmentConfig config,
WorkerSetupManager workerSetupManager
AtomicReference<WorkerSetupData> workerSetupdDataRef
)
{
this.autoScalingStrategy = autoScalingStrategy;
this.config = config;
this.workerSetupManager = workerSetupManager;
this.workerSetupdDataRef = workerSetupdDataRef;
this.scalingStats = new ScalingStats(config.getNumEventsToTrack());
}
@ -151,7 +151,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat
boolean nothingTerminating = currentlyTerminating.isEmpty();
if (nothingTerminating) {
final int minNumWorkers = workerSetupManager.getWorkerSetupData().getMinNumWorkers();
final int minNumWorkers = workerSetupdDataRef.get().getMinNumWorkers();
if (zkWorkers.size() <= minNumWorkers) {
log.info("Only [%d <= %d] nodes in the cluster, not terminating anything.", zkWorkers.size(), minNumWorkers);
return false;

View File

@ -31,6 +31,8 @@ import java.util.List;
*/
public class WorkerSetupData
{
public static final String CONFIG_KEY = "worker.setup";
private final String minVersion;
private final int minNumWorkers;
private final EC2NodeData nodeData;

View File

@ -1,227 +0,0 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.merger.coordinator.setup;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.google.common.collect.Lists;
import com.metamx.common.ISE;
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 com.metamx.druid.merger.coordinator.config.WorkerSetupManagerConfig;
import org.apache.commons.collections.MapUtils;
import org.joda.time.Duration;
import org.skife.jdbi.v2.DBI;
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.tweak.HandleCallback;
import java.sql.SQLException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicReference;
/**
*/
public class WorkerSetupManager
{
private static final Logger log = new Logger(WorkerSetupManager.class);
private final DBI dbi;
private final ObjectMapper jsonMapper;
private final ScheduledExecutorService exec;
private final WorkerSetupManagerConfig config;
private final Object lock = new Object();
private volatile AtomicReference<WorkerSetupData> workerSetupData = new AtomicReference<WorkerSetupData>(null);
private volatile boolean started = false;
public WorkerSetupManager(
DBI dbi,
ScheduledExecutorService exec,
ObjectMapper jsonMapper,
WorkerSetupManagerConfig config
)
{
this.dbi = dbi;
this.exec = exec;
this.jsonMapper = jsonMapper;
this.config = config;
}
@LifecycleStart
public void start()
{
synchronized (lock) {
if (started) {
return;
}
ScheduledExecutors.scheduleWithFixedDelay(
exec,
new Duration(0),
config.getPollDuration(),
new Runnable()
{
@Override
public void run()
{
poll();
}
}
);
started = true;
}
}
@LifecycleStop
public void stop()
{
synchronized (lock) {
if (!started) {
return;
}
started = false;
}
}
public void poll()
{
try {
List<WorkerSetupData> setupDataList = dbi.withHandle(
new HandleCallback<List<WorkerSetupData>>()
{
@Override
public List<WorkerSetupData> withHandle(Handle handle) throws Exception
{
return handle.createQuery(
String.format(
"SELECT payload FROM %s WHERE name = :name",
config.getConfigTable()
)
)
.bind("name", config.getWorkerSetupConfigName())
.fold(
Lists.<WorkerSetupData>newArrayList(),
new Folder3<ArrayList<WorkerSetupData>, Map<String, Object>>()
{
@Override
public ArrayList<WorkerSetupData> fold(
ArrayList<WorkerSetupData> workerNodeConfigurations,
Map<String, Object> stringObjectMap,
FoldController foldController,
StatementContext statementContext
) throws SQLException
{
try {
// stringObjectMap lowercases and jackson may fail serde
workerNodeConfigurations.add(
jsonMapper.readValue(
MapUtils.getString(stringObjectMap, "payload"),
WorkerSetupData.class
)
);
return workerNodeConfigurations;
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
}
);
}
}
);
if (setupDataList.isEmpty()) {
throw new ISE("WTF?! No configuration found for worker nodes!");
} else if (setupDataList.size() != 1) {
throw new ISE("WTF?! Found more than one configuration for worker nodes");
}
workerSetupData.set(setupDataList.get(0));
}
catch (Exception e) {
log.error(e, "Exception while polling for worker setup data!");
}
}
@SuppressWarnings("unchecked")
public WorkerSetupData getWorkerSetupData()
{
synchronized (lock) {
if (!started) {
throw new ISE("Must start WorkerSetupManager first!");
}
return workerSetupData.get();
}
}
public boolean setWorkerSetupData(final WorkerSetupData value)
{
synchronized (lock) {
try {
if (!started) {
throw new ISE("Must start WorkerSetupManager first!");
}
dbi.withHandle(
new HandleCallback<Void>()
{
@Override
public Void withHandle(Handle handle) throws Exception
{
handle.createStatement(
String.format(
"INSERT INTO %s (name, payload) VALUES (:name, :payload) ON DUPLICATE KEY UPDATE payload = :payload",
config.getConfigTable()
)
)
.bind("name", config.getWorkerSetupConfigName())
.bind("payload", jsonMapper.writeValueAsString(value))
.execute();
return null;
}
}
);
workerSetupData.set(value);
}
catch (Exception e) {
log.error(e, "Exception updating worker config");
return false;
}
}
return true;
}
}

View File

@ -23,6 +23,7 @@ import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.emitter.EmittingLogger;
import com.netflix.curator.framework.CuratorFramework;
@ -45,21 +46,21 @@ public class TaskMonitor
private final PathChildrenCache pathChildrenCache;
private final CuratorFramework cf;
private final WorkerCuratorCoordinator workerCuratorCoordinator;
private final TaskToolbox toolbox;
private final TaskToolboxFactory toolboxFactory;
private final ExecutorService exec;
public TaskMonitor(
PathChildrenCache pathChildrenCache,
CuratorFramework cf,
WorkerCuratorCoordinator workerCuratorCoordinator,
TaskToolbox toolbox,
TaskToolboxFactory toolboxFactory,
ExecutorService exec
)
{
this.pathChildrenCache = pathChildrenCache;
this.cf = cf;
this.workerCuratorCoordinator = workerCuratorCoordinator;
this.toolbox = toolbox;
this.toolboxFactory = toolboxFactory;
this.exec = exec;
}
@ -81,10 +82,11 @@ public class TaskMonitor
throws Exception
{
if (pathChildrenCacheEvent.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) {
final Task task = toolbox.getObjectMapper().readValue(
final Task task = toolboxFactory.getObjectMapper().readValue(
cf.getData().forPath(pathChildrenCacheEvent.getData().getPath()),
Task.class
);
final TaskToolbox toolbox = toolboxFactory.build(task);
if (workerCuratorCoordinator.statusExists(task.getId())) {
log.warn("Got task %s that I am already running...", task.getId());
@ -99,7 +101,7 @@ public class TaskMonitor
public void run()
{
final long startTime = System.currentTimeMillis();
final File taskDir = toolbox.getConfig().getTaskDir(task);
final File taskDir = toolbox.getTaskDir();
log.info("Running task [%s]", task.getId());

View File

@ -35,15 +35,14 @@ import com.metamx.druid.http.StatusServlet;
import com.metamx.druid.initialization.CuratorConfig;
import com.metamx.druid.initialization.Initialization;
import com.metamx.druid.initialization.ServerConfig;
import com.metamx.druid.initialization.ServerInit;
import com.metamx.druid.initialization.ServiceDiscoveryConfig;
import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.loading.S3DataSegmentPusher;
import com.metamx.druid.loading.S3DataSegmentPusherConfig;
import com.metamx.druid.loading.S3SegmentKiller;
import com.metamx.druid.loading.SegmentKiller;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.actions.RemoteTaskActionClient;
import com.metamx.druid.loading.S3DataSegmentKiller;
import com.metamx.druid.loading.DataSegmentKiller;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.actions.RemoteTaskActionClientFactory;
import com.metamx.druid.merger.common.config.IndexerZkConfig;
import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory;
@ -106,7 +105,8 @@ public class WorkerNode extends RegisteringNode
private ServiceEmitter emitter = null;
private TaskConfig taskConfig = null;
private WorkerConfig workerConfig = null;
private TaskToolbox taskToolbox = null;
private DataSegmentPusher segmentPusher = null;
private TaskToolboxFactory taskToolboxFactory = null;
private CuratorFramework curatorFramework = null;
private ServiceDiscovery serviceDiscovery = null;
private ServiceProvider coordinatorServiceProvider = null;
@ -149,9 +149,15 @@ public class WorkerNode extends RegisteringNode
return this;
}
public WorkerNode setTaskToolbox(TaskToolbox taskToolbox)
public WorkerNode setSegmentPusher(DataSegmentPusher segmentPusher)
{
this.taskToolbox = taskToolbox;
this.segmentPusher = segmentPusher;
return this;
}
public WorkerNode setTaskToolboxFactory(TaskToolboxFactory taskToolboxFactory)
{
this.taskToolboxFactory = taskToolboxFactory;
return this;
}
@ -195,6 +201,7 @@ public class WorkerNode extends RegisteringNode
initializeCuratorFramework();
initializeServiceDiscovery();
initializeCoordinatorServiceProvider();
initializeDataSegmentPusher();
initializeTaskToolbox();
initializeJacksonInjections();
initializeJacksonSubtypes();
@ -271,7 +278,7 @@ public class WorkerNode extends RegisteringNode
InjectableValues.Std injectables = new InjectableValues.Std();
injectables.addValue("s3Client", s3Service)
.addValue("segmentPusher", taskToolbox.getSegmentPusher());
.addValue("segmentPusher", segmentPusher);
jsonMapper.setInjectableValues(injectables);
}
@ -334,24 +341,24 @@ public class WorkerNode extends RegisteringNode
}
}
public void initializeDataSegmentPusher()
{
if (segmentPusher == null) {
segmentPusher = ServerInit.getSegmentPusher(props, configFactory, jsonMapper);
}
}
public void initializeTaskToolbox() throws S3ServiceException
{
if (taskToolbox == null) {
final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher(
s3Service,
configFactory.build(S3DataSegmentPusherConfig.class),
jsonMapper
);
final SegmentKiller segmentKiller = new S3SegmentKiller(
s3Service
);
taskToolbox = new TaskToolbox(
if (taskToolboxFactory == null) {
final DataSegmentKiller dataSegmentKiller = new S3DataSegmentKiller(s3Service);
taskToolboxFactory = new TaskToolboxFactory(
taskConfig,
new RemoteTaskActionClient(httpClient, coordinatorServiceProvider, jsonMapper),
new RemoteTaskActionClientFactory(httpClient, coordinatorServiceProvider, jsonMapper),
emitter,
s3Service,
dataSegmentPusher,
segmentKiller,
segmentPusher,
dataSegmentKiller,
jsonMapper
);
}
@ -417,7 +424,7 @@ public class WorkerNode extends RegisteringNode
pathChildrenCache,
curatorFramework,
workerCuratorCoordinator,
taskToolbox,
taskToolboxFactory,
workerExec
);
lifecycle.addManagedInstance(taskMonitor);

View File

@ -26,14 +26,14 @@ import com.metamx.druid.aggregation.AggregatorFactory;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.task.DefaultMergeTask;
import com.metamx.druid.merger.common.task.MergeTask;
import java.util.List;
/**
*/
@JsonTypeName("test")
public class TestTask extends DefaultMergeTask
public class TestTask extends MergeTask
{
private final String id;
private final TaskStatus status;

View File

@ -31,7 +31,7 @@ import java.io.File;
import java.util.List;
import java.util.Map;
public class MergeTaskTest
public class MergeTaskBaseTest
{
private final DataSegment.Builder segmentBuilder = DataSegment.builder()
.dataSource("foo")
@ -43,7 +43,7 @@ public class MergeTaskTest
.add(segmentBuilder.interval(new Interval("2012-01-03/2012-01-05")).build())
.build();
final MergeTask testMergeTask = new MergeTask("foo", segments)
final MergeTaskBase testMergeTaskBase = new MergeTaskBase("foo", segments)
{
@Override
protected File merge(Map<DataSegment, File> segments, File outDir) throws Exception
@ -61,13 +61,13 @@ public class MergeTaskTest
@Test
public void testDataSource()
{
Assert.assertEquals("foo", testMergeTask.getDataSource());
Assert.assertEquals("foo", testMergeTaskBase.getDataSource());
}
@Test
public void testInterval()
{
Assert.assertEquals(new Interval("2012-01-03/2012-01-07"), testMergeTask.getImplicitLockInterval().get());
Assert.assertEquals(new Interval("2012-01-03/2012-01-07"), testMergeTaskBase.getImplicitLockInterval().get());
}
@Test
@ -81,7 +81,7 @@ public class MergeTaskTest
).toString().toLowerCase() + "_";
Assert.assertEquals(
desiredPrefix,
testMergeTask.getId().substring(0, desiredPrefix.length())
testMergeTaskBase.getId().substring(0, desiredPrefix.length())
);
}
}

View File

@ -11,13 +11,12 @@ import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.merger.TestTask;
import com.metamx.druid.merger.common.TaskCallback;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.config.IndexerZkConfig;
import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig;
import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
import com.metamx.druid.merger.worker.TaskMonitor;
import com.metamx.druid.merger.worker.Worker;
import com.metamx.druid.merger.worker.WorkerCuratorCoordinator;
@ -42,6 +41,7 @@ import org.junit.Test;
import java.io.File;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicReference;
import static junit.framework.Assert.fail;
@ -60,7 +60,6 @@ public class RemoteTaskRunnerTest
private PathChildrenCache pathChildrenCache;
private RemoteTaskRunner remoteTaskRunner;
private TaskMonitor taskMonitor;
private WorkerSetupManager workerSetupManager;
private ScheduledExecutorService scheduledExec;
@ -280,7 +279,7 @@ public class RemoteTaskRunnerTest
new PathChildrenCache(cf, String.format("%s/worker1", tasksPath), true),
cf,
workerCuratorCoordinator,
new TaskToolbox(
new TaskToolboxFactory(
new TaskConfig()
{
@Override
@ -316,17 +315,6 @@ public class RemoteTaskRunnerTest
private void makeRemoteTaskRunner() throws Exception
{
scheduledExec = EasyMock.createMock(ScheduledExecutorService.class);
workerSetupManager = EasyMock.createMock(WorkerSetupManager.class);
EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn(
new WorkerSetupData(
"0",
0,
null,
null
)
).atLeastOnce();
EasyMock.replay(workerSetupManager);
remoteTaskRunner = new RemoteTaskRunner(
jsonMapper,
@ -335,7 +323,7 @@ public class RemoteTaskRunnerTest
pathChildrenCache,
scheduledExec,
new RetryPolicyFactory(new TestRetryPolicyConfig()),
workerSetupManager
new AtomicReference<WorkerSetupData>(new WorkerSetupData("0", 0, null, null))
);
// Create a single worker and wait for things for be ready

View File

@ -1,3 +1,22 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.merger.coordinator;
import com.google.common.base.Optional;
@ -21,12 +40,13 @@ import com.metamx.druid.input.InputRow;
import com.metamx.druid.input.MapBasedInputRow;
import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.loading.SegmentKiller;
import com.metamx.druid.loading.DataSegmentKiller;
import com.metamx.druid.loading.SegmentLoadingException;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.actions.LocalTaskActionClient;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory;
import com.metamx.druid.merger.common.actions.LockAcquireAction;
import com.metamx.druid.merger.common.actions.LockListAction;
import com.metamx.druid.merger.common.actions.LockReleaseAction;
@ -56,10 +76,8 @@ import org.junit.Test;
import java.io.File;
import java.io.IOException;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.Executors;
@ -71,7 +89,7 @@ public class TaskLifecycleTest
private TaskQueue tq = null;
private TaskRunner tr = null;
private MockMergerDBCoordinator mdc = null;
private TaskToolbox tb = null;
private TaskToolboxFactory tb = null;
private TaskConsumer tc = null;
TaskStorageQueryAdapter tsqa = null;
@ -91,12 +109,12 @@ public class TaskLifecycleTest
tmp = Files.createTempDir();
ts = new LocalTaskStorage();
ts = new HeapMemoryTaskStorage();
tl = new TaskLockbox(ts);
tq = new TaskQueue(ts, tl);
mdc = newMockMDC();
tb = new TaskToolbox(
tb = new TaskToolboxFactory(
new TaskConfig()
{
@Override
@ -117,7 +135,7 @@ public class TaskLifecycleTest
return null;
}
},
new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, mdc, newMockEmitter())),
new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, mdc, newMockEmitter())),
newMockEmitter(),
null, // s3 client
new DataSegmentPusher()
@ -128,25 +146,16 @@ public class TaskLifecycleTest
return segment;
}
},
new SegmentKiller()
new DataSegmentKiller()
{
@Override
public void kill(Collection<DataSegment> segments) throws ServiceException
public void kill(DataSegment segments) throws SegmentLoadingException
{
}
},
new DefaultObjectMapper()
)
{
@Override
public Map<DataSegment, File> getSegments(
Task task, List<DataSegment> segments
) throws SegmentLoadingException
{
return ImmutableMap.of();
}
};
);
tr = new LocalTaskRunner(
tb,
@ -239,11 +248,12 @@ public class TaskLifecycleTest
@Test
public void testKillTask() throws Exception
{
// TODO: Worst test ever
// This test doesn't actually do anything right now. We should actually put things into the Mocked coordinator
// Such that this test can test things...
final Task killTask = new KillTask("foo", new Interval("2010-01-02/P2D"));
final TaskStatus mergedStatus = runTask(killTask);
Assert.assertEquals("merged statusCode", TaskStatus.Status.SUCCESS, mergedStatus.getStatusCode());
final TaskStatus status = runTask(killTask);
Assert.assertEquals("merged statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode());
Assert.assertEquals("num segments published", 0, mdc.getPublished().size());
Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size());
}
@ -273,8 +283,8 @@ public class TaskLifecycleTest
// Sort of similar to what realtime tasks do:
// Acquire lock for first interval
final Optional<TaskLock> lock1 = toolbox.getTaskActionClient().submit(new LockAcquireAction(this, interval1));
final List<TaskLock> locks1 = toolbox.getTaskActionClient().submit(new LockListAction(this));
final Optional<TaskLock> lock1 = toolbox.getTaskActionClientFactory().submit(new LockAcquireAction(interval1));
final List<TaskLock> locks1 = toolbox.getTaskActionClientFactory().submit(new LockListAction());
// (Confirm lock sanity)
Assert.assertTrue("lock1 present", lock1.isPresent());
@ -282,8 +292,8 @@ public class TaskLifecycleTest
Assert.assertEquals("locks1", ImmutableList.of(lock1.get()), locks1);
// Acquire lock for second interval
final Optional<TaskLock> lock2 = toolbox.getTaskActionClient().submit(new LockAcquireAction(this, interval2));
final List<TaskLock> locks2 = toolbox.getTaskActionClient().submit(new LockListAction(this));
final Optional<TaskLock> lock2 = toolbox.getTaskActionClientFactory().submit(new LockAcquireAction(interval2));
final List<TaskLock> locks2 = toolbox.getTaskActionClientFactory().submit(new LockListAction());
// (Confirm lock sanity)
Assert.assertTrue("lock2 present", lock2.isPresent());
@ -291,10 +301,9 @@ public class TaskLifecycleTest
Assert.assertEquals("locks2", ImmutableList.of(lock1.get(), lock2.get()), locks2);
// Push first segment
toolbox.getTaskActionClient()
toolbox.getTaskActionClientFactory()
.submit(
new SegmentInsertAction(
this,
ImmutableSet.of(
DataSegment.builder()
.dataSource("foo")
@ -306,17 +315,16 @@ public class TaskLifecycleTest
);
// Release first lock
toolbox.getTaskActionClient().submit(new LockReleaseAction(this, interval1));
final List<TaskLock> locks3 = toolbox.getTaskActionClient().submit(new LockListAction(this));
toolbox.getTaskActionClientFactory().submit(new LockReleaseAction(interval1));
final List<TaskLock> locks3 = toolbox.getTaskActionClientFactory().submit(new LockListAction());
// (Confirm lock sanity)
Assert.assertEquals("locks3", ImmutableList.of(lock2.get()), locks3);
// Push second segment
toolbox.getTaskActionClient()
toolbox.getTaskActionClientFactory()
.submit(
new SegmentInsertAction(
this,
ImmutableSet.of(
DataSegment.builder()
.dataSource("foo")
@ -328,8 +336,8 @@ public class TaskLifecycleTest
);
// Release second lock
toolbox.getTaskActionClient().submit(new LockReleaseAction(this, interval2));
final List<TaskLock> locks4 = toolbox.getTaskActionClient().submit(new LockListAction(this));
toolbox.getTaskActionClientFactory().submit(new LockReleaseAction(interval2));
final List<TaskLock> locks4 = toolbox.getTaskActionClientFactory().submit(new LockListAction());
// (Confirm lock sanity)
Assert.assertEquals("locks4", ImmutableList.<TaskLock>of(), locks4);
@ -362,8 +370,8 @@ public class TaskLifecycleTest
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
final TaskLock myLock = Iterables.getOnlyElement(
toolbox.getTaskActionClient()
.submit(new LockListAction(this))
toolbox.getTaskActionClientFactory()
.submit(new LockListAction())
);
final DataSegment segment = DataSegment.builder()
@ -372,7 +380,7 @@ public class TaskLifecycleTest
.version(myLock.getVersion())
.build();
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment)));
toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(segment)));
return TaskStatus.success(getId());
}
};
@ -398,10 +406,7 @@ public class TaskLifecycleTest
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
final TaskLock myLock = Iterables.getOnlyElement(
toolbox.getTaskActionClient()
.submit(new LockListAction(this))
);
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction()));
final DataSegment segment = DataSegment.builder()
.dataSource("ds")
@ -409,7 +414,7 @@ public class TaskLifecycleTest
.version(myLock.getVersion())
.build();
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment)));
toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(segment)));
return TaskStatus.success(getId());
}
};
@ -435,10 +440,7 @@ public class TaskLifecycleTest
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
final TaskLock myLock = Iterables.getOnlyElement(
toolbox.getTaskActionClient()
.submit(new LockListAction(this))
);
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction()));
final DataSegment segment = DataSegment.builder()
.dataSource("ds")
@ -446,7 +448,7 @@ public class TaskLifecycleTest
.version(myLock.getVersion() + "1!!!1!!")
.build();
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment)));
toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(segment)));
return TaskStatus.success(getId());
}
};

View File

@ -26,7 +26,8 @@ import com.google.common.collect.Sets;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.actions.LocalTaskActionClient;
import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory;
import com.metamx.druid.merger.common.actions.SpawnTasksAction;
import com.metamx.druid.merger.common.actions.TaskActionToolbox;
import com.metamx.druid.merger.common.task.AbstractTask;
@ -43,7 +44,7 @@ public class TaskQueueTest
@Test
public void testEmptyQueue() throws Exception
{
final TaskStorage ts = new LocalTaskStorage();
final TaskStorage ts = new HeapMemoryTaskStorage();
final TaskLockbox tl = new TaskLockbox(ts);
final TaskQueue tq = newTaskQueue(ts, tl);
@ -65,7 +66,7 @@ public class TaskQueueTest
@Test
public void testAddRemove() throws Exception
{
final TaskStorage ts = new LocalTaskStorage();
final TaskStorage ts = new HeapMemoryTaskStorage();
final TaskLockbox tl = new TaskLockbox(ts);
final TaskQueue tq = newTaskQueue(ts, tl);
@ -154,12 +155,12 @@ public class TaskQueueTest
@Test
public void testContinues() throws Exception
{
final TaskStorage ts = new LocalTaskStorage();
final TaskStorage ts = new HeapMemoryTaskStorage();
final TaskLockbox tl = new TaskLockbox(ts);
final TaskQueue tq = newTaskQueue(ts, tl);
final TaskToolbox tb = new TaskToolbox(
final TaskToolboxFactory tb = new TaskToolboxFactory(
null,
new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, null, null)),
new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, null, null)),
null,
null,
null,
@ -181,7 +182,7 @@ public class TaskQueueTest
Assert.assertNull("poll #2", tq.poll());
// report T1 done. Should cause T0 to be created
tq.notify(t1, t1.run(tb));
tq.notify(t1, t1.run(tb.build(t1)));
Assert.assertTrue("T0 isPresent (#2)", ts.getStatus("T0").isPresent());
Assert.assertTrue("T0 isRunnable (#2)", ts.getStatus("T0").get().isRunnable());
@ -195,7 +196,7 @@ public class TaskQueueTest
Assert.assertNull("poll #4", tq.poll());
// report T0 done. Should cause T0, T1 to be marked complete
tq.notify(t0, t0.run(tb));
tq.notify(t0, t0.run(tb.build(t0)));
Assert.assertTrue("T0 isPresent (#3)", ts.getStatus("T0").isPresent());
Assert.assertTrue("T0 isRunnable (#3)", !ts.getStatus("T0").get().isRunnable());
@ -211,12 +212,12 @@ public class TaskQueueTest
@Test
public void testConcurrency() throws Exception
{
final TaskStorage ts = new LocalTaskStorage();
final TaskStorage ts = new HeapMemoryTaskStorage();
final TaskLockbox tl = new TaskLockbox(ts);
final TaskQueue tq = newTaskQueue(ts, tl);
final TaskToolbox tb = new TaskToolbox(
final TaskToolboxFactory tb = new TaskToolboxFactory(
null,
new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, null, null)),
new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, null, null)),
null,
null,
null,
@ -248,7 +249,7 @@ public class TaskQueueTest
Thread.sleep(5);
// Finish t0
tq.notify(t0, t0.run(tb));
tq.notify(t0, t0.run(tb.build(t0)));
// take max number of tasks
final Set<String> taken = Sets.newHashSet();
@ -280,7 +281,7 @@ public class TaskQueueTest
Assert.assertNull("null poll #2", tq.poll());
// Finish t3
tq.notify(t3, t3.run(tb));
tq.notify(t3, t3.run(tb.build(t3)));
// We should be able to get t2 now
final Task wt2 = tq.poll();
@ -291,7 +292,7 @@ public class TaskQueueTest
Assert.assertNull("null poll #3", tq.poll());
// Finish t2
tq.notify(t2, t2.run(tb));
tq.notify(t2, t2.run(tb.build(t2)));
// We should be able to get t4
// And it should be in group G0, but that group should have a different version than last time
@ -305,14 +306,14 @@ public class TaskQueueTest
Assert.assertNotSame("wt4 version", wt2Lock.getVersion(), wt4Lock.getVersion());
// Kind of done testing at this point, but let's finish t4 anyway
tq.notify(t4, t4.run(tb));
tq.notify(t4, t4.run(tb.build(t4)));
Assert.assertNull("null poll #4", tq.poll());
}
@Test
public void testBootstrap() throws Exception
{
final TaskStorage storage = new LocalTaskStorage();
final TaskStorage storage = new HeapMemoryTaskStorage();
final TaskLockbox lockbox = new TaskLockbox(storage);
storage.insert(newTask("T1", "G1", "bar", new Interval("2011-01-01/P1D")), TaskStatus.running("T1"));
@ -374,7 +375,7 @@ public class TaskQueueTest
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, nextTasks));
toolbox.getTaskActionClientFactory().submit(new SpawnTasksAction(nextTasks));
return TaskStatus.success(id);
}
};

View File

@ -33,7 +33,6 @@ import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig;
import com.metamx.druid.merger.coordinator.setup.EC2NodeData;
import com.metamx.druid.merger.coordinator.setup.GalaxyUserData;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
import org.easymock.EasyMock;
import org.junit.After;
import org.junit.Assert;
@ -42,6 +41,7 @@ import org.junit.Test;
import java.util.Arrays;
import java.util.Date;
import java.util.concurrent.atomic.AtomicReference;
/**
*/
@ -57,7 +57,7 @@ public class EC2AutoScalingStrategyTest
private Reservation reservation;
private Instance instance;
private EC2AutoScalingStrategy strategy;
private WorkerSetupManager workerSetupManager;
private AtomicReference<WorkerSetupData> workerSetupData;
@Before
public void setUp() throws Exception
@ -66,7 +66,7 @@ public class EC2AutoScalingStrategyTest
runInstancesResult = EasyMock.createMock(RunInstancesResult.class);
describeInstancesResult = EasyMock.createMock(DescribeInstancesResult.class);
reservation = EasyMock.createMock(Reservation.class);
workerSetupManager = EasyMock.createMock(WorkerSetupManager.class);
workerSetupData = new AtomicReference<WorkerSetupData>(null);
instance = new Instance()
.withInstanceId(INSTANCE_ID)
@ -85,7 +85,7 @@ public class EC2AutoScalingStrategyTest
return "8080";
}
},
workerSetupManager
workerSetupData
);
}
@ -96,13 +96,12 @@ public class EC2AutoScalingStrategyTest
EasyMock.verify(runInstancesResult);
EasyMock.verify(describeInstancesResult);
EasyMock.verify(reservation);
EasyMock.verify(workerSetupManager);
}
@Test
public void testScale()
{
EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn(
workerSetupData.set(
new WorkerSetupData(
"0",
0,
@ -110,7 +109,6 @@ public class EC2AutoScalingStrategyTest
new GalaxyUserData("env", "version", "type")
)
);
EasyMock.replay(workerSetupManager);
EasyMock.expect(amazonEC2Client.runInstances(EasyMock.anyObject(RunInstancesRequest.class))).andReturn(
runInstancesResult

View File

@ -29,7 +29,6 @@ import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem;
import com.metamx.druid.merger.coordinator.ZkWorker;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
import com.metamx.druid.merger.worker.Worker;
import junit.framework.Assert;
import org.easymock.EasyMock;
@ -42,21 +41,22 @@ import org.junit.Test;
import java.util.Arrays;
import java.util.List;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
/**
*/
public class SimpleResourceManagementStrategyTest
{
private AutoScalingStrategy autoScalingStrategy;
private WorkerSetupManager workerSetupManager;
private Task testTask;
private SimpleResourceManagementStrategy simpleResourceManagementStrategy;
private AtomicReference<WorkerSetupData> workerSetupData;
@Before
public void setUp() throws Exception
{
workerSetupManager = EasyMock.createMock(WorkerSetupManager.class);
autoScalingStrategy = EasyMock.createMock(AutoScalingStrategy.class);
workerSetupData = new AtomicReference<WorkerSetupData>(null);
testTask = new TestTask(
"task1",
@ -105,7 +105,7 @@ public class SimpleResourceManagementStrategyTest
return new Duration(0);
}
},
workerSetupManager
workerSetupData
);
}
@ -187,8 +187,7 @@ public class SimpleResourceManagementStrategyTest
@Test
public void testDoSuccessfulTerminate() throws Exception
{
EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn(new WorkerSetupData("0", 0, null, null));
EasyMock.replay(workerSetupManager);
workerSetupData.set(new WorkerSetupData("0", 0, null, null));
EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.<List<String>>anyObject()))
.andReturn(Lists.<String>newArrayList());
@ -212,15 +211,13 @@ public class SimpleResourceManagementStrategyTest
simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.TERMINATE
);
EasyMock.verify(workerSetupManager);
EasyMock.verify(autoScalingStrategy);
}
@Test
public void testSomethingTerminating() throws Exception
{
EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn(new WorkerSetupData("0", 0, null, null));
EasyMock.replay(workerSetupManager);
workerSetupData.set(new WorkerSetupData("0", 0, null, null));
EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.<List<String>>anyObject()))
.andReturn(Lists.<String>newArrayList("ip")).times(2);
@ -259,7 +256,6 @@ public class SimpleResourceManagementStrategyTest
simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.TERMINATE
);
EasyMock.verify(workerSetupManager);
EasyMock.verify(autoScalingStrategy);
}

12
pom.xml
View File

@ -38,7 +38,7 @@
<properties>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<metamx.java-util.version>0.20.0</metamx.java-util.version>
<metamx.java-util.version>0.21.0</metamx.java-util.version>
</properties>
<modules>
@ -165,17 +165,17 @@
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-annotations</artifactId>
<version>2.1.2</version>
<version>2.1.4</version>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-core</artifactId>
<version>2.1.3</version>
<version>2.1.4</version>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
<version>2.1.4-mmx-2</version>
<version>2.1.4</version>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.datatype</groupId>
@ -190,12 +190,12 @@
<dependency>
<groupId>com.fasterxml.jackson.dataformat</groupId>
<artifactId>jackson-dataformat-smile</artifactId>
<version>2.1.3</version>
<version>2.1.4</version>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.jaxrs</groupId>
<artifactId>jackson-jaxrs-json-provider</artifactId>
<version>2.1.3</version>
<version>2.1.4</version>
</dependency>
<dependency>
<groupId>org.codehaus.jackson</groupId>

View File

@ -45,25 +45,15 @@ import com.metamx.druid.db.DbConnectorConfig;
import com.metamx.druid.http.QueryServlet;
import com.metamx.druid.http.StatusServlet;
import com.metamx.druid.initialization.Initialization;
import com.metamx.druid.initialization.ServerInit;
import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.loading.LocalDataSegmentPusher;
import com.metamx.druid.loading.LocalDataSegmentPusherConfig;
import com.metamx.druid.loading.S3DataSegmentPusher;
import com.metamx.druid.loading.S3DataSegmentPusherConfig;
import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
import com.metamx.druid.utils.PropUtils;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.metrics.Monitor;
import org.jets3t.service.S3ServiceException;
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
import org.jets3t.service.security.AWSCredentials;
import org.mortbay.jetty.servlet.Context;
import org.mortbay.jetty.servlet.ServletHolder;
import org.skife.config.ConfigurationObjectFactory;
@ -259,31 +249,7 @@ public class RealtimeNode extends BaseServerNode<RealtimeNode>
private void initializeSegmentPusher()
{
if (dataSegmentPusher == null) {
final Properties props = getProps();
if (Boolean.parseBoolean(props.getProperty("druid.pusher.local", "false"))) {
dataSegmentPusher = new LocalDataSegmentPusher(
getConfigFactory().build(LocalDataSegmentPusherConfig.class), getJsonMapper()
);
}
else {
final RestS3Service s3Client;
try {
s3Client = new RestS3Service(
new AWSCredentials(
PropUtils.getProperty(props, "com.metamx.aws.accessKey"),
PropUtils.getProperty(props, "com.metamx.aws.secretKey")
)
);
}
catch (S3ServiceException e) {
throw Throwables.propagate(e);
}
dataSegmentPusher = new S3DataSegmentPusher(
s3Client, getConfigFactory().build(S3DataSegmentPusherConfig.class), getJsonMapper()
);
}
dataSegmentPusher = ServerInit.getSegmentPusher(getProps(), getConfigFactory(), getJsonMapper());
}
}

View File

@ -34,8 +34,8 @@ import com.metamx.druid.coordination.DruidClusterInfo;
import com.metamx.druid.db.DatabaseRuleManager;
import com.metamx.druid.db.DatabaseSegmentManager;
import com.metamx.druid.master.DruidMaster;
import com.metamx.druid.client.indexing.IndexingServiceClient;
import com.metamx.druid.master.rules.Rule;
import com.metamx.druid.merge.ClientKillQuery;
import org.joda.time.Interval;
import javax.annotation.Nullable;
@ -66,6 +66,7 @@ public class InfoResource
private final DatabaseSegmentManager databaseSegmentManager;
private final DatabaseRuleManager databaseRuleManager;
private final DruidClusterInfo druidClusterInfo;
private final IndexingServiceClient indexingServiceClient;
@Inject
public InfoResource(
@ -73,7 +74,8 @@ public class InfoResource
ServerInventoryManager serverInventoryManager,
DatabaseSegmentManager databaseSegmentManager,
DatabaseRuleManager databaseRuleManager,
DruidClusterInfo druidClusterInfo
DruidClusterInfo druidClusterInfo,
IndexingServiceClient indexingServiceClient
)
{
this.master = master;
@ -81,6 +83,7 @@ public class InfoResource
this.databaseSegmentManager = databaseSegmentManager;
this.databaseRuleManager = databaseRuleManager;
this.druidClusterInfo = druidClusterInfo;
this.indexingServiceClient = indexingServiceClient;
}
@GET
@ -374,7 +377,7 @@ public class InfoResource
)
{
if (kill != null && Boolean.valueOf(kill)) {
master.killSegments(new ClientKillQuery(dataSourceName, new Interval(interval)));
indexingServiceClient.killSegments(dataSourceName, new Interval(interval));
} else {
if (!databaseSegmentManager.removeDatasource(dataSourceName)) {
return Response.status(Response.Status.NOT_FOUND).build();

View File

@ -33,6 +33,9 @@ import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.ServerInventoryManager;
import com.metamx.druid.client.ServerInventoryManagerConfig;
import com.metamx.druid.config.ConfigManager;
import com.metamx.druid.config.ConfigManagerConfig;
import com.metamx.druid.config.JacksonConfigManager;
import com.metamx.druid.coordination.DruidClusterInfo;
import com.metamx.druid.coordination.DruidClusterInfoConfig;
import com.metamx.druid.db.DatabaseRuleManager;
@ -49,6 +52,7 @@ import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.log.LogLevelAdjuster;
import com.metamx.druid.master.DruidMaster;
import com.metamx.druid.master.DruidMasterConfig;
import com.metamx.druid.client.indexing.IndexingServiceClient;
import com.metamx.druid.master.LoadQueuePeon;
import com.metamx.druid.utils.PropUtils;
import com.metamx.emitter.EmittingLogger;
@ -86,7 +90,7 @@ import java.util.concurrent.ScheduledExecutorService;
*/
public class MasterMain
{
private static final Logger log = new Logger(ServerMain.class);
private static final Logger log = new Logger(MasterMain.class);
public static void main(String[] args) throws Exception
{
@ -166,13 +170,14 @@ public class MasterMain
lifecycle
);
ServiceProvider serviceProvider = null;
IndexingServiceClient indexingServiceClient = null;
if (druidMasterConfig.getMergerServiceName() != null) {
serviceProvider = Initialization.makeServiceProvider(
ServiceProvider serviceProvider = Initialization.makeServiceProvider(
druidMasterConfig.getMergerServiceName(),
serviceDiscovery,
lifecycle
);
indexingServiceClient = new IndexingServiceClient(httpClient, jsonMapper, serviceProvider);
}
final DruidClusterInfo druidClusterInfo = new DruidClusterInfo(
@ -180,10 +185,14 @@ public class MasterMain
masterYp
);
JacksonConfigManager configManager = new JacksonConfigManager(
new ConfigManager(dbi, configFactory.build(ConfigManagerConfig.class)), jsonMapper
);
final DruidMaster master = new DruidMaster(
druidMasterConfig,
druidClusterInfo,
jsonMapper,
configManager,
databaseSegmentManager,
serverInventoryManager,
databaseRuleManager,
@ -191,9 +200,7 @@ public class MasterMain
emitter,
scheduledExecutorFactory,
new ConcurrentHashMap<String, LoadQueuePeon>(),
serviceProvider,
httpClient,
new ToStringResponseHandler(Charsets.UTF_8)
indexingServiceClient
);
lifecycle.addManagedInstance(master);
@ -226,7 +233,8 @@ public class MasterMain
databaseRuleManager,
druidClusterInfo,
master,
jsonMapper
jsonMapper,
indexingServiceClient
)
);

View File

@ -21,7 +21,6 @@ package com.metamx.druid.http;
import com.metamx.druid.master.DruidMaster;
import com.metamx.druid.master.LoadPeonCallback;
import com.metamx.druid.merge.ClientKillQuery;
import javax.inject.Inject;
import javax.ws.rs.Consumes;

View File

@ -27,6 +27,7 @@ import com.metamx.druid.coordination.DruidClusterInfo;
import com.metamx.druid.db.DatabaseRuleManager;
import com.metamx.druid.db.DatabaseSegmentManager;
import com.metamx.druid.master.DruidMaster;
import com.metamx.druid.client.indexing.IndexingServiceClient;
import com.sun.jersey.guice.JerseyServletModule;
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
@ -44,6 +45,7 @@ public class MasterServletModule extends JerseyServletModule
private final DruidClusterInfo druidClusterInfo;
private final DruidMaster master;
private final ObjectMapper jsonMapper;
private final IndexingServiceClient indexingServiceClient;
public MasterServletModule(
ServerInventoryManager serverInventoryManager,
@ -51,7 +53,8 @@ public class MasterServletModule extends JerseyServletModule
DatabaseRuleManager databaseRuleManager,
DruidClusterInfo druidClusterInfo,
DruidMaster master,
ObjectMapper jsonMapper
ObjectMapper jsonMapper,
IndexingServiceClient indexingServiceClient
)
{
this.serverInventoryManager = serverInventoryManager;
@ -60,6 +63,7 @@ public class MasterServletModule extends JerseyServletModule
this.druidClusterInfo = druidClusterInfo;
this.master = master;
this.jsonMapper = jsonMapper;
this.indexingServiceClient = indexingServiceClient;
}
@Override
@ -72,6 +76,7 @@ public class MasterServletModule extends JerseyServletModule
bind(DatabaseRuleManager.class).toInstance(databaseRuleManager);
bind(DruidMaster.class).toInstance(master);
bind(DruidClusterInfo.class).toInstance(druidClusterInfo);
bind(IndexingServiceClient.class).toInstance(indexingServiceClient);
serve("/*").with(GuiceContainer.class);
}

View File

@ -19,17 +19,24 @@
package com.metamx.druid.initialization;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Supplier;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import com.metamx.common.ISE;
import com.metamx.common.logger.Logger;
import com.metamx.druid.DruidProcessingConfig;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.loading.DelegatingSegmentLoader;
import com.metamx.druid.loading.LocalDataSegmentPuller;
import com.metamx.druid.loading.LocalDataSegmentPusher;
import com.metamx.druid.loading.LocalDataSegmentPusherConfig;
import com.metamx.druid.loading.MMappedQueryableIndexFactory;
import com.metamx.druid.loading.QueryableIndexFactory;
import com.metamx.druid.loading.S3DataSegmentPuller;
import com.metamx.druid.loading.S3DataSegmentPusher;
import com.metamx.druid.loading.S3DataSegmentPusherConfig;
import com.metamx.druid.loading.SegmentLoaderConfig;
import com.metamx.druid.loading.SingleSegmentLoader;
import com.metamx.druid.query.group.GroupByQueryEngine;
@ -48,12 +55,16 @@ import com.metamx.druid.query.timeboundary.TimeBoundaryQuery;
import com.metamx.druid.query.timeboundary.TimeBoundaryQueryRunnerFactory;
import com.metamx.druid.query.timeseries.TimeseriesQuery;
import com.metamx.druid.query.timeseries.TimeseriesQueryRunnerFactory;
import com.metamx.druid.utils.PropUtils;
import org.jets3t.service.S3ServiceException;
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
import org.jets3t.service.security.AWSCredentials;
import org.skife.config.ConfigurationObjectFactory;
import java.lang.reflect.InvocationTargetException;
import java.nio.ByteBuffer;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.atomic.AtomicLong;
/**
@ -145,6 +156,34 @@ public class ServerInit
return queryRunners;
}
public static DataSegmentPusher getSegmentPusher(
final Properties props,
final ConfigurationObjectFactory configFactory,
final ObjectMapper jsonMapper
)
{
if (Boolean.parseBoolean(props.getProperty("druid.pusher.local", "false"))) {
return new LocalDataSegmentPusher(configFactory.build(LocalDataSegmentPusherConfig.class), jsonMapper);
}
else {
final RestS3Service s3Client;
try {
s3Client = new RestS3Service(
new AWSCredentials(
PropUtils.getProperty(props, "com.metamx.aws.accessKey"),
PropUtils.getProperty(props, "com.metamx.aws.secretKey")
)
);
}
catch (S3ServiceException e) {
throw Throwables.propagate(e);
}
return new S3DataSegmentPusher(s3Client, configFactory.build(S3DataSegmentPusherConfig.class), jsonMapper);
}
}
private static class ComputeScratchPool extends StupidPool<ByteBuffer>
{
private static final Logger log = new Logger(ComputeScratchPool.class);

View File

@ -17,15 +17,13 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.master;
package com.metamx.druid.loading;
import com.metamx.druid.client.DataSegment;
import java.util.List;
/**
*/
public interface MergerClient
public interface DataSegmentKiller
{
public void runRequest(String dataSource, List<DataSegment> segments);
public void kill(DataSegment segments) throws SegmentLoadingException;
}

View File

@ -0,0 +1,70 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.loading;
import com.google.inject.Inject;
import com.metamx.common.MapUtils;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment;
import org.jets3t.service.ServiceException;
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
import java.util.Map;
/**
*/
public class S3DataSegmentKiller implements DataSegmentKiller
{
private static final Logger log = new Logger(S3DataSegmentKiller.class);
private final RestS3Service s3Client;
@Inject
public S3DataSegmentKiller(
RestS3Service s3Client
)
{
this.s3Client = s3Client;
}
@Override
public void kill(DataSegment segment) throws SegmentLoadingException
{
try {
Map<String, Object> loadSpec = segment.getLoadSpec();
String s3Bucket = MapUtils.getString(loadSpec, "bucket");
String s3Path = MapUtils.getString(loadSpec, "key");
String s3DescriptorPath = s3Path.substring(0, s3Path.lastIndexOf("/")) + "/descriptor.json";
if (s3Client.isObjectInBucket(s3Bucket, s3Path)) {
log.info("Removing index file[s3://%s/%s] from s3!", s3Bucket, s3Path);
s3Client.deleteObject(s3Bucket, s3Path);
}
if (s3Client.isObjectInBucket(s3Bucket, s3DescriptorPath)) {
log.info("Removing descriptor file[s3://%s/%s] from s3!", s3Bucket, s3DescriptorPath);
s3Client.deleteObject(s3Bucket, s3DescriptorPath);
}
}
catch (ServiceException e) {
throw new SegmentLoadingException(e, "Couldn't kill segment[%s]", segment.getIdentifier());
}
}
}

View File

@ -87,11 +87,7 @@ public class S3DataSegmentPusher implements DataSegmentPusher
segment = segment.withSize(indexSize)
.withLoadSpec(
ImmutableMap.<String, Object>of(
"type", "s3_zip",
"bucket", outputBucket,
"key", toPush.getKey()
)
ImmutableMap.<String, Object>of("type", "s3_zip", "bucket", outputBucket, "key", toPush.getKey())
)
.withBinaryVersion(IndexIO.getVersionFromDir(indexFilesDir));

View File

@ -1,49 +0,0 @@
package com.metamx.druid.loading;
import com.google.inject.Inject;
import com.metamx.common.MapUtils;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment;
import org.jets3t.service.ServiceException;
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
import java.util.Collection;
import java.util.Map;
/**
*/
public class S3SegmentKiller implements SegmentKiller
{
private static final Logger log = new Logger(S3SegmentKiller.class);
private final RestS3Service s3Client;
@Inject
public S3SegmentKiller(
RestS3Service s3Client
)
{
this.s3Client = s3Client;
}
@Override
public void kill(Collection<DataSegment> segments) throws ServiceException
{
for (final DataSegment segment : segments) {
Map<String, Object> loadSpec = segment.getLoadSpec();
String s3Bucket = MapUtils.getString(loadSpec, "bucket");
String s3Path = MapUtils.getString(loadSpec, "key");
String s3DescriptorPath = s3Path.substring(0, s3Path.lastIndexOf("/")) + "/descriptor.json";
if (s3Client.isObjectInBucket(s3Bucket, s3Path)) {
log.info("Removing index file[s3://%s/%s] from s3!", s3Bucket, s3Path);
s3Client.deleteObject(s3Bucket, s3Path);
}
if (s3Client.isObjectInBucket(s3Bucket, s3DescriptorPath)) {
log.info("Removing descriptor file[s3://%s/%s] from s3!", s3Bucket, s3DescriptorPath);
s3Client.deleteObject(s3Bucket, s3DescriptorPath);
}
}
}
}

View File

@ -1,14 +0,0 @@
package com.metamx.druid.loading;
import com.metamx.druid.client.DataSegment;
import org.jets3t.service.ServiceException;
import java.util.Collection;
import java.util.List;
/**
*/
public interface SegmentKiller
{
public void kill(Collection<DataSegment> segments) throws ServiceException;
}

View File

@ -19,10 +19,8 @@
package com.metamx.druid.master;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Predicate;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
@ -41,32 +39,30 @@ import com.metamx.druid.client.DataSegment;
import com.metamx.druid.client.DruidDataSource;
import com.metamx.druid.client.DruidServer;
import com.metamx.druid.client.ServerInventoryManager;
import com.metamx.druid.client.indexing.IndexingServiceClient;
import com.metamx.druid.config.JacksonConfigManager;
import com.metamx.druid.coordination.DruidClusterInfo;
import com.metamx.druid.db.DatabaseRuleManager;
import com.metamx.druid.db.DatabaseSegmentManager;
import com.metamx.druid.merge.ClientKillQuery;
import com.metamx.druid.index.v1.IndexIO;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent;
import com.metamx.http.client.HttpClient;
import com.metamx.http.client.response.HttpResponseHandler;
import com.metamx.phonebook.PhoneBook;
import com.metamx.phonebook.PhoneBookPeon;
import com.netflix.curator.x.discovery.ServiceProvider;
import org.I0Itec.zkclient.exception.ZkNodeExistsException;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import javax.annotation.Nullable;
import java.net.URL;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.Callable;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicReference;
/**
*/
@ -83,26 +79,22 @@ public class DruidMaster
private final DruidMasterConfig config;
private final DruidClusterInfo clusterInfo;
private final JacksonConfigManager configManager;
private final DatabaseSegmentManager databaseSegmentManager;
private final ServerInventoryManager serverInventoryManager;
private final DatabaseRuleManager databaseRuleManager;
private final PhoneBook yp;
private final ServiceEmitter emitter;
private final IndexingServiceClient indexingServiceClient;
private final ScheduledExecutorService exec;
private final ScheduledExecutorService peonExec;
private final PhoneBookPeon masterPeon;
private final Map<String, LoadQueuePeon> loadManagementPeons;
private final ServiceProvider serviceProvider;
private final HttpClient httpClient;
private final HttpResponseHandler responseHandler;
private final ObjectMapper jsonMapper;
public DruidMaster(
DruidMasterConfig config,
DruidClusterInfo clusterInfo,
ObjectMapper jsonMapper,
JacksonConfigManager configManager,
DatabaseSegmentManager databaseSegmentManager,
ServerInventoryManager serverInventoryManager,
DatabaseRuleManager databaseRuleManager,
@ -110,31 +102,25 @@ public class DruidMaster
ServiceEmitter emitter,
ScheduledExecutorFactory scheduledExecutorFactory,
Map<String, LoadQueuePeon> loadManagementPeons,
ServiceProvider serviceProvider,
HttpClient httpClient,
HttpResponseHandler responseHandler
IndexingServiceClient indexingServiceClient
)
{
this.config = config;
this.clusterInfo = clusterInfo;
this.jsonMapper = jsonMapper;
this.configManager = configManager;
this.databaseSegmentManager = databaseSegmentManager;
this.serverInventoryManager = serverInventoryManager;
this.databaseRuleManager = databaseRuleManager;
this.yp = zkPhoneBook;
this.emitter = emitter;
this.indexingServiceClient = indexingServiceClient;
this.masterPeon = new MasterListeningPeon();
this.exec = scheduledExecutorFactory.create(1, "Master-Exec--%d");
this.peonExec = scheduledExecutorFactory.create(1, "Master-PeonExec--%d");
this.loadManagementPeons = loadManagementPeons;
this.serviceProvider = serviceProvider;
this.httpClient = httpClient;
this.responseHandler = responseHandler;
}
public boolean isClusterMaster()
@ -349,27 +335,6 @@ public class DruidMaster
}
}
public void killSegments(ClientKillQuery killQuery)
{
try {
httpClient.post(
new URL(
String.format(
"http://%s:%s/mmx/merger/v1/index",
serviceProvider.getInstance().getAddress(),
serviceProvider.getInstance().getPort()
)
)
)
.setContent("application/json", jsonMapper.writeValueAsBytes(killQuery))
.go(responseHandler)
.get();
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
public Set<DataSegment> getAvailableDataSegments()
{
Set<DataSegment> availableSegments = Sets.newTreeSet(Comparators.inverse(DataSegment.bucketMonthComparator()));
@ -390,7 +355,9 @@ public class DruidMaster
for (DataSegment dataSegment : dataSegments) {
if (dataSegment.getSize() < 0) {
log.warn("No size on Segment[%s], wtf?", dataSegment);
log.makeAlert("No size on Segment, wtf?")
.addData("segment", dataSegment)
.emit();
}
availableSegments.add(dataSegment);
}
@ -466,8 +433,14 @@ public class DruidMaster
final List<Pair<? extends MasterRunnable, Duration>> masterRunnables = Lists.newArrayList();
masterRunnables.add(Pair.of(new MasterComputeManagerRunnable(), config.getMasterPeriod()));
if (config.isMergeSegments() && serviceProvider != null) {
masterRunnables.add(Pair.of(new MasterSegmentMergerRunnable(), config.getMasterSegmentMergerPeriod()));
if (config.isMergeSegments() && indexingServiceClient != null) {
masterRunnables.add(
Pair.of(
new MasterSegmentMergerRunnable(configManager.watch(MergerWhitelist.CONFIG_KEY, MergerWhitelist.class)),
config.getMasterSegmentMergerPeriod()
)
);
}
for (final Pair<? extends MasterRunnable, Duration> masterRunnable : masterRunnables) {
@ -529,6 +502,39 @@ public class DruidMaster
}
}
public static class DruidMasterVersionConverter implements DruidMasterHelper
{
private final IndexingServiceClient indexingServiceClient;
private final AtomicReference<MergerWhitelist> whitelistRef;
public DruidMasterVersionConverter(
IndexingServiceClient indexingServiceClient,
AtomicReference<MergerWhitelist> whitelistRef
)
{
this.indexingServiceClient = indexingServiceClient;
this.whitelistRef = whitelistRef;
}
@Override
public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params)
{
MergerWhitelist whitelist = whitelistRef.get();
for (DataSegment dataSegment : params.getAvailableSegments()) {
if (whitelist == null || whitelist.contains(dataSegment.getDataSource())) {
final Integer binaryVersion = dataSegment.getBinaryVersion();
if (binaryVersion == null || binaryVersion < IndexIO.CURRENT_VERSION_ID) {
indexingServiceClient.upgradeSegment(dataSegment);
}
}
}
return params;
}
}
private class MasterListeningPeon implements PhoneBookPeon<Map>
{
@Override
@ -723,12 +729,13 @@ public class DruidMaster
private class MasterSegmentMergerRunnable extends MasterRunnable
{
private MasterSegmentMergerRunnable()
private MasterSegmentMergerRunnable(final AtomicReference<MergerWhitelist> whitelistRef)
{
super(
ImmutableList.of(
new DruidMasterSegmentInfoLoader(DruidMaster.this),
new DruidMasterSegmentMerger(jsonMapper, serviceProvider),
new DruidMasterVersionConverter(indexingServiceClient, whitelistRef),
new DruidMasterSegmentMerger(indexingServiceClient, whitelistRef),
new DruidMasterHelper()
{
@Override
@ -739,8 +746,7 @@ public class DruidMaster
params.getEmitter().emit(
new ServiceMetricEvent.Builder().build(
"master/merge/count",
stats.getGlobalStats().get("mergedCount")
"master/merge/count", stats.getGlobalStats().get("mergedCount")
)
);

View File

@ -19,8 +19,6 @@
package com.metamx.druid.master;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Charsets;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.collect.HashMultiset;
@ -32,22 +30,19 @@ import com.google.common.collect.Multiset;
import com.google.common.collect.Ordering;
import com.metamx.common.Pair;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.logger.Logger;
import com.metamx.druid.TimelineObjectHolder;
import com.metamx.druid.VersionedIntervalTimeline;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.client.indexing.IndexingServiceClient;
import com.metamx.druid.partition.PartitionChunk;
import com.metamx.http.client.HttpClientConfig;
import com.metamx.http.client.HttpClientInit;
import com.metamx.http.client.response.ToStringResponseHandler;
import com.netflix.curator.x.discovery.ServiceProvider;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicReference;
/**
*/
@ -55,44 +50,40 @@ public class DruidMasterSegmentMerger implements DruidMasterHelper
{
private static final Logger log = new Logger(DruidMasterSegmentMerger.class);
private final MergerClient mergerClient;
private final IndexingServiceClient indexingServiceClient;
private final AtomicReference<MergerWhitelist> whiteListRef;
public DruidMasterSegmentMerger(MergerClient mergerClient)
public DruidMasterSegmentMerger(
IndexingServiceClient indexingServiceClient,
AtomicReference<MergerWhitelist> whitelistRef
)
{
this.mergerClient = mergerClient;
}
public DruidMasterSegmentMerger(ObjectMapper jsonMapper, ServiceProvider serviceProvider)
{
this.mergerClient = new HttpMergerClient(
HttpClientInit.createClient(
HttpClientConfig.builder().withNumConnections(1).build(),
new Lifecycle()
),
new ToStringResponseHandler(Charsets.UTF_8),
jsonMapper,
serviceProvider
);
this.indexingServiceClient = indexingServiceClient;
this.whiteListRef = whitelistRef;
}
@Override
public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params)
{
MergerWhitelist whitelist = whiteListRef.get();
MasterStats stats = new MasterStats();
Map<String, VersionedIntervalTimeline<String, DataSegment>> dataSources = Maps.newHashMap();
// Find serviced segments by using a timeline
for (DataSegment dataSegment : params.getAvailableSegments()) {
VersionedIntervalTimeline<String, DataSegment> timeline = dataSources.get(dataSegment.getDataSource());
if (timeline == null) {
timeline = new VersionedIntervalTimeline<String, DataSegment>(Ordering.<String>natural());
dataSources.put(dataSegment.getDataSource(), timeline);
if (whitelist == null || whitelist.contains(dataSegment.getDataSource())) {
VersionedIntervalTimeline<String, DataSegment> timeline = dataSources.get(dataSegment.getDataSource());
if (timeline == null) {
timeline = new VersionedIntervalTimeline<String, DataSegment>(Ordering.<String>natural());
dataSources.put(dataSegment.getDataSource(), timeline);
}
timeline.add(
dataSegment.getInterval(),
dataSegment.getVersion(),
dataSegment.getShardSpec().createChunk(dataSegment)
);
}
timeline.add(
dataSegment.getInterval(),
dataSegment.getVersion(),
dataSegment.getShardSpec().createChunk(dataSegment)
);
}
// Find segments to merge
@ -161,7 +152,7 @@ public class DruidMasterSegmentMerger implements DruidMasterHelper
log.info("[%s] Found %d segments to merge %s", dataSource, segments.size(), segmentNames);
try {
mergerClient.runRequest(dataSource, segments);
indexingServiceClient.mergeSegments(segments);
}
catch (Exception e) {
log.error(

View File

@ -1,78 +0,0 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.master;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merge.ClientAppendQuery;
import com.metamx.http.client.HttpClient;
import com.metamx.http.client.response.HttpResponseHandler;
import com.netflix.curator.x.discovery.ServiceProvider;
import java.net.URL;
import java.util.List;
public class HttpMergerClient implements MergerClient
{
private final HttpClient client;
private final HttpResponseHandler<StringBuilder, String> responseHandler;
private final ObjectMapper jsonMapper;
private final ServiceProvider serviceProvider;
public HttpMergerClient(
HttpClient client,
HttpResponseHandler<StringBuilder, String> responseHandler,
ObjectMapper jsonMapper,
ServiceProvider serviceProvider
)
{
this.client = client;
this.responseHandler = responseHandler;
this.jsonMapper = jsonMapper;
this.serviceProvider = serviceProvider;
}
public void runRequest(String dataSource, List<DataSegment> segments)
{
try {
byte[] dataToSend = jsonMapper.writeValueAsBytes(
new ClientAppendQuery(dataSource, segments)
);
client.post(
new URL(
String.format(
"http://%s:%s/mmx/merger/v1/merge",
serviceProvider.getInstance().getAddress(),
serviceProvider.getInstance().getPort()
)
)
)
.setContent("application/json", dataToSend)
.go(responseHandler)
.get();
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
}

View File

@ -0,0 +1,54 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 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 com.metamx.druid.master;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonValue;
import com.google.common.collect.Sets;
import java.util.List;
import java.util.Set;
/**
*/
public class MergerWhitelist
{
public static final String CONFIG_KEY = "merger.whitelist";
private final Set<String> dataSources;
@JsonCreator
public MergerWhitelist(Set<String> dataSources)
{
this.dataSources = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER);
this.dataSources.addAll(dataSources);
}
@JsonValue
public Set<String> getDataSources()
{
return dataSources;
}
public boolean contains(String val)
{
return dataSources.contains(val);
}
}

View File

@ -23,12 +23,14 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.client.indexing.IndexingServiceClient;
import junit.framework.Assert;
import org.joda.time.Interval;
import org.junit.Test;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.atomic.AtomicReference;
public class DruidMasterSegmentMergerTest
{
@ -367,11 +369,7 @@ public class DruidMasterSegmentMergerTest
DataSegment.builder().dataSource("foo").interval(new Interval("2012-01-06/P1D")).version("2").size(80).build()
);
Assert.assertEquals(
ImmutableList.of(
ImmutableList.of(segments.get(4), segments.get(5))
), merge(segments)
);
Assert.assertEquals(ImmutableList.of(ImmutableList.of(segments.get(4), segments.get(5))), merge(segments));
}
/**
@ -380,16 +378,17 @@ public class DruidMasterSegmentMergerTest
private static List<List<DataSegment>> merge(final Collection<DataSegment> segments)
{
final List<List<DataSegment>> retVal = Lists.newArrayList();
final MergerClient mergerClient = new MergerClient()
final IndexingServiceClient indexingServiceClient = new IndexingServiceClient(null, null, null)
{
@Override
public void runRequest(String dataSource, List<DataSegment> segmentsToMerge)
public void mergeSegments(List<DataSegment> segmentsToMerge)
{
retVal.add(segmentsToMerge);
}
};
final DruidMasterSegmentMerger merger = new DruidMasterSegmentMerger(mergerClient);
final AtomicReference<MergerWhitelist> whitelistRef = new AtomicReference<MergerWhitelist>(null);
final DruidMasterSegmentMerger merger = new DruidMasterSegmentMerger(indexingServiceClient, whitelistRef);
final DruidMasterRuntimeParams params = DruidMasterRuntimeParams.newBuilder()
.withAvailableSegments(ImmutableSet.copyOf(segments))
.withMergeBytesLimit(mergeBytesLimit)

View File

@ -152,8 +152,6 @@ public class DruidMasterTest
new NoopServiceEmitter(),
scheduledExecutorFactory,
loadManagementPeons,
null,
null,
null
);
}