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 + ", loadSpec=" + loadSpec +
", interval=" + interval + ", interval=" + interval +
", dataSource='" + dataSource + '\'' + ", dataSource='" + dataSource + '\'' +
", binaryVersion='" + binaryVersion + '\'' +
'}'; '}';
} }

View File

@ -17,7 +17,7 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. * 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.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; 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 String dataSource;
private final List<DataSegment> segments; private final List<DataSegment> segments;
@ -45,14 +45,18 @@ public class ClientAppendQuery implements ClientMergeQuery
} }
@JsonProperty @JsonProperty
@Override public String getType()
{
return "append";
}
@JsonProperty
public String getDataSource() public String getDataSource()
{ {
return dataSource; return dataSource;
} }
@JsonProperty @JsonProperty
@Override
public List<DataSegment> getSegments() public List<DataSegment> getSegments()
{ {
return segments; 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. * 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.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import org.joda.time.Interval; import org.joda.time.Interval;
public class ClientDeleteQuery /**
*/
public class ClientKillQuery
{ {
private final String dataSource; private final String dataSource;
private final Interval interval; private final Interval interval;
@JsonCreator @JsonCreator
public ClientDeleteQuery( public ClientKillQuery(
@JsonProperty("dataSource") String dataSource, @JsonProperty("dataSource") String dataSource,
@JsonProperty("interval") Interval interval @JsonProperty("interval") Interval interval
) )
@ -40,22 +40,21 @@ public class ClientDeleteQuery
this.interval = interval; this.interval = interval;
} }
@JsonProperty
public String getType()
{
return "kill";
}
@JsonProperty
public String getDataSource() public String getDataSource()
{ {
return dataSource; return dataSource;
} }
@JsonProperty
public Interval getInterval() public Interval getInterval()
{ {
return interval; 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. * 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.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; 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 String dataSource;
private final List<DataSegment> segments; private final List<DataSegment> segments;
private final List<AggregatorFactory> aggregators; private final List<AggregatorFactory> aggregators;
@JsonCreator @JsonCreator
public ClientDefaultMergeQuery( public ClientMergeQuery(
@JsonProperty("dataSource") String dataSource, @JsonProperty("dataSource") String dataSource,
@JsonProperty("segments") List<DataSegment> segments, @JsonProperty("segments") List<DataSegment> segments,
@JsonProperty("aggregations") List<AggregatorFactory> aggregators @JsonProperty("aggregations") List<AggregatorFactory> aggregators
@ -50,14 +50,18 @@ public class ClientDefaultMergeQuery implements ClientMergeQuery
} }
@JsonProperty @JsonProperty
@Override public String getType()
{
return "append";
}
@JsonProperty
public String getDataSource() public String getDataSource()
{ {
return dataSource; return dataSource;
} }
@JsonProperty @JsonProperty
@Override
public List<DataSegment> getSegments() public List<DataSegment> getSegments()
{ {
return segments; return segments;
@ -72,7 +76,7 @@ public class ClientDefaultMergeQuery implements ClientMergeQuery
@Override @Override
public String toString() public String toString()
{ {
return "ClientDefaultMergeQuery{" + return "ClientMergeQuery{" +
"dataSource='" + dataSource + '\'' + "dataSource='" + dataSource + '\'' +
", segments=" + segments + ", segments=" + segments +
", aggregators=" + aggregators + ", 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, dbi,
configTableName, configTableName,
String.format( 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 configTableName
) )
); );
@ -84,12 +84,7 @@ public class DbConnector
@Override @Override
public Void withHandle(Handle handle) throws Exception public Void withHandle(Handle handle) throws Exception
{ {
List<Map<String, Object>> table = handle.select( List<Map<String, Object>> table = handle.select(String.format("SHOW tables LIKE '%s'", tableName));
String.format(
"SHOW tables LIKE '%s'",
tableName
)
);
if (table.isEmpty()) { if (table.isEmpty()) {
log.info("Creating table[%s]", tableName); 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 public static interface IndexIOHandler
{ {
/** /**
@ -229,7 +242,7 @@ public class IndexIO
public void storeLatest(Index index, File file) throws IOException; 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); private static final Logger log = new Logger(DefaultIndexIOHandler.class);
@Override @Override

View File

@ -25,11 +25,12 @@ import com.metamx.druid.client.DataSegment;
import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.loading.MMappedQueryableIndexFactory; import com.metamx.druid.loading.MMappedQueryableIndexFactory;
import com.metamx.druid.loading.S3DataSegmentPuller; 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.SegmentLoaderConfig;
import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.loading.SegmentLoadingException;
import com.metamx.druid.loading.SingleSegmentLoader; import com.metamx.druid.loading.SingleSegmentLoader;
import com.metamx.druid.merger.common.actions.TaskActionClient; 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.config.TaskConfig;
import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.common.task.Task;
import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEmitter;
@ -45,29 +46,32 @@ import java.util.Map;
public class TaskToolbox public class TaskToolbox
{ {
private final TaskConfig config; private final TaskConfig config;
private final TaskActionClient taskActionClient; private final Task task;
private final TaskActionClientFactory taskActionClientFactory;
private final ServiceEmitter emitter; private final ServiceEmitter emitter;
private final RestS3Service s3Client; private final RestS3Service s3Client;
private final DataSegmentPusher segmentPusher; private final DataSegmentPusher segmentPusher;
private final SegmentKiller segmentKiller; private final DataSegmentKiller dataSegmentKiller;
private final ObjectMapper objectMapper; private final ObjectMapper objectMapper;
public TaskToolbox( public TaskToolbox(
TaskConfig config, TaskConfig config,
TaskActionClient taskActionClient, Task task,
TaskActionClientFactory taskActionClientFactory,
ServiceEmitter emitter, ServiceEmitter emitter,
RestS3Service s3Client, RestS3Service s3Client,
DataSegmentPusher segmentPusher, DataSegmentPusher segmentPusher,
SegmentKiller segmentKiller, DataSegmentKiller dataSegmentKiller,
ObjectMapper objectMapper ObjectMapper objectMapper
) )
{ {
this.config = config; this.config = config;
this.taskActionClient = taskActionClient; this.task = task;
this.taskActionClientFactory = taskActionClientFactory;
this.emitter = emitter; this.emitter = emitter;
this.s3Client = s3Client; this.s3Client = s3Client;
this.segmentPusher = segmentPusher; this.segmentPusher = segmentPusher;
this.segmentKiller = segmentKiller; this.dataSegmentKiller = dataSegmentKiller;
this.objectMapper = objectMapper; this.objectMapper = objectMapper;
} }
@ -76,9 +80,9 @@ public class TaskToolbox
return config; return config;
} }
public TaskActionClient getTaskActionClient() public TaskActionClient getTaskActionClientFactory()
{ {
return taskActionClient; return taskActionClientFactory.create(task);
} }
public ServiceEmitter getEmitter() public ServiceEmitter getEmitter()
@ -91,9 +95,9 @@ public class TaskToolbox
return segmentPusher; return segmentPusher;
} }
public SegmentKiller getSegmentKiller() public DataSegmentKiller getDataSegmentKiller()
{ {
return segmentKiller; return dataSegmentKiller;
} }
public ObjectMapper getObjectMapper() public ObjectMapper getObjectMapper()
@ -101,7 +105,7 @@ public class TaskToolbox
return objectMapper; return objectMapper;
} }
public Map<DataSegment, File> getSegments(final Task task, List<DataSegment> segments) public Map<DataSegment, File> getSegments(List<DataSegment> segments)
throws SegmentLoadingException throws SegmentLoadingException
{ {
final SingleSegmentLoader loader = new SingleSegmentLoader( final SingleSegmentLoader loader = new SingleSegmentLoader(
@ -112,7 +116,7 @@ public class TaskToolbox
@Override @Override
public File getCacheDirectory() 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; 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; 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.druid.merger.coordinator.TaskStorage;
import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.EmittingLogger;
public class LocalTaskActionClient implements TaskActionClient public class LocalTaskActionClient implements TaskActionClient
{ {
private final Task task;
private final TaskStorage storage; private final TaskStorage storage;
private final TaskActionToolbox toolbox; private final TaskActionToolbox toolbox;
private static final EmittingLogger log = new EmittingLogger(LocalTaskActionClient.class); 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.storage = storage;
this.toolbox = toolbox; this.toolbox = toolbox;
} }
@ -19,15 +22,15 @@ public class LocalTaskActionClient implements TaskActionClient
@Override @Override
public <RetType> RetType submit(TaskAction<RetType> taskAction) public <RetType> RetType submit(TaskAction<RetType> taskAction)
{ {
final RetType ret = taskAction.perform(toolbox); final RetType ret = taskAction.perform(task, toolbox);
// Add audit log // Add audit log
try { try {
storage.addAuditLog(taskAction); storage.addAuditLog(task, taskAction);
} }
catch (Exception e) { catch (Exception e) {
log.makeAlert(e, "Failed to record action in audit log") log.makeAlert(e, "Failed to record action in audit log")
.addData("task", taskAction.getTask().getId()) .addData("task", task.getId())
.addData("actionClass", taskAction.getClass().getName()) .addData("actionClass", taskAction.getClass().getName())
.emit(); .emit();
} }

View File

@ -17,25 +17,27 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. * 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.metamx.druid.merger.common.task.Task;
import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.druid.merger.coordinator.TaskStorage;
import com.metamx.druid.client.DataSegment;
import java.util.List;
/** /**
*/ */
@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type", defaultImpl = ClientDefaultMergeQuery.class) public class LocalTaskActionClientFactory implements TaskActionClientFactory
@JsonSubTypes(value={
@JsonSubTypes.Type(name="append", value=ClientAppendQuery.class)
})
public interface ClientMergeQuery
{ {
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>> public class LockAcquireAction implements TaskAction<Optional<TaskLock>>
{ {
private final Task task;
private final Interval interval; private final Interval interval;
@JsonCreator @JsonCreator
public LockAcquireAction( public LockAcquireAction(
@JsonProperty("task") Task task,
@JsonProperty("interval") Interval interval @JsonProperty("interval") Interval interval
) )
{ {
this.task = task;
this.interval = interval; this.interval = interval;
} }
@JsonProperty
public Task getTask()
{
return task;
}
@JsonProperty @JsonProperty
public Interval getInterval() public Interval getInterval()
{ {
@ -42,7 +33,7 @@ public class LockAcquireAction implements TaskAction<Optional<TaskLock>>
} }
@Override @Override
public Optional<TaskLock> perform(TaskActionToolbox toolbox) public Optional<TaskLock> perform(Task task, TaskActionToolbox toolbox)
{ {
try { try {
return toolbox.getTaskLockbox().tryLock(task, interval); return toolbox.getTaskLockbox().tryLock(task, interval);

View File

@ -12,29 +12,13 @@ import java.util.List;
public class LockListAction implements TaskAction<List<TaskLock>> 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() public TypeReference<List<TaskLock>> getReturnTypeReference()
{ {
return new TypeReference<List<TaskLock>>() {}; return new TypeReference<List<TaskLock>>() {};
} }
@Override @Override
public List<TaskLock> perform(TaskActionToolbox toolbox) public List<TaskLock> perform(Task task, TaskActionToolbox toolbox)
{ {
try { try {
return toolbox.getTaskLockbox().findLocksForTask(task); return toolbox.getTaskLockbox().findLocksForTask(task);

View File

@ -12,25 +12,16 @@ import java.util.List;
public class LockReleaseAction implements TaskAction<Void> public class LockReleaseAction implements TaskAction<Void>
{ {
private final Task task;
private final Interval interval; private final Interval interval;
@JsonCreator @JsonCreator
public LockReleaseAction( public LockReleaseAction(
@JsonProperty("task") Task task,
@JsonProperty("interval") Interval interval @JsonProperty("interval") Interval interval
) )
{ {
this.task = task;
this.interval = interval; this.interval = interval;
} }
@JsonProperty
public Task getTask()
{
return task;
}
@JsonProperty @JsonProperty
public Interval getInterval() public Interval getInterval()
{ {
@ -43,7 +34,7 @@ public class LockReleaseAction implements TaskAction<Void>
} }
@Override @Override
public Void perform(TaskActionToolbox toolbox) public Void perform(Task task, TaskActionToolbox toolbox)
{ {
try { try {
toolbox.getTaskLockbox().unlock(task, interval); 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.Charsets;
import com.google.common.base.Throwables; import com.google.common.base.Throwables;
import com.metamx.common.logger.Logger; 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.HttpClient;
import com.metamx.http.client.response.ToStringResponseHandler; import com.metamx.http.client.response.ToStringResponseHandler;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
@ -16,14 +17,16 @@ import java.util.Map;
public class RemoteTaskActionClient implements TaskActionClient public class RemoteTaskActionClient implements TaskActionClient
{ {
private final Task task;
private final HttpClient httpClient; private final HttpClient httpClient;
private final ServiceProvider serviceProvider; private final ServiceProvider serviceProvider;
private final ObjectMapper jsonMapper; private final ObjectMapper jsonMapper;
private static final Logger log = new Logger(RemoteTaskActionClient.class); 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.httpClient = httpClient;
this.serviceProvider = serviceProvider; this.serviceProvider = serviceProvider;
this.jsonMapper = jsonMapper; this.jsonMapper = jsonMapper;
@ -33,7 +36,7 @@ public class RemoteTaskActionClient implements TaskActionClient
public <RetType> RetType submit(TaskAction<RetType> taskAction) public <RetType> RetType submit(TaskAction<RetType> taskAction)
{ {
try { try {
byte[] dataToSend = jsonMapper.writeValueAsBytes(taskAction); byte[] dataToSend = jsonMapper.writeValueAsBytes(new TaskActionHolder(task, taskAction));
final String response = httpClient.post(getServiceUri().toURL()) final String response = httpClient.post(getServiceUri().toURL())
.setContent("application/json", dataToSend) .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> public class SegmentInsertAction implements TaskAction<Void>
{ {
private final Task task;
private final Set<DataSegment> segments; private final Set<DataSegment> segments;
@JsonCreator @JsonCreator
public SegmentInsertAction( public SegmentInsertAction(
@JsonProperty("task") Task task,
@JsonProperty("segments") Set<DataSegment> segments @JsonProperty("segments") Set<DataSegment> segments
) )
{ {
this.task = task;
this.segments = ImmutableSet.copyOf(segments); this.segments = ImmutableSet.copyOf(segments);
} }
@JsonProperty
public Task getTask()
{
return task;
}
@JsonProperty @JsonProperty
public Set<DataSegment> getSegments() public Set<DataSegment> getSegments()
{ {
@ -49,7 +40,7 @@ public class SegmentInsertAction implements TaskAction<Void>
} }
@Override @Override
public Void perform(TaskActionToolbox toolbox) public Void perform(Task task, TaskActionToolbox toolbox)
{ {
if(!toolbox.taskLockCoversSegments(task, segments, false)) { if(!toolbox.taskLockCoversSegments(task, segments, false)) {
throw new ISE("Segments not covered by locks for task: %s", task.getId()); 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>> public class SegmentListUnusedAction implements TaskAction<List<DataSegment>>
{ {
private final Task task;
private final String dataSource; private final String dataSource;
private final Interval interval; private final Interval interval;
@JsonCreator @JsonCreator
public SegmentListUnusedAction( public SegmentListUnusedAction(
@JsonProperty("task") Task task,
@JsonProperty("dataSource") String dataSource, @JsonProperty("dataSource") String dataSource,
@JsonProperty("interval") Interval interval @JsonProperty("interval") Interval interval
) )
{ {
this.task = task;
this.dataSource = dataSource; this.dataSource = dataSource;
this.interval = interval; this.interval = interval;
} }
@JsonProperty
public Task getTask()
{
return task;
}
@JsonProperty @JsonProperty
public String getDataSource() public String getDataSource()
{ {
@ -52,7 +43,7 @@ public class SegmentListUnusedAction implements TaskAction<List<DataSegment>>
} }
@Override @Override
public List<DataSegment> perform(TaskActionToolbox toolbox) public List<DataSegment> perform(Task task, TaskActionToolbox toolbox)
{ {
try { try {
return toolbox.getMergerDBCoordinator().getUnusedSegmentsForInterval(dataSource, interval); return toolbox.getMergerDBCoordinator().getUnusedSegmentsForInterval(dataSource, interval);

View File

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

View File

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

View File

@ -18,7 +18,6 @@ import com.fasterxml.jackson.core.type.TypeReference;
}) })
public interface TaskAction<RetType> 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 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. * 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 com.metamx.druid.merger.common.task.Task;
import org.skife.config.Config;
import org.skife.config.Default;
/** /**
*/ */
public abstract class WorkerSetupManagerConfig public interface TaskActionClientFactory
{ {
@Config("druid.indexer.configTable") public TaskActionClient create(Task task);
public abstract String getConfigTable();
@Config("druid.indexer.workerSetupConfigName")
public abstract String getWorkerSetupConfigName();
@Config("druid.indexer.poll.duration")
@Default("PT1M")
public abstract Duration getPollDuration();
} }

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; package com.metamx.druid.merger.common.config;
import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.common.task.Task;
@ -17,8 +36,4 @@ public abstract class TaskConfig
@Config("druid.merger.hadoopWorkingPath") @Config("druid.merger.hadoopWorkingPath")
public abstract String getHadoopWorkingPath(); 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; package com.metamx.druid.merger.common.task;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Joiner;
import com.google.common.base.Objects; import com.google.common.base.Objects;
import com.google.common.base.Optional; import com.google.common.base.Optional;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.actions.SegmentListUsedAction;
import org.joda.time.Interval; import org.joda.time.Interval;
public abstract class AbstractTask implements Task public abstract class AbstractTask implements Task
{ {
private static final Joiner ID_JOINER = Joiner.on("_");
private final String id; private final String id;
private final String groupId; private final String groupId;
private final String dataSource; private final String dataSource;
@ -91,4 +95,20 @@ public abstract class AbstractTask implements Task
.add("interval", getImplicitLockInterval()) .add("interval", getImplicitLockInterval())
.toString(); .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 @JsonCreator
public AppendTask( 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 public TaskStatus run(TaskToolbox toolbox) throws Exception
{ {
// Strategy: Create an empty segment covering the interval to be deleted // 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 Interval interval = this.getImplicitLockInterval().get();
final IncrementalIndex empty = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]); final IncrementalIndex empty = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]);
final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(interval, empty); final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(interval, empty);
@ -91,7 +91,7 @@ public class DeleteTask extends AbstractTask
.shardSpec(new NoneShardSpec()) .shardSpec(new NoneShardSpec())
.build(); .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); final File fileToUpload = IndexMerger.merge(Lists.newArrayList(emptyAdapter), new AggregatorFactory[0], outDir);
// Upload the segment // Upload the segment
@ -104,7 +104,7 @@ public class DeleteTask extends AbstractTask
segment.getVersion() segment.getVersion()
); );
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(uploadedSegment))); toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment)));
return TaskStatus.success(getId()); 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; package com.metamx.druid.merger.common.task;
import com.fasterxml.jackson.annotation.JsonCreator; 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 // 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()); log.info("Setting version to: %s", myLock.getVersion());
configCopy.setVersion(myLock.getVersion()); configCopy.setVersion(myLock.getVersion());
@ -105,7 +124,7 @@ public class HadoopIndexTask extends AbstractTask
List<DataSegment> publishedSegments = job.getPublishedSegments(); List<DataSegment> publishedSegments = job.getPublishedSegments();
// Request segment pushes // Request segment pushes
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.copyOf(publishedSegments))); toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.copyOf(publishedSegments)));
// Done // Done
return TaskStatus.success(getId()); 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()); return TaskStatus.success(getId());
} }

View File

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

View File

@ -142,7 +142,7 @@ public class IndexTask extends AbstractTask
@Override @Override
public TaskStatus preflight(TaskToolbox toolbox) throws Exception public TaskStatus preflight(TaskToolbox toolbox) throws Exception
{ {
toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, toSubtasks())); toolbox.getTaskActionClientFactory().submit(new SpawnTasksAction(toSubtasks()));
return TaskStatus.success(getId()); 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; package com.metamx.druid.merger.common.task;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
@ -53,7 +72,7 @@ public class KillTask extends AbstractTask
public TaskStatus run(TaskToolbox toolbox) throws Exception public TaskStatus run(TaskToolbox toolbox) throws Exception
{ {
// Confirm we have a lock (will throw if there isn't exactly one element) // 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())) { if(!myLock.getDataSource().equals(getDataSource())) {
throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), 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 // List unused segments
final List<DataSegment> unusedSegments = toolbox.getTaskActionClient() final List<DataSegment> unusedSegments = toolbox
.submit( .getTaskActionClientFactory()
new SegmentListUnusedAction( .submit(new SegmentListUnusedAction(myLock.getDataSource(), myLock.getInterval()));
this,
myLock.getDataSource(),
myLock.getInterval()
)
);
// Verify none of these segments have versions > lock version // Verify none of these segments have versions > lock version
for(final DataSegment unusedSegment : unusedSegments) { for(final DataSegment unusedSegment : unusedSegments) {
@ -88,10 +102,12 @@ public class KillTask extends AbstractTask
} }
// Kill segments // Kill segments
toolbox.getSegmentKiller().kill(unusedSegments); for (DataSegment segment : unusedSegments) {
toolbox.getDataSegmentKiller().kill(segment);
}
// Remove metadata for these segments // 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()); return TaskStatus.success(getId());
} }

View File

@ -19,318 +19,71 @@
package com.metamx.druid.merger.common.task; package com.metamx.druid.merger.common.task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; 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.Function;
import com.google.common.base.Joiner; import com.google.common.base.Throwables;
import com.google.common.base.Objects; import com.google.common.collect.ImmutableList;
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.Lists;
import com.google.common.collect.Ordering; import com.metamx.druid.aggregation.AggregatorFactory;
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.client.DataSegment;
import com.metamx.druid.merger.common.TaskLock; import com.metamx.druid.index.QueryableIndex;
import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.index.v1.IndexIO;
import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.index.v1.IndexMerger;
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 javax.annotation.Nullable; import javax.annotation.Nullable;
import java.io.File; import java.io.File;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set;
/** /**
*/ */
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultMergeTask.class) public class MergeTask extends MergeTaskBase
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "append", value = AppendTask.class)
})
public abstract class MergeTask extends AbstractTask
{ {
private final List<DataSegment> segments; private final List<AggregatorFactory> aggregators;
private static final Logger log = new Logger(MergeTask.class); @JsonCreator
public MergeTask(
protected MergeTask(final String dataSource, final List<DataSegment> segments) @JsonProperty("dataSource") String dataSource,
{ @JsonProperty("segments") List<DataSegment> segments,
super( @JsonProperty("aggregations") List<AggregatorFactory> aggregators
// _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
) )
{ {
final Interval mergedInterval = computeMergedInterval(segments); super(dataSource, segments);
final Set<String> mergedDimensions = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); this.aggregators = aggregators;
final Set<String> mergedMetrics = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); }
for (DataSegment segment : segments) { @Override
mergedDimensions.addAll(segment.getDimensions()); public File merge(final Map<DataSegment, File> segments, final File outDir)
mergedMetrics.addAll(segment.getMetrics()); 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() @Override
.dataSource(dataSource) public String getType()
.interval(mergedInterval) {
.version(version) return "merge";
.shardSpec(new NoneShardSpec())
.dimensions(Lists.newArrayList(mergedDimensions))
.metrics(Lists.newArrayList(mergedMetrics))
.build();
} }
} }

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> * to release locks early if they desire.</li>
* </ul> * </ul>
*/ */
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultMergeTask.class) @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonSubTypes(value = { @JsonSubTypes(value = {
@JsonSubTypes.Type(name = "append", value = AppendTask.class), @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 = "delete", value = DeleteTask.class),
@JsonSubTypes.Type(name = "kill", value = KillTask.class), @JsonSubTypes.Type(name = "kill", value = KillTask.class),
@JsonSubTypes.Type(name = "index", value = IndexTask.class), @JsonSubTypes.Type(name = "index", value = IndexTask.class),
@JsonSubTypes.Type(name = "index_partitions", value = IndexDeterminePartitionsTask.class), @JsonSubTypes.Type(name = "index_partitions", value = IndexDeterminePartitionsTask.class),
@JsonSubTypes.Type(name = "index_generator", value = IndexGeneratorTask.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 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 @Override
public <T> void addAuditLog(final TaskAction<T> taskAction) public <T> void addAuditLog(final Task task, final TaskAction<T> taskAction)
{ {
Preconditions.checkNotNull(taskAction, "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( dbi.withHandle(
new HandleCallback<Integer>() new HandleCallback<Integer>()
@ -345,7 +345,7 @@ public class DbTaskStorage implements TaskStorage
dbConnectorConfig.getTaskLogTable() dbConnectorConfig.getTaskLogTable()
) )
) )
.bind("task_id", taskAction.getTask().getId()) .bind("task_id", task.getId())
.bind("log_payload", jsonMapper.writeValueAsString(taskAction)) .bind("log_payload", jsonMapper.writeValueAsString(taskAction))
.execute(); .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 * Implements an in-heap TaskStorage facility, with no persistence across restarts. This class is not
* thread safe. * thread safe.
*/ */
public class LocalTaskStorage implements TaskStorage public class HeapMemoryTaskStorage implements TaskStorage
{ {
private final ReentrantLock giant = new ReentrantLock(); private final ReentrantLock giant = new ReentrantLock();
private final Map<String, TaskStuff> tasks = Maps.newHashMap(); private final Map<String, TaskStuff> tasks = Maps.newHashMap();
private final Multimap<String, TaskLock> taskLocks = HashMultimap.create(); private final Multimap<String, TaskLock> taskLocks = HashMultimap.create();
private final Multimap<String, TaskAction> taskActions = ArrayListMultimap.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 @Override
public void insert(Task task, TaskStatus status) public void insert(Task task, TaskStatus status)
@ -185,12 +185,12 @@ public class LocalTaskStorage implements TaskStorage
} }
@Override @Override
public <T> void addAuditLog(TaskAction<T> taskAction) public <T> void addAuditLog(Task task, TaskAction<T> taskAction)
{ {
giant.lock(); giant.lock();
try { try {
taskActions.put(taskAction.getTask().getId(), taskAction); taskActions.put(task.getId(), taskAction);
} finally { } finally {
giant.unlock(); 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.TaskCallback;
import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox; 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.common.task.Task;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
@ -38,17 +39,17 @@ import java.util.concurrent.ExecutorService;
*/ */
public class LocalTaskRunner implements TaskRunner public class LocalTaskRunner implements TaskRunner
{ {
private final TaskToolbox toolbox; private final TaskToolboxFactory toolboxFactory;
private final ExecutorService exec; private final ExecutorService exec;
private static final Logger log = new Logger(LocalTaskRunner.class); private static final Logger log = new Logger(LocalTaskRunner.class);
public LocalTaskRunner( public LocalTaskRunner(
TaskToolbox toolbox, TaskToolboxFactory toolboxFactory,
ExecutorService exec ExecutorService exec
) )
{ {
this.toolbox = toolbox; this.toolboxFactory = toolboxFactory;
this.exec = exec; this.exec = exec;
} }
@ -61,6 +62,8 @@ public class LocalTaskRunner implements TaskRunner
@Override @Override
public void run(final Task task, final TaskCallback callback) public void run(final Task task, final TaskCallback callback)
{ {
final TaskToolbox toolbox = toolboxFactory.build(task);
exec.submit( exec.submit(
new Runnable() new Runnable()
{ {
@ -89,7 +92,7 @@ public class LocalTaskRunner implements TaskRunner
} }
try { try {
final File taskDir = toolbox.getConfig().getTaskDir(task); final File taskDir = toolbox.getTaskDir();
if (taskDir.exists()) { if (taskDir.exists()) {
log.info("Removing task directory: %s", taskDir); 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.TaskStatus;
import com.metamx.druid.merger.common.task.Task; import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; 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.druid.merger.worker.Worker;
import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.EmittingLogger;
import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.framework.CuratorFramework;
@ -57,6 +57,7 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit; 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 * 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 PathChildrenCache workerPathCache;
private final ScheduledExecutorService scheduledExec; private final ScheduledExecutorService scheduledExec;
private final RetryPolicyFactory retryPolicyFactory; private final RetryPolicyFactory retryPolicyFactory;
private final WorkerSetupManager workerSetupManager; private final AtomicReference<WorkerSetupData> workerSetupData;
// all workers that exist in ZK // all workers that exist in ZK
private final Map<String, ZkWorker> zkWorkers = new ConcurrentHashMap<String, ZkWorker>(); private final Map<String, ZkWorker> zkWorkers = new ConcurrentHashMap<String, ZkWorker>();
@ -104,7 +105,7 @@ public class RemoteTaskRunner implements TaskRunner
PathChildrenCache workerPathCache, PathChildrenCache workerPathCache,
ScheduledExecutorService scheduledExec, ScheduledExecutorService scheduledExec,
RetryPolicyFactory retryPolicyFactory, RetryPolicyFactory retryPolicyFactory,
WorkerSetupManager workerSetupManager AtomicReference<WorkerSetupData> workerSetupData
) )
{ {
this.jsonMapper = jsonMapper; this.jsonMapper = jsonMapper;
@ -113,7 +114,7 @@ public class RemoteTaskRunner implements TaskRunner
this.workerPathCache = workerPathCache; this.workerPathCache = workerPathCache;
this.scheduledExec = scheduledExec; this.scheduledExec = scheduledExec;
this.retryPolicyFactory = retryPolicyFactory; this.retryPolicyFactory = retryPolicyFactory;
this.workerSetupManager = workerSetupManager; this.workerSetupData = workerSetupData;
} }
@LifecycleStart @LifecycleStart
@ -548,7 +549,7 @@ public class RemoteTaskRunner implements TaskRunner
return (!input.isAtCapacity() && return (!input.isAtCapacity() &&
input.getWorker() input.getWorker()
.getVersion() .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; package com.metamx.druid.merger.coordinator;
import com.google.common.base.Function; 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.Initialization;
import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.initialization.ServiceDiscoveryConfig;
import com.metamx.druid.merger.common.TaskToolbox; 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.config.IndexerCoordinatorConfig;
import com.metamx.druid.merger.coordinator.exec.TaskConsumer; import com.metamx.druid.merger.coordinator.exec.TaskConsumer;
import com.metamx.druid.merger.coordinator.scaling.ResourceManagementScheduler; import com.metamx.druid.merger.coordinator.scaling.ResourceManagementScheduler;
@ -49,7 +51,7 @@ public class TaskMasterLifecycle
private final ReentrantLock giant = new ReentrantLock(); private final ReentrantLock giant = new ReentrantLock();
private final Condition mayBeStopped = giant.newCondition(); private final Condition mayBeStopped = giant.newCondition();
private final TaskQueue taskQueue; private final TaskQueue taskQueue;
private final TaskToolbox taskToolbox; private final TaskToolboxFactory taskToolboxFactory;
private volatile boolean leading = false; private volatile boolean leading = false;
private volatile TaskRunner taskRunner; private volatile TaskRunner taskRunner;
@ -59,7 +61,7 @@ public class TaskMasterLifecycle
public TaskMasterLifecycle( public TaskMasterLifecycle(
final TaskQueue taskQueue, final TaskQueue taskQueue,
final TaskToolbox taskToolbox, final TaskToolboxFactory taskToolboxFactory,
final IndexerCoordinatorConfig indexerCoordinatorConfig, final IndexerCoordinatorConfig indexerCoordinatorConfig,
final ServiceDiscoveryConfig serviceDiscoveryConfig, final ServiceDiscoveryConfig serviceDiscoveryConfig,
final TaskRunnerFactory runnerFactory, final TaskRunnerFactory runnerFactory,
@ -69,7 +71,7 @@ public class TaskMasterLifecycle
) )
{ {
this.taskQueue = taskQueue; this.taskQueue = taskQueue;
this.taskToolbox = taskToolbox; this.taskToolboxFactory = taskToolboxFactory;
this.leaderSelector = new LeaderSelector( this.leaderSelector = new LeaderSelector(
curator, indexerCoordinatorConfig.getLeaderLatchPath(), new LeaderSelectorListener() curator, indexerCoordinatorConfig.getLeaderLatchPath(), new LeaderSelectorListener()
@ -87,7 +89,7 @@ public class TaskMasterLifecycle
final TaskConsumer taskConsumer = new TaskConsumer( final TaskConsumer taskConsumer = new TaskConsumer(
taskQueue, taskQueue,
taskRunner, taskRunner,
taskToolbox, taskToolboxFactory,
emitter emitter
); );
@ -221,9 +223,4 @@ public class TaskMasterLifecycle
{ {
return taskToolbox; 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. * 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. * 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.common.lifecycle.LifecycleStop;
import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskCallback;
import com.metamx.druid.merger.common.TaskStatus; 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.common.task.Task;
import com.metamx.druid.merger.coordinator.TaskQueue; import com.metamx.druid.merger.coordinator.TaskQueue;
import com.metamx.druid.merger.coordinator.TaskRunner; import com.metamx.druid.merger.coordinator.TaskRunner;
@ -36,7 +36,7 @@ public class TaskConsumer implements Runnable
{ {
private final TaskQueue queue; private final TaskQueue queue;
private final TaskRunner runner; private final TaskRunner runner;
private final TaskToolbox toolbox; private final TaskToolboxFactory toolboxFactory;
private final ServiceEmitter emitter; private final ServiceEmitter emitter;
private final Thread thready; private final Thread thready;
@ -47,13 +47,13 @@ public class TaskConsumer implements Runnable
public TaskConsumer( public TaskConsumer(
TaskQueue queue, TaskQueue queue,
TaskRunner runner, TaskRunner runner,
TaskToolbox toolbox, TaskToolboxFactory toolboxFactory,
ServiceEmitter emitter ServiceEmitter emitter
) )
{ {
this.queue = queue; this.queue = queue;
this.runner = runner; this.runner = runner;
this.toolbox = toolbox; this.toolboxFactory = toolboxFactory;
this.emitter = emitter; this.emitter = emitter;
this.thready = new Thread(this); this.thready = new Thread(this);
} }
@ -123,7 +123,7 @@ public class TaskConsumer implements Runnable
// Run preflight checks // Run preflight checks
TaskStatus preflightStatus; TaskStatus preflightStatus;
try { try {
preflightStatus = task.preflight(toolbox); preflightStatus = task.preflight(toolboxFactory.build(task));
log.info("Preflight done for task: %s", task.getId()); log.info("Preflight done for task: %s", task.getId());
} }
catch (Exception e) { 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.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger; import com.metamx.common.logger.Logger;
import com.metamx.druid.RegisteringNode; 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.DbConnector;
import com.metamx.druid.db.DbConnectorConfig; import com.metamx.druid.db.DbConnectorConfig;
import com.metamx.druid.http.GuiceServletConfig; 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.http.StatusServlet;
import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.Initialization;
import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ServerConfig;
import com.metamx.druid.initialization.ServerInit;
import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.initialization.ServiceDiscoveryConfig;
import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.loading.DataSegmentKiller;
import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.loading.S3DataSegmentPusher; import com.metamx.druid.loading.S3DataSegmentKiller;
import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.loading.S3SegmentKiller; import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory;
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.merger.common.actions.TaskActionToolbox; import com.metamx.druid.merger.common.actions.TaskActionToolbox;
import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.config.IndexerZkConfig;
import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory;
import com.metamx.druid.merger.coordinator.DbTaskStorage; 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.LocalTaskRunner;
import com.metamx.druid.merger.coordinator.LocalTaskStorage;
import com.metamx.druid.merger.coordinator.MergerDBCoordinator; import com.metamx.druid.merger.coordinator.MergerDBCoordinator;
import com.metamx.druid.merger.coordinator.RemoteTaskRunner; import com.metamx.druid.merger.coordinator.RemoteTaskRunner;
import com.metamx.druid.merger.coordinator.RetryPolicyFactory; 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.IndexerDbConnectorConfig;
import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig;
import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; 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.AutoScalingStrategy;
import com.metamx.druid.merger.coordinator.scaling.EC2AutoScalingStrategy; import com.metamx.druid.merger.coordinator.scaling.EC2AutoScalingStrategy;
import com.metamx.druid.merger.coordinator.scaling.NoopAutoScalingStrategy; 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.ResourceManagementSchedulerFactory;
import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagementStrategy; import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagementStrategy;
import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagmentConfig; 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.druid.utils.PropUtils;
import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.core.Emitters; import com.metamx.emitter.core.Emitters;
@ -123,6 +124,7 @@ import java.util.Properties;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService; 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 RestS3Service s3Service = null;
private IndexerCoordinatorConfig config = null; private IndexerCoordinatorConfig config = null;
private TaskConfig taskConfig = null; private TaskConfig taskConfig = null;
private TaskToolbox taskToolbox = null; private DataSegmentPusher segmentPusher = null;
private TaskToolboxFactory taskToolboxFactory = null;
private MergerDBCoordinator mergerDBCoordinator = null; private MergerDBCoordinator mergerDBCoordinator = null;
private TaskStorage taskStorage = null; private TaskStorage taskStorage = null;
private TaskQueue taskQueue = null; private TaskQueue taskQueue = null;
@ -155,7 +158,6 @@ public class IndexerCoordinatorNode extends RegisteringNode
private CuratorFramework curatorFramework = null; private CuratorFramework curatorFramework = null;
private ScheduledExecutorFactory scheduledExecutorFactory = null; private ScheduledExecutorFactory scheduledExecutorFactory = null;
private IndexerZkConfig indexerZkConfig; private IndexerZkConfig indexerZkConfig;
private WorkerSetupManager workerSetupManager = null;
private TaskRunnerFactory taskRunnerFactory = null; private TaskRunnerFactory taskRunnerFactory = null;
private ResourceManagementSchedulerFactory resourceManagementSchedulerFactory = null; private ResourceManagementSchedulerFactory resourceManagementSchedulerFactory = null;
private TaskMasterLifecycle taskMasterLifecycle = null; private TaskMasterLifecycle taskMasterLifecycle = null;
@ -208,6 +210,12 @@ public class IndexerCoordinatorNode extends RegisteringNode
return this; return this;
} }
public IndexerCoordinatorNode setSegmentPusher(DataSegmentPusher segmentPusher)
{
this.segmentPusher = segmentPusher;
return this;
}
public IndexerCoordinatorNode setMergeDbCoordinator(MergerDBCoordinator mergeDbCoordinator) public IndexerCoordinatorNode setMergeDbCoordinator(MergerDBCoordinator mergeDbCoordinator)
{ {
this.mergerDBCoordinator = mergeDbCoordinator; this.mergerDBCoordinator = mergeDbCoordinator;
@ -220,12 +228,6 @@ public class IndexerCoordinatorNode extends RegisteringNode
return this; return this;
} }
public IndexerCoordinatorNode setWorkerSetupManager(WorkerSetupManager workerSetupManager)
{
this.workerSetupManager = workerSetupManager;
return this;
}
public IndexerCoordinatorNode setTaskRunnerFactory(TaskRunnerFactory taskRunnerFactory) public IndexerCoordinatorNode setTaskRunnerFactory(TaskRunnerFactory taskRunnerFactory)
{ {
this.taskRunnerFactory = taskRunnerFactory; this.taskRunnerFactory = taskRunnerFactory;
@ -242,6 +244,10 @@ public class IndexerCoordinatorNode extends RegisteringNode
{ {
scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle); 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(); initializeEmitter();
initializeMonitors(); initializeMonitors();
initializeDB(); initializeDB();
@ -252,14 +258,14 @@ public class IndexerCoordinatorNode extends RegisteringNode
initializeTaskStorage(); initializeTaskStorage();
initializeTaskLockbox(); initializeTaskLockbox();
initializeTaskQueue(); initializeTaskQueue();
initializeDataSegmentPusher();
initializeTaskToolbox(); initializeTaskToolbox();
initializeJacksonInjections(); initializeJacksonInjections();
initializeJacksonSubtypes(); initializeJacksonSubtypes();
initializeCurator(); initializeCurator();
initializeIndexerZkConfig(); initializeIndexerZkConfig();
initializeWorkerSetupManager(); initializeTaskRunnerFactory(configManager);
initializeTaskRunnerFactory(); initializeResourceManagement(configManager);
initializeResourceManagement();
initializeTaskMasterLifecycle(); initializeTaskMasterLifecycle();
initializeServer(); initializeServer();
@ -279,7 +285,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
emitter, emitter,
taskMasterLifecycle, taskMasterLifecycle,
new TaskStorageQueryAdapter(taskStorage), new TaskStorageQueryAdapter(taskStorage),
workerSetupManager configManager
) )
); );
@ -339,7 +345,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class); final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class);
taskMasterLifecycle = new TaskMasterLifecycle( taskMasterLifecycle = new TaskMasterLifecycle(
taskQueue, taskQueue,
taskToolbox, taskToolboxFactory,
config, config,
serviceDiscoveryConfig, serviceDiscoveryConfig,
taskRunnerFactory, taskRunnerFactory,
@ -403,7 +409,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
InjectableValues.Std injectables = new InjectableValues.Std(); InjectableValues.Std injectables = new InjectableValues.Std();
injectables.addValue("s3Client", s3Service) injectables.addValue("s3Client", s3Service)
.addValue("segmentPusher", taskToolbox.getSegmentPusher()); .addValue("segmentPusher", segmentPusher);
jsonMapper.setInjectableValues(injectables); 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() public void initializeTaskToolbox()
{ {
if (taskToolbox == null) { if (taskToolboxFactory == null) {
final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher( final DataSegmentKiller dataSegmentKiller = new S3DataSegmentKiller(s3Service);
s3Service, taskToolboxFactory = new TaskToolboxFactory(
configFactory.build(S3DataSegmentPusherConfig.class),
jsonMapper
);
final SegmentKiller segmentKiller = new S3SegmentKiller(
s3Service
);
taskToolbox = new TaskToolbox(
taskConfig, taskConfig,
new LocalTaskActionClient( new LocalTaskActionClientFactory(
taskStorage, taskStorage,
new TaskActionToolbox(taskQueue, taskLockbox, mergerDBCoordinator, emitter) new TaskActionToolbox(taskQueue, taskLockbox, mergerDBCoordinator, emitter)
), ),
emitter, emitter,
s3Service, s3Service,
dataSegmentPusher, segmentPusher,
segmentKiller, dataSegmentKiller,
jsonMapper jsonMapper
); );
} }
@ -546,7 +552,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
{ {
if (taskStorage == null) { if (taskStorage == null) {
if (config.getStorageImpl().equals("local")) { if (config.getStorageImpl().equals("local")) {
taskStorage = new LocalTaskStorage(); taskStorage = new HeapMemoryTaskStorage();
} else if (config.getStorageImpl().equals("db")) { } else if (config.getStorageImpl().equals("db")) {
final IndexerDbConnectorConfig dbConnectorConfig = configFactory.build(IndexerDbConnectorConfig.class); final IndexerDbConnectorConfig dbConnectorConfig = configFactory.build(IndexerDbConnectorConfig.class);
taskStorage = new DbTaskStorage(jsonMapper, dbConnectorConfig, new DbConnector(dbConnectorConfig).getDBI()); taskStorage = new DbTaskStorage(jsonMapper, dbConnectorConfig, new DbConnector(dbConnectorConfig).getDBI());
@ -556,26 +562,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
} }
} }
public void initializeWorkerSetupManager() private void initializeTaskRunnerFactory(final JacksonConfigManager configManager)
{
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()
{ {
if (taskRunnerFactory == null) { if (taskRunnerFactory == null) {
if (config.getRunnerImpl().equals("remote")) { if (config.getRunnerImpl().equals("remote")) {
@ -601,7 +588,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
new PathChildrenCache(curatorFramework, indexerZkConfig.getAnnouncementPath(), true), new PathChildrenCache(curatorFramework, indexerZkConfig.getAnnouncementPath(), true),
retryScheduledExec, retryScheduledExec,
new RetryPolicyFactory(configFactory.build(RetryPolicyConfig.class)), new RetryPolicyFactory(configFactory.build(RetryPolicyConfig.class)),
workerSetupManager configManager.watch(WorkerSetupData.CONFIG_KEY, WorkerSetupData.class)
); );
return remoteTaskRunner; return remoteTaskRunner;
@ -615,7 +602,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
public TaskRunner build() public TaskRunner build()
{ {
final ExecutorService runnerExec = Executors.newFixedThreadPool(config.getNumLocalThreads()); final ExecutorService runnerExec = Executors.newFixedThreadPool(config.getNumLocalThreads());
return new LocalTaskRunner(taskToolbox, runnerExec); return new LocalTaskRunner(taskToolboxFactory, runnerExec);
} }
}; };
} else { } else {
@ -624,7 +611,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
} }
} }
private void initializeResourceManagement() private void initializeResourceManagement(final JacksonConfigManager configManager)
{ {
if (resourceManagementSchedulerFactory == null) { if (resourceManagementSchedulerFactory == null) {
resourceManagementSchedulerFactory = new ResourceManagementSchedulerFactory() resourceManagementSchedulerFactory = new ResourceManagementSchedulerFactory()
@ -639,6 +626,9 @@ public class IndexerCoordinatorNode extends RegisteringNode
.setNameFormat("ScalingExec--%d") .setNameFormat("ScalingExec--%d")
.build() .build()
); );
final AtomicReference<WorkerSetupData> workerSetupData = configManager.watch(
WorkerSetupData.CONFIG_KEY, WorkerSetupData.class
);
AutoScalingStrategy strategy; AutoScalingStrategy strategy;
if (config.getStrategyImpl().equalsIgnoreCase("ec2")) { if (config.getStrategyImpl().equalsIgnoreCase("ec2")) {
@ -651,7 +641,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
) )
), ),
configFactory.build(EC2AutoScalingStrategyConfig.class), configFactory.build(EC2AutoScalingStrategyConfig.class),
workerSetupManager workerSetupData
); );
} else if (config.getStrategyImpl().equalsIgnoreCase("noop")) { } else if (config.getStrategyImpl().equalsIgnoreCase("noop")) {
strategy = new NoopAutoScalingStrategy(); strategy = new NoopAutoScalingStrategy();
@ -664,7 +654,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
new SimpleResourceManagementStrategy( new SimpleResourceManagementStrategy(
strategy, strategy,
configFactory.build(SimpleResourceManagmentConfig.class), configFactory.build(SimpleResourceManagmentConfig.class),
workerSetupManager workerSetupData
), ),
configFactory.build(ResourceManagementSchedulerConfig.class), configFactory.build(ResourceManagementSchedulerConfig.class),
scalingScheduledExec scalingScheduledExec

View File

@ -19,6 +19,8 @@
package com.metamx.druid.merger.coordinator.http; 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.base.Optional;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
@ -26,8 +28,9 @@ import com.google.common.collect.Maps;
import com.google.inject.Inject; import com.google.inject.Inject;
import com.metamx.common.logger.Logger; import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment; 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.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.common.task.Task;
import com.metamx.druid.merger.coordinator.TaskMasterLifecycle; import com.metamx.druid.merger.coordinator.TaskMasterLifecycle;
import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter; 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.AutoScalingData;
import com.metamx.druid.merger.coordinator.scaling.ScalingStats; import com.metamx.druid.merger.coordinator.scaling.ScalingStats;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
import com.metamx.emitter.service.ServiceEmitter; 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.Consumes;
import javax.ws.rs.GET; import javax.ws.rs.GET;
@ -49,6 +49,7 @@ import javax.ws.rs.Produces;
import javax.ws.rs.core.Response; import javax.ws.rs.core.Response;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
/** /**
*/ */
@ -61,16 +62,18 @@ public class IndexerCoordinatorResource
private final ServiceEmitter emitter; private final ServiceEmitter emitter;
private final TaskMasterLifecycle taskMasterLifecycle; private final TaskMasterLifecycle taskMasterLifecycle;
private final TaskStorageQueryAdapter taskStorageQueryAdapter; private final TaskStorageQueryAdapter taskStorageQueryAdapter;
private final WorkerSetupManager workerSetupManager; private final JacksonConfigManager configManager;
private final ObjectMapper jsonMapper; private final ObjectMapper jsonMapper;
private AtomicReference<WorkerSetupData> workerSetupDataRef = null;
@Inject @Inject
public IndexerCoordinatorResource( public IndexerCoordinatorResource(
IndexerCoordinatorConfig config, IndexerCoordinatorConfig config,
ServiceEmitter emitter, ServiceEmitter emitter,
TaskMasterLifecycle taskMasterLifecycle, TaskMasterLifecycle taskMasterLifecycle,
TaskStorageQueryAdapter taskStorageQueryAdapter, TaskStorageQueryAdapter taskStorageQueryAdapter,
WorkerSetupManager workerSetupManager, JacksonConfigManager configManager,
ObjectMapper jsonMapper ObjectMapper jsonMapper
) throws Exception ) throws Exception
{ {
@ -78,7 +81,7 @@ public class IndexerCoordinatorResource
this.emitter = emitter; this.emitter = emitter;
this.taskMasterLifecycle = taskMasterLifecycle; this.taskMasterLifecycle = taskMasterLifecycle;
this.taskStorageQueryAdapter = taskStorageQueryAdapter; this.taskStorageQueryAdapter = taskStorageQueryAdapter;
this.workerSetupManager = workerSetupManager; this.configManager = configManager;
this.jsonMapper = jsonMapper; this.jsonMapper = jsonMapper;
} }
@ -98,18 +101,15 @@ public class IndexerCoordinatorResource
@Produces("application/json") @Produces("application/json")
public Response doIndex(final Task task) public Response doIndex(final Task task)
{ {
// verify against whitelist return taskPost(task);
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();
}
@POST
@Path("/task")
@Consumes("application/json")
@Produces("application/json")
public Response taskPost(final Task task)
{
taskMasterLifecycle.getTaskQueue().add(task); taskMasterLifecycle.getTaskQueue().add(task);
return Response.ok(ImmutableMap.of("task", task.getId())).build(); return Response.ok(ImmutableMap.of("task", task.getId())).build();
} }
@ -162,7 +162,11 @@ public class IndexerCoordinatorResource
@Produces("application/json") @Produces("application/json")
public Response getWorkerSetupData() 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 @POST
@ -172,7 +176,7 @@ public class IndexerCoordinatorResource
final WorkerSetupData workerSetupData final WorkerSetupData workerSetupData
) )
{ {
if (!workerSetupManager.setWorkerSetupData(workerSetupData)) { if (!configManager.set(WorkerSetupData.CONFIG_KEY, workerSetupData)) {
return Response.status(Response.Status.BAD_REQUEST).build(); return Response.status(Response.Status.BAD_REQUEST).build();
} }
return Response.ok().build(); return Response.ok().build();
@ -181,9 +185,12 @@ public class IndexerCoordinatorResource
@POST @POST
@Path("/action") @Path("/action")
@Produces("application/json") @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(); final Map<String, Object> retMap = Maps.newHashMap();
retMap.put("result", ret); 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.databind.ObjectMapper;
import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider;
import com.google.inject.Provides; import com.google.inject.Provides;
import com.metamx.druid.config.JacksonConfigManager;
import com.metamx.druid.merger.coordinator.TaskMasterLifecycle; import com.metamx.druid.merger.coordinator.TaskMasterLifecycle;
import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter; import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter;
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEmitter;
import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.JerseyServletModule;
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
@ -41,7 +41,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
private final ServiceEmitter emitter; private final ServiceEmitter emitter;
private final TaskMasterLifecycle taskMasterLifecycle; private final TaskMasterLifecycle taskMasterLifecycle;
private final TaskStorageQueryAdapter taskStorageQueryAdapter; private final TaskStorageQueryAdapter taskStorageQueryAdapter;
private final WorkerSetupManager workerSetupManager; private final JacksonConfigManager configManager;
public IndexerCoordinatorServletModule( public IndexerCoordinatorServletModule(
ObjectMapper jsonMapper, ObjectMapper jsonMapper,
@ -49,7 +49,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
ServiceEmitter emitter, ServiceEmitter emitter,
TaskMasterLifecycle taskMasterLifecycle, TaskMasterLifecycle taskMasterLifecycle,
TaskStorageQueryAdapter taskStorageQueryAdapter, TaskStorageQueryAdapter taskStorageQueryAdapter,
WorkerSetupManager workerSetupManager JacksonConfigManager configManager
) )
{ {
this.jsonMapper = jsonMapper; this.jsonMapper = jsonMapper;
@ -57,7 +57,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
this.emitter = emitter; this.emitter = emitter;
this.taskMasterLifecycle = taskMasterLifecycle; this.taskMasterLifecycle = taskMasterLifecycle;
this.taskStorageQueryAdapter = taskStorageQueryAdapter; this.taskStorageQueryAdapter = taskStorageQueryAdapter;
this.workerSetupManager = workerSetupManager; this.configManager = configManager;
} }
@Override @Override
@ -69,7 +69,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
bind(ServiceEmitter.class).toInstance(emitter); bind(ServiceEmitter.class).toInstance(emitter);
bind(TaskMasterLifecycle.class).toInstance(taskMasterLifecycle); bind(TaskMasterLifecycle.class).toInstance(taskMasterLifecycle);
bind(TaskStorageQueryAdapter.class).toInstance(taskStorageQueryAdapter); bind(TaskStorageQueryAdapter.class).toInstance(taskStorageQueryAdapter);
bind(WorkerSetupManager.class).toInstance(workerSetupManager); bind(JacksonConfigManager.class).toInstance(configManager);
serve("/*").with(GuiceContainer.class); 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.config.EC2AutoScalingStrategyConfig;
import com.metamx.druid.merger.coordinator.setup.EC2NodeData; import com.metamx.druid.merger.coordinator.setup.EC2NodeData;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.EmittingLogger;
import org.apache.commons.codec.binary.Base64; import org.apache.commons.codec.binary.Base64;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.util.List; 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 ObjectMapper jsonMapper;
private final AmazonEC2Client amazonEC2Client; private final AmazonEC2Client amazonEC2Client;
private final EC2AutoScalingStrategyConfig config; private final EC2AutoScalingStrategyConfig config;
private final WorkerSetupManager workerSetupManager; private final AtomicReference<WorkerSetupData> workerSetupDataRef;
public EC2AutoScalingStrategy( public EC2AutoScalingStrategy(
ObjectMapper jsonMapper, ObjectMapper jsonMapper,
AmazonEC2Client amazonEC2Client, AmazonEC2Client amazonEC2Client,
EC2AutoScalingStrategyConfig config, EC2AutoScalingStrategyConfig config,
WorkerSetupManager workerSetupManager AtomicReference<WorkerSetupData> workerSetupDataRef
) )
{ {
this.jsonMapper = jsonMapper; this.jsonMapper = jsonMapper;
this.amazonEC2Client = amazonEC2Client; this.amazonEC2Client = amazonEC2Client;
this.config = config; this.config = config;
this.workerSetupManager = workerSetupManager; this.workerSetupDataRef = workerSetupDataRef;
} }
@Override @Override
public AutoScalingData<Instance> provision() public AutoScalingData<Instance> provision()
{ {
try { try {
WorkerSetupData setupData = workerSetupManager.getWorkerSetupData(); WorkerSetupData setupData = workerSetupDataRef.get();
EC2NodeData workerConfig = setupData.getNodeData(); EC2NodeData workerConfig = setupData.getNodeData();
RunInstancesResult result = amazonEC2Client.runInstances( 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; package com.metamx.druid.merger.coordinator.scaling;
import com.fasterxml.jackson.annotation.JsonProperty; 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.common.guava.FunctionalIterable;
import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem; import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem;
import com.metamx.druid.merger.coordinator.ZkWorker; 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 com.metamx.emitter.EmittingLogger;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.Duration; import org.joda.time.Duration;
import java.util.Collection; import java.util.Collection;
import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.concurrent.ConcurrentSkipListSet; 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 AutoScalingStrategy autoScalingStrategy;
private final SimpleResourceManagmentConfig config; private final SimpleResourceManagmentConfig config;
private final WorkerSetupManager workerSetupManager; private final AtomicReference<WorkerSetupData> workerSetupdDataRef;
private final ScalingStats scalingStats; private final ScalingStats scalingStats;
private final ConcurrentSkipListSet<String> currentlyProvisioning = new ConcurrentSkipListSet<String>(); private final ConcurrentSkipListSet<String> currentlyProvisioning = new ConcurrentSkipListSet<String>();
@ -58,12 +58,12 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat
public SimpleResourceManagementStrategy( public SimpleResourceManagementStrategy(
AutoScalingStrategy autoScalingStrategy, AutoScalingStrategy autoScalingStrategy,
SimpleResourceManagmentConfig config, SimpleResourceManagmentConfig config,
WorkerSetupManager workerSetupManager AtomicReference<WorkerSetupData> workerSetupdDataRef
) )
{ {
this.autoScalingStrategy = autoScalingStrategy; this.autoScalingStrategy = autoScalingStrategy;
this.config = config; this.config = config;
this.workerSetupManager = workerSetupManager; this.workerSetupdDataRef = workerSetupdDataRef;
this.scalingStats = new ScalingStats(config.getNumEventsToTrack()); this.scalingStats = new ScalingStats(config.getNumEventsToTrack());
} }
@ -151,7 +151,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat
boolean nothingTerminating = currentlyTerminating.isEmpty(); boolean nothingTerminating = currentlyTerminating.isEmpty();
if (nothingTerminating) { if (nothingTerminating) {
final int minNumWorkers = workerSetupManager.getWorkerSetupData().getMinNumWorkers(); final int minNumWorkers = workerSetupdDataRef.get().getMinNumWorkers();
if (zkWorkers.size() <= minNumWorkers) { if (zkWorkers.size() <= minNumWorkers) {
log.info("Only [%d <= %d] nodes in the cluster, not terminating anything.", zkWorkers.size(), minNumWorkers); log.info("Only [%d <= %d] nodes in the cluster, not terminating anything.", zkWorkers.size(), minNumWorkers);
return false; return false;

View File

@ -31,6 +31,8 @@ import java.util.List;
*/ */
public class WorkerSetupData public class WorkerSetupData
{ {
public static final String CONFIG_KEY = "worker.setup";
private final String minVersion; private final String minVersion;
private final int minNumWorkers; private final int minNumWorkers;
private final EC2NodeData nodeData; 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.common.lifecycle.LifecycleStop;
import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox; 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.common.task.Task;
import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.EmittingLogger;
import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.framework.CuratorFramework;
@ -45,21 +46,21 @@ public class TaskMonitor
private final PathChildrenCache pathChildrenCache; private final PathChildrenCache pathChildrenCache;
private final CuratorFramework cf; private final CuratorFramework cf;
private final WorkerCuratorCoordinator workerCuratorCoordinator; private final WorkerCuratorCoordinator workerCuratorCoordinator;
private final TaskToolbox toolbox; private final TaskToolboxFactory toolboxFactory;
private final ExecutorService exec; private final ExecutorService exec;
public TaskMonitor( public TaskMonitor(
PathChildrenCache pathChildrenCache, PathChildrenCache pathChildrenCache,
CuratorFramework cf, CuratorFramework cf,
WorkerCuratorCoordinator workerCuratorCoordinator, WorkerCuratorCoordinator workerCuratorCoordinator,
TaskToolbox toolbox, TaskToolboxFactory toolboxFactory,
ExecutorService exec ExecutorService exec
) )
{ {
this.pathChildrenCache = pathChildrenCache; this.pathChildrenCache = pathChildrenCache;
this.cf = cf; this.cf = cf;
this.workerCuratorCoordinator = workerCuratorCoordinator; this.workerCuratorCoordinator = workerCuratorCoordinator;
this.toolbox = toolbox; this.toolboxFactory = toolboxFactory;
this.exec = exec; this.exec = exec;
} }
@ -81,10 +82,11 @@ public class TaskMonitor
throws Exception throws Exception
{ {
if (pathChildrenCacheEvent.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) { 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()), cf.getData().forPath(pathChildrenCacheEvent.getData().getPath()),
Task.class Task.class
); );
final TaskToolbox toolbox = toolboxFactory.build(task);
if (workerCuratorCoordinator.statusExists(task.getId())) { if (workerCuratorCoordinator.statusExists(task.getId())) {
log.warn("Got task %s that I am already running...", task.getId()); log.warn("Got task %s that I am already running...", task.getId());
@ -99,7 +101,7 @@ public class TaskMonitor
public void run() public void run()
{ {
final long startTime = System.currentTimeMillis(); final long startTime = System.currentTimeMillis();
final File taskDir = toolbox.getConfig().getTaskDir(task); final File taskDir = toolbox.getTaskDir();
log.info("Running task [%s]", task.getId()); 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.CuratorConfig;
import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.Initialization;
import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ServerConfig;
import com.metamx.druid.initialization.ServerInit;
import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.initialization.ServiceDiscoveryConfig;
import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.loading.S3DataSegmentPusher; import com.metamx.druid.loading.S3DataSegmentKiller;
import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.loading.DataSegmentKiller;
import com.metamx.druid.loading.S3SegmentKiller; import com.metamx.druid.merger.common.TaskToolboxFactory;
import com.metamx.druid.loading.SegmentKiller; import com.metamx.druid.merger.common.actions.RemoteTaskActionClientFactory;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.actions.RemoteTaskActionClient;
import com.metamx.druid.merger.common.config.IndexerZkConfig; import com.metamx.druid.merger.common.config.IndexerZkConfig;
import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory;
@ -106,7 +105,8 @@ public class WorkerNode extends RegisteringNode
private ServiceEmitter emitter = null; private ServiceEmitter emitter = null;
private TaskConfig taskConfig = null; private TaskConfig taskConfig = null;
private WorkerConfig workerConfig = null; private WorkerConfig workerConfig = null;
private TaskToolbox taskToolbox = null; private DataSegmentPusher segmentPusher = null;
private TaskToolboxFactory taskToolboxFactory = null;
private CuratorFramework curatorFramework = null; private CuratorFramework curatorFramework = null;
private ServiceDiscovery serviceDiscovery = null; private ServiceDiscovery serviceDiscovery = null;
private ServiceProvider coordinatorServiceProvider = null; private ServiceProvider coordinatorServiceProvider = null;
@ -149,9 +149,15 @@ public class WorkerNode extends RegisteringNode
return this; 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; return this;
} }
@ -195,6 +201,7 @@ public class WorkerNode extends RegisteringNode
initializeCuratorFramework(); initializeCuratorFramework();
initializeServiceDiscovery(); initializeServiceDiscovery();
initializeCoordinatorServiceProvider(); initializeCoordinatorServiceProvider();
initializeDataSegmentPusher();
initializeTaskToolbox(); initializeTaskToolbox();
initializeJacksonInjections(); initializeJacksonInjections();
initializeJacksonSubtypes(); initializeJacksonSubtypes();
@ -271,7 +278,7 @@ public class WorkerNode extends RegisteringNode
InjectableValues.Std injectables = new InjectableValues.Std(); InjectableValues.Std injectables = new InjectableValues.Std();
injectables.addValue("s3Client", s3Service) injectables.addValue("s3Client", s3Service)
.addValue("segmentPusher", taskToolbox.getSegmentPusher()); .addValue("segmentPusher", segmentPusher);
jsonMapper.setInjectableValues(injectables); 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 public void initializeTaskToolbox() throws S3ServiceException
{ {
if (taskToolbox == null) { if (taskToolboxFactory == null) {
final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher( final DataSegmentKiller dataSegmentKiller = new S3DataSegmentKiller(s3Service);
s3Service, taskToolboxFactory = new TaskToolboxFactory(
configFactory.build(S3DataSegmentPusherConfig.class),
jsonMapper
);
final SegmentKiller segmentKiller = new S3SegmentKiller(
s3Service
);
taskToolbox = new TaskToolbox(
taskConfig, taskConfig,
new RemoteTaskActionClient(httpClient, coordinatorServiceProvider, jsonMapper), new RemoteTaskActionClientFactory(httpClient, coordinatorServiceProvider, jsonMapper),
emitter, emitter,
s3Service, s3Service,
dataSegmentPusher, segmentPusher,
segmentKiller, dataSegmentKiller,
jsonMapper jsonMapper
); );
} }
@ -417,7 +424,7 @@ public class WorkerNode extends RegisteringNode
pathChildrenCache, pathChildrenCache,
curatorFramework, curatorFramework,
workerCuratorCoordinator, workerCuratorCoordinator,
taskToolbox, taskToolboxFactory,
workerExec workerExec
); );
lifecycle.addManagedInstance(taskMonitor); 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.client.DataSegment;
import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox; 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; import java.util.List;
/** /**
*/ */
@JsonTypeName("test") @JsonTypeName("test")
public class TestTask extends DefaultMergeTask public class TestTask extends MergeTask
{ {
private final String id; private final String id;
private final TaskStatus status; private final TaskStatus status;

View File

@ -31,7 +31,7 @@ import java.io.File;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
public class MergeTaskTest public class MergeTaskBaseTest
{ {
private final DataSegment.Builder segmentBuilder = DataSegment.builder() private final DataSegment.Builder segmentBuilder = DataSegment.builder()
.dataSource("foo") .dataSource("foo")
@ -43,7 +43,7 @@ public class MergeTaskTest
.add(segmentBuilder.interval(new Interval("2012-01-03/2012-01-05")).build()) .add(segmentBuilder.interval(new Interval("2012-01-03/2012-01-05")).build())
.build(); .build();
final MergeTask testMergeTask = new MergeTask("foo", segments) final MergeTaskBase testMergeTaskBase = new MergeTaskBase("foo", segments)
{ {
@Override @Override
protected File merge(Map<DataSegment, File> segments, File outDir) throws Exception protected File merge(Map<DataSegment, File> segments, File outDir) throws Exception
@ -61,13 +61,13 @@ public class MergeTaskTest
@Test @Test
public void testDataSource() public void testDataSource()
{ {
Assert.assertEquals("foo", testMergeTask.getDataSource()); Assert.assertEquals("foo", testMergeTaskBase.getDataSource());
} }
@Test @Test
public void testInterval() 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 @Test
@ -81,7 +81,7 @@ public class MergeTaskTest
).toString().toLowerCase() + "_"; ).toString().toLowerCase() + "_";
Assert.assertEquals( Assert.assertEquals(
desiredPrefix, 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.TestTask;
import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskCallback;
import com.metamx.druid.merger.common.TaskStatus; 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.IndexerZkConfig;
import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig;
import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; 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.TaskMonitor;
import com.metamx.druid.merger.worker.Worker; import com.metamx.druid.merger.worker.Worker;
import com.metamx.druid.merger.worker.WorkerCuratorCoordinator; import com.metamx.druid.merger.worker.WorkerCuratorCoordinator;
@ -42,6 +41,7 @@ import org.junit.Test;
import java.io.File; import java.io.File;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicReference;
import static junit.framework.Assert.fail; import static junit.framework.Assert.fail;
@ -60,7 +60,6 @@ public class RemoteTaskRunnerTest
private PathChildrenCache pathChildrenCache; private PathChildrenCache pathChildrenCache;
private RemoteTaskRunner remoteTaskRunner; private RemoteTaskRunner remoteTaskRunner;
private TaskMonitor taskMonitor; private TaskMonitor taskMonitor;
private WorkerSetupManager workerSetupManager;
private ScheduledExecutorService scheduledExec; private ScheduledExecutorService scheduledExec;
@ -280,7 +279,7 @@ public class RemoteTaskRunnerTest
new PathChildrenCache(cf, String.format("%s/worker1", tasksPath), true), new PathChildrenCache(cf, String.format("%s/worker1", tasksPath), true),
cf, cf,
workerCuratorCoordinator, workerCuratorCoordinator,
new TaskToolbox( new TaskToolboxFactory(
new TaskConfig() new TaskConfig()
{ {
@Override @Override
@ -316,17 +315,6 @@ public class RemoteTaskRunnerTest
private void makeRemoteTaskRunner() throws Exception private void makeRemoteTaskRunner() throws Exception
{ {
scheduledExec = EasyMock.createMock(ScheduledExecutorService.class); 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( remoteTaskRunner = new RemoteTaskRunner(
jsonMapper, jsonMapper,
@ -335,7 +323,7 @@ public class RemoteTaskRunnerTest
pathChildrenCache, pathChildrenCache,
scheduledExec, scheduledExec,
new RetryPolicyFactory(new TestRetryPolicyConfig()), 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 // 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; package com.metamx.druid.merger.coordinator;
import com.google.common.base.Optional; 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.input.MapBasedInputRow;
import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.loading.DataSegmentPusher; 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.loading.SegmentLoadingException;
import com.metamx.druid.merger.common.TaskLock; import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox; 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.LockAcquireAction;
import com.metamx.druid.merger.common.actions.LockListAction; import com.metamx.druid.merger.common.actions.LockListAction;
import com.metamx.druid.merger.common.actions.LockReleaseAction; import com.metamx.druid.merger.common.actions.LockReleaseAction;
@ -56,10 +76,8 @@ import org.junit.Test;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.Collection;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
@ -71,7 +89,7 @@ public class TaskLifecycleTest
private TaskQueue tq = null; private TaskQueue tq = null;
private TaskRunner tr = null; private TaskRunner tr = null;
private MockMergerDBCoordinator mdc = null; private MockMergerDBCoordinator mdc = null;
private TaskToolbox tb = null; private TaskToolboxFactory tb = null;
private TaskConsumer tc = null; private TaskConsumer tc = null;
TaskStorageQueryAdapter tsqa = null; TaskStorageQueryAdapter tsqa = null;
@ -91,12 +109,12 @@ public class TaskLifecycleTest
tmp = Files.createTempDir(); tmp = Files.createTempDir();
ts = new LocalTaskStorage(); ts = new HeapMemoryTaskStorage();
tl = new TaskLockbox(ts); tl = new TaskLockbox(ts);
tq = new TaskQueue(ts, tl); tq = new TaskQueue(ts, tl);
mdc = newMockMDC(); mdc = newMockMDC();
tb = new TaskToolbox( tb = new TaskToolboxFactory(
new TaskConfig() new TaskConfig()
{ {
@Override @Override
@ -117,7 +135,7 @@ public class TaskLifecycleTest
return null; return null;
} }
}, },
new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, mdc, newMockEmitter())), new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, mdc, newMockEmitter())),
newMockEmitter(), newMockEmitter(),
null, // s3 client null, // s3 client
new DataSegmentPusher() new DataSegmentPusher()
@ -128,25 +146,16 @@ public class TaskLifecycleTest
return segment; return segment;
} }
}, },
new SegmentKiller() new DataSegmentKiller()
{ {
@Override @Override
public void kill(Collection<DataSegment> segments) throws ServiceException public void kill(DataSegment segments) throws SegmentLoadingException
{ {
} }
}, },
new DefaultObjectMapper() new DefaultObjectMapper()
) );
{
@Override
public Map<DataSegment, File> getSegments(
Task task, List<DataSegment> segments
) throws SegmentLoadingException
{
return ImmutableMap.of();
}
};
tr = new LocalTaskRunner( tr = new LocalTaskRunner(
tb, tb,
@ -239,11 +248,12 @@ public class TaskLifecycleTest
@Test @Test
public void testKillTask() throws Exception 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 Task killTask = new KillTask("foo", new Interval("2010-01-02/P2D"));
final TaskStatus mergedStatus = runTask(killTask); final TaskStatus status = runTask(killTask);
Assert.assertEquals("merged statusCode", TaskStatus.Status.SUCCESS, mergedStatus.getStatusCode()); Assert.assertEquals("merged statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode());
Assert.assertEquals("num segments published", 0, mdc.getPublished().size()); Assert.assertEquals("num segments published", 0, mdc.getPublished().size());
Assert.assertEquals("num segments nuked", 0, mdc.getNuked().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: // Sort of similar to what realtime tasks do:
// Acquire lock for first interval // Acquire lock for first interval
final Optional<TaskLock> lock1 = toolbox.getTaskActionClient().submit(new LockAcquireAction(this, interval1)); final Optional<TaskLock> lock1 = toolbox.getTaskActionClientFactory().submit(new LockAcquireAction(interval1));
final List<TaskLock> locks1 = toolbox.getTaskActionClient().submit(new LockListAction(this)); final List<TaskLock> locks1 = toolbox.getTaskActionClientFactory().submit(new LockListAction());
// (Confirm lock sanity) // (Confirm lock sanity)
Assert.assertTrue("lock1 present", lock1.isPresent()); Assert.assertTrue("lock1 present", lock1.isPresent());
@ -282,8 +292,8 @@ public class TaskLifecycleTest
Assert.assertEquals("locks1", ImmutableList.of(lock1.get()), locks1); Assert.assertEquals("locks1", ImmutableList.of(lock1.get()), locks1);
// Acquire lock for second interval // Acquire lock for second interval
final Optional<TaskLock> lock2 = toolbox.getTaskActionClient().submit(new LockAcquireAction(this, interval2)); final Optional<TaskLock> lock2 = toolbox.getTaskActionClientFactory().submit(new LockAcquireAction(interval2));
final List<TaskLock> locks2 = toolbox.getTaskActionClient().submit(new LockListAction(this)); final List<TaskLock> locks2 = toolbox.getTaskActionClientFactory().submit(new LockListAction());
// (Confirm lock sanity) // (Confirm lock sanity)
Assert.assertTrue("lock2 present", lock2.isPresent()); Assert.assertTrue("lock2 present", lock2.isPresent());
@ -291,10 +301,9 @@ public class TaskLifecycleTest
Assert.assertEquals("locks2", ImmutableList.of(lock1.get(), lock2.get()), locks2); Assert.assertEquals("locks2", ImmutableList.of(lock1.get(), lock2.get()), locks2);
// Push first segment // Push first segment
toolbox.getTaskActionClient() toolbox.getTaskActionClientFactory()
.submit( .submit(
new SegmentInsertAction( new SegmentInsertAction(
this,
ImmutableSet.of( ImmutableSet.of(
DataSegment.builder() DataSegment.builder()
.dataSource("foo") .dataSource("foo")
@ -306,17 +315,16 @@ public class TaskLifecycleTest
); );
// Release first lock // Release first lock
toolbox.getTaskActionClient().submit(new LockReleaseAction(this, interval1)); toolbox.getTaskActionClientFactory().submit(new LockReleaseAction(interval1));
final List<TaskLock> locks3 = toolbox.getTaskActionClient().submit(new LockListAction(this)); final List<TaskLock> locks3 = toolbox.getTaskActionClientFactory().submit(new LockListAction());
// (Confirm lock sanity) // (Confirm lock sanity)
Assert.assertEquals("locks3", ImmutableList.of(lock2.get()), locks3); Assert.assertEquals("locks3", ImmutableList.of(lock2.get()), locks3);
// Push second segment // Push second segment
toolbox.getTaskActionClient() toolbox.getTaskActionClientFactory()
.submit( .submit(
new SegmentInsertAction( new SegmentInsertAction(
this,
ImmutableSet.of( ImmutableSet.of(
DataSegment.builder() DataSegment.builder()
.dataSource("foo") .dataSource("foo")
@ -328,8 +336,8 @@ public class TaskLifecycleTest
); );
// Release second lock // Release second lock
toolbox.getTaskActionClient().submit(new LockReleaseAction(this, interval2)); toolbox.getTaskActionClientFactory().submit(new LockReleaseAction(interval2));
final List<TaskLock> locks4 = toolbox.getTaskActionClient().submit(new LockListAction(this)); final List<TaskLock> locks4 = toolbox.getTaskActionClientFactory().submit(new LockListAction());
// (Confirm lock sanity) // (Confirm lock sanity)
Assert.assertEquals("locks4", ImmutableList.<TaskLock>of(), locks4); Assert.assertEquals("locks4", ImmutableList.<TaskLock>of(), locks4);
@ -362,8 +370,8 @@ public class TaskLifecycleTest
public TaskStatus run(TaskToolbox toolbox) throws Exception public TaskStatus run(TaskToolbox toolbox) throws Exception
{ {
final TaskLock myLock = Iterables.getOnlyElement( final TaskLock myLock = Iterables.getOnlyElement(
toolbox.getTaskActionClient() toolbox.getTaskActionClientFactory()
.submit(new LockListAction(this)) .submit(new LockListAction())
); );
final DataSegment segment = DataSegment.builder() final DataSegment segment = DataSegment.builder()
@ -372,7 +380,7 @@ public class TaskLifecycleTest
.version(myLock.getVersion()) .version(myLock.getVersion())
.build(); .build();
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment))); toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(segment)));
return TaskStatus.success(getId()); return TaskStatus.success(getId());
} }
}; };
@ -398,10 +406,7 @@ public class TaskLifecycleTest
@Override @Override
public TaskStatus run(TaskToolbox toolbox) throws Exception public TaskStatus run(TaskToolbox toolbox) throws Exception
{ {
final TaskLock myLock = Iterables.getOnlyElement( final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction()));
toolbox.getTaskActionClient()
.submit(new LockListAction(this))
);
final DataSegment segment = DataSegment.builder() final DataSegment segment = DataSegment.builder()
.dataSource("ds") .dataSource("ds")
@ -409,7 +414,7 @@ public class TaskLifecycleTest
.version(myLock.getVersion()) .version(myLock.getVersion())
.build(); .build();
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment))); toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(segment)));
return TaskStatus.success(getId()); return TaskStatus.success(getId());
} }
}; };
@ -435,10 +440,7 @@ public class TaskLifecycleTest
@Override @Override
public TaskStatus run(TaskToolbox toolbox) throws Exception public TaskStatus run(TaskToolbox toolbox) throws Exception
{ {
final TaskLock myLock = Iterables.getOnlyElement( final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction()));
toolbox.getTaskActionClient()
.submit(new LockListAction(this))
);
final DataSegment segment = DataSegment.builder() final DataSegment segment = DataSegment.builder()
.dataSource("ds") .dataSource("ds")
@ -446,7 +448,7 @@ public class TaskLifecycleTest
.version(myLock.getVersion() + "1!!!1!!") .version(myLock.getVersion() + "1!!!1!!")
.build(); .build();
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment))); toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(segment)));
return TaskStatus.success(getId()); 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.TaskLock;
import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox; 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.SpawnTasksAction;
import com.metamx.druid.merger.common.actions.TaskActionToolbox; import com.metamx.druid.merger.common.actions.TaskActionToolbox;
import com.metamx.druid.merger.common.task.AbstractTask; import com.metamx.druid.merger.common.task.AbstractTask;
@ -43,7 +44,7 @@ public class TaskQueueTest
@Test @Test
public void testEmptyQueue() throws Exception public void testEmptyQueue() throws Exception
{ {
final TaskStorage ts = new LocalTaskStorage(); final TaskStorage ts = new HeapMemoryTaskStorage();
final TaskLockbox tl = new TaskLockbox(ts); final TaskLockbox tl = new TaskLockbox(ts);
final TaskQueue tq = newTaskQueue(ts, tl); final TaskQueue tq = newTaskQueue(ts, tl);
@ -65,7 +66,7 @@ public class TaskQueueTest
@Test @Test
public void testAddRemove() throws Exception public void testAddRemove() throws Exception
{ {
final TaskStorage ts = new LocalTaskStorage(); final TaskStorage ts = new HeapMemoryTaskStorage();
final TaskLockbox tl = new TaskLockbox(ts); final TaskLockbox tl = new TaskLockbox(ts);
final TaskQueue tq = newTaskQueue(ts, tl); final TaskQueue tq = newTaskQueue(ts, tl);
@ -154,12 +155,12 @@ public class TaskQueueTest
@Test @Test
public void testContinues() throws Exception public void testContinues() throws Exception
{ {
final TaskStorage ts = new LocalTaskStorage(); final TaskStorage ts = new HeapMemoryTaskStorage();
final TaskLockbox tl = new TaskLockbox(ts); final TaskLockbox tl = new TaskLockbox(ts);
final TaskQueue tq = newTaskQueue(ts, tl); final TaskQueue tq = newTaskQueue(ts, tl);
final TaskToolbox tb = new TaskToolbox( final TaskToolboxFactory tb = new TaskToolboxFactory(
null, null,
new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, null, null)), new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, null, null)),
null, null,
null, null,
null, null,
@ -181,7 +182,7 @@ public class TaskQueueTest
Assert.assertNull("poll #2", tq.poll()); Assert.assertNull("poll #2", tq.poll());
// report T1 done. Should cause T0 to be created // 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 isPresent (#2)", ts.getStatus("T0").isPresent());
Assert.assertTrue("T0 isRunnable (#2)", ts.getStatus("T0").get().isRunnable()); Assert.assertTrue("T0 isRunnable (#2)", ts.getStatus("T0").get().isRunnable());
@ -195,7 +196,7 @@ public class TaskQueueTest
Assert.assertNull("poll #4", tq.poll()); Assert.assertNull("poll #4", tq.poll());
// report T0 done. Should cause T0, T1 to be marked complete // 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 isPresent (#3)", ts.getStatus("T0").isPresent());
Assert.assertTrue("T0 isRunnable (#3)", !ts.getStatus("T0").get().isRunnable()); Assert.assertTrue("T0 isRunnable (#3)", !ts.getStatus("T0").get().isRunnable());
@ -211,12 +212,12 @@ public class TaskQueueTest
@Test @Test
public void testConcurrency() throws Exception public void testConcurrency() throws Exception
{ {
final TaskStorage ts = new LocalTaskStorage(); final TaskStorage ts = new HeapMemoryTaskStorage();
final TaskLockbox tl = new TaskLockbox(ts); final TaskLockbox tl = new TaskLockbox(ts);
final TaskQueue tq = newTaskQueue(ts, tl); final TaskQueue tq = newTaskQueue(ts, tl);
final TaskToolbox tb = new TaskToolbox( final TaskToolboxFactory tb = new TaskToolboxFactory(
null, null,
new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, null, null)), new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, null, null)),
null, null,
null, null,
null, null,
@ -248,7 +249,7 @@ public class TaskQueueTest
Thread.sleep(5); Thread.sleep(5);
// Finish t0 // Finish t0
tq.notify(t0, t0.run(tb)); tq.notify(t0, t0.run(tb.build(t0)));
// take max number of tasks // take max number of tasks
final Set<String> taken = Sets.newHashSet(); final Set<String> taken = Sets.newHashSet();
@ -280,7 +281,7 @@ public class TaskQueueTest
Assert.assertNull("null poll #2", tq.poll()); Assert.assertNull("null poll #2", tq.poll());
// Finish t3 // Finish t3
tq.notify(t3, t3.run(tb)); tq.notify(t3, t3.run(tb.build(t3)));
// We should be able to get t2 now // We should be able to get t2 now
final Task wt2 = tq.poll(); final Task wt2 = tq.poll();
@ -291,7 +292,7 @@ public class TaskQueueTest
Assert.assertNull("null poll #3", tq.poll()); Assert.assertNull("null poll #3", tq.poll());
// Finish t2 // Finish t2
tq.notify(t2, t2.run(tb)); tq.notify(t2, t2.run(tb.build(t2)));
// We should be able to get t4 // 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 // 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()); Assert.assertNotSame("wt4 version", wt2Lock.getVersion(), wt4Lock.getVersion());
// Kind of done testing at this point, but let's finish t4 anyway // 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()); Assert.assertNull("null poll #4", tq.poll());
} }
@Test @Test
public void testBootstrap() throws Exception public void testBootstrap() throws Exception
{ {
final TaskStorage storage = new LocalTaskStorage(); final TaskStorage storage = new HeapMemoryTaskStorage();
final TaskLockbox lockbox = new TaskLockbox(storage); final TaskLockbox lockbox = new TaskLockbox(storage);
storage.insert(newTask("T1", "G1", "bar", new Interval("2011-01-01/P1D")), TaskStatus.running("T1")); storage.insert(newTask("T1", "G1", "bar", new Interval("2011-01-01/P1D")), TaskStatus.running("T1"));
@ -374,7 +375,7 @@ public class TaskQueueTest
@Override @Override
public TaskStatus run(TaskToolbox toolbox) throws Exception public TaskStatus run(TaskToolbox toolbox) throws Exception
{ {
toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, nextTasks)); toolbox.getTaskActionClientFactory().submit(new SpawnTasksAction(nextTasks));
return TaskStatus.success(id); 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.EC2NodeData;
import com.metamx.druid.merger.coordinator.setup.GalaxyUserData; import com.metamx.druid.merger.coordinator.setup.GalaxyUserData;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.junit.After; import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
@ -42,6 +41,7 @@ import org.junit.Test;
import java.util.Arrays; import java.util.Arrays;
import java.util.Date; import java.util.Date;
import java.util.concurrent.atomic.AtomicReference;
/** /**
*/ */
@ -57,7 +57,7 @@ public class EC2AutoScalingStrategyTest
private Reservation reservation; private Reservation reservation;
private Instance instance; private Instance instance;
private EC2AutoScalingStrategy strategy; private EC2AutoScalingStrategy strategy;
private WorkerSetupManager workerSetupManager; private AtomicReference<WorkerSetupData> workerSetupData;
@Before @Before
public void setUp() throws Exception public void setUp() throws Exception
@ -66,7 +66,7 @@ public class EC2AutoScalingStrategyTest
runInstancesResult = EasyMock.createMock(RunInstancesResult.class); runInstancesResult = EasyMock.createMock(RunInstancesResult.class);
describeInstancesResult = EasyMock.createMock(DescribeInstancesResult.class); describeInstancesResult = EasyMock.createMock(DescribeInstancesResult.class);
reservation = EasyMock.createMock(Reservation.class); reservation = EasyMock.createMock(Reservation.class);
workerSetupManager = EasyMock.createMock(WorkerSetupManager.class); workerSetupData = new AtomicReference<WorkerSetupData>(null);
instance = new Instance() instance = new Instance()
.withInstanceId(INSTANCE_ID) .withInstanceId(INSTANCE_ID)
@ -85,7 +85,7 @@ public class EC2AutoScalingStrategyTest
return "8080"; return "8080";
} }
}, },
workerSetupManager workerSetupData
); );
} }
@ -96,13 +96,12 @@ public class EC2AutoScalingStrategyTest
EasyMock.verify(runInstancesResult); EasyMock.verify(runInstancesResult);
EasyMock.verify(describeInstancesResult); EasyMock.verify(describeInstancesResult);
EasyMock.verify(reservation); EasyMock.verify(reservation);
EasyMock.verify(workerSetupManager);
} }
@Test @Test
public void testScale() public void testScale()
{ {
EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn( workerSetupData.set(
new WorkerSetupData( new WorkerSetupData(
"0", "0",
0, 0,
@ -110,7 +109,6 @@ public class EC2AutoScalingStrategyTest
new GalaxyUserData("env", "version", "type") new GalaxyUserData("env", "version", "type")
) )
); );
EasyMock.replay(workerSetupManager);
EasyMock.expect(amazonEC2Client.runInstances(EasyMock.anyObject(RunInstancesRequest.class))).andReturn( EasyMock.expect(amazonEC2Client.runInstances(EasyMock.anyObject(RunInstancesRequest.class))).andReturn(
runInstancesResult 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.TaskRunnerWorkItem;
import com.metamx.druid.merger.coordinator.ZkWorker; import com.metamx.druid.merger.coordinator.ZkWorker;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
import com.metamx.druid.merger.worker.Worker; import com.metamx.druid.merger.worker.Worker;
import junit.framework.Assert; import junit.framework.Assert;
import org.easymock.EasyMock; import org.easymock.EasyMock;
@ -42,21 +41,22 @@ import org.junit.Test;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
/** /**
*/ */
public class SimpleResourceManagementStrategyTest public class SimpleResourceManagementStrategyTest
{ {
private AutoScalingStrategy autoScalingStrategy; private AutoScalingStrategy autoScalingStrategy;
private WorkerSetupManager workerSetupManager;
private Task testTask; private Task testTask;
private SimpleResourceManagementStrategy simpleResourceManagementStrategy; private SimpleResourceManagementStrategy simpleResourceManagementStrategy;
private AtomicReference<WorkerSetupData> workerSetupData;
@Before @Before
public void setUp() throws Exception public void setUp() throws Exception
{ {
workerSetupManager = EasyMock.createMock(WorkerSetupManager.class);
autoScalingStrategy = EasyMock.createMock(AutoScalingStrategy.class); autoScalingStrategy = EasyMock.createMock(AutoScalingStrategy.class);
workerSetupData = new AtomicReference<WorkerSetupData>(null);
testTask = new TestTask( testTask = new TestTask(
"task1", "task1",
@ -105,7 +105,7 @@ public class SimpleResourceManagementStrategyTest
return new Duration(0); return new Duration(0);
} }
}, },
workerSetupManager workerSetupData
); );
} }
@ -187,8 +187,7 @@ public class SimpleResourceManagementStrategyTest
@Test @Test
public void testDoSuccessfulTerminate() throws Exception public void testDoSuccessfulTerminate() throws Exception
{ {
EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn(new WorkerSetupData("0", 0, null, null)); workerSetupData.set(new WorkerSetupData("0", 0, null, null));
EasyMock.replay(workerSetupManager);
EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.<List<String>>anyObject())) EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.<List<String>>anyObject()))
.andReturn(Lists.<String>newArrayList()); .andReturn(Lists.<String>newArrayList());
@ -212,15 +211,13 @@ public class SimpleResourceManagementStrategyTest
simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.TERMINATE simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.TERMINATE
); );
EasyMock.verify(workerSetupManager);
EasyMock.verify(autoScalingStrategy); EasyMock.verify(autoScalingStrategy);
} }
@Test @Test
public void testSomethingTerminating() throws Exception public void testSomethingTerminating() throws Exception
{ {
EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn(new WorkerSetupData("0", 0, null, null)); workerSetupData.set(new WorkerSetupData("0", 0, null, null));
EasyMock.replay(workerSetupManager);
EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.<List<String>>anyObject())) EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.<List<String>>anyObject()))
.andReturn(Lists.<String>newArrayList("ip")).times(2); .andReturn(Lists.<String>newArrayList("ip")).times(2);
@ -259,7 +256,6 @@ public class SimpleResourceManagementStrategyTest
simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.TERMINATE simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.TERMINATE
); );
EasyMock.verify(workerSetupManager);
EasyMock.verify(autoScalingStrategy); EasyMock.verify(autoScalingStrategy);
} }

12
pom.xml
View File

@ -38,7 +38,7 @@
<properties> <properties>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding> <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> </properties>
<modules> <modules>
@ -165,17 +165,17 @@
<dependency> <dependency>
<groupId>com.fasterxml.jackson.core</groupId> <groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-annotations</artifactId> <artifactId>jackson-annotations</artifactId>
<version>2.1.2</version> <version>2.1.4</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.fasterxml.jackson.core</groupId> <groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-core</artifactId> <artifactId>jackson-core</artifactId>
<version>2.1.3</version> <version>2.1.4</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.fasterxml.jackson.core</groupId> <groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId> <artifactId>jackson-databind</artifactId>
<version>2.1.4-mmx-2</version> <version>2.1.4</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.fasterxml.jackson.datatype</groupId> <groupId>com.fasterxml.jackson.datatype</groupId>
@ -190,12 +190,12 @@
<dependency> <dependency>
<groupId>com.fasterxml.jackson.dataformat</groupId> <groupId>com.fasterxml.jackson.dataformat</groupId>
<artifactId>jackson-dataformat-smile</artifactId> <artifactId>jackson-dataformat-smile</artifactId>
<version>2.1.3</version> <version>2.1.4</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>com.fasterxml.jackson.jaxrs</groupId> <groupId>com.fasterxml.jackson.jaxrs</groupId>
<artifactId>jackson-jaxrs-json-provider</artifactId> <artifactId>jackson-jaxrs-json-provider</artifactId>
<version>2.1.3</version> <version>2.1.4</version>
</dependency> </dependency>
<dependency> <dependency>
<groupId>org.codehaus.jackson</groupId> <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.QueryServlet;
import com.metamx.druid.http.StatusServlet; import com.metamx.druid.http.StatusServlet;
import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.Initialization;
import com.metamx.druid.initialization.ServerInit;
import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.loading.DataSegmentPusher; 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.query.QueryRunnerFactoryConglomerate;
import com.metamx.druid.utils.PropUtils; import com.metamx.druid.utils.PropUtils;
import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.metrics.Monitor; 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.Context;
import org.mortbay.jetty.servlet.ServletHolder; import org.mortbay.jetty.servlet.ServletHolder;
import org.skife.config.ConfigurationObjectFactory; import org.skife.config.ConfigurationObjectFactory;
@ -259,31 +249,7 @@ public class RealtimeNode extends BaseServerNode<RealtimeNode>
private void initializeSegmentPusher() private void initializeSegmentPusher()
{ {
if (dataSegmentPusher == null) { if (dataSegmentPusher == null) {
final Properties props = getProps(); dataSegmentPusher = ServerInit.getSegmentPusher(getProps(), getConfigFactory(), getJsonMapper());
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()
);
}
} }
} }

View File

@ -34,8 +34,8 @@ import com.metamx.druid.coordination.DruidClusterInfo;
import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseRuleManager;
import com.metamx.druid.db.DatabaseSegmentManager; import com.metamx.druid.db.DatabaseSegmentManager;
import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.DruidMaster;
import com.metamx.druid.client.indexing.IndexingServiceClient;
import com.metamx.druid.master.rules.Rule; import com.metamx.druid.master.rules.Rule;
import com.metamx.druid.merge.ClientKillQuery;
import org.joda.time.Interval; import org.joda.time.Interval;
import javax.annotation.Nullable; import javax.annotation.Nullable;
@ -66,6 +66,7 @@ public class InfoResource
private final DatabaseSegmentManager databaseSegmentManager; private final DatabaseSegmentManager databaseSegmentManager;
private final DatabaseRuleManager databaseRuleManager; private final DatabaseRuleManager databaseRuleManager;
private final DruidClusterInfo druidClusterInfo; private final DruidClusterInfo druidClusterInfo;
private final IndexingServiceClient indexingServiceClient;
@Inject @Inject
public InfoResource( public InfoResource(
@ -73,7 +74,8 @@ public class InfoResource
ServerInventoryManager serverInventoryManager, ServerInventoryManager serverInventoryManager,
DatabaseSegmentManager databaseSegmentManager, DatabaseSegmentManager databaseSegmentManager,
DatabaseRuleManager databaseRuleManager, DatabaseRuleManager databaseRuleManager,
DruidClusterInfo druidClusterInfo DruidClusterInfo druidClusterInfo,
IndexingServiceClient indexingServiceClient
) )
{ {
this.master = master; this.master = master;
@ -81,6 +83,7 @@ public class InfoResource
this.databaseSegmentManager = databaseSegmentManager; this.databaseSegmentManager = databaseSegmentManager;
this.databaseRuleManager = databaseRuleManager; this.databaseRuleManager = databaseRuleManager;
this.druidClusterInfo = druidClusterInfo; this.druidClusterInfo = druidClusterInfo;
this.indexingServiceClient = indexingServiceClient;
} }
@GET @GET
@ -374,7 +377,7 @@ public class InfoResource
) )
{ {
if (kill != null && Boolean.valueOf(kill)) { if (kill != null && Boolean.valueOf(kill)) {
master.killSegments(new ClientKillQuery(dataSourceName, new Interval(interval))); indexingServiceClient.killSegments(dataSourceName, new Interval(interval));
} else { } else {
if (!databaseSegmentManager.removeDatasource(dataSourceName)) { if (!databaseSegmentManager.removeDatasource(dataSourceName)) {
return Response.status(Response.Status.NOT_FOUND).build(); 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.common.logger.Logger;
import com.metamx.druid.client.ServerInventoryManager; import com.metamx.druid.client.ServerInventoryManager;
import com.metamx.druid.client.ServerInventoryManagerConfig; 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.DruidClusterInfo;
import com.metamx.druid.coordination.DruidClusterInfoConfig; import com.metamx.druid.coordination.DruidClusterInfoConfig;
import com.metamx.druid.db.DatabaseRuleManager; 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.log.LogLevelAdjuster;
import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.DruidMaster;
import com.metamx.druid.master.DruidMasterConfig; import com.metamx.druid.master.DruidMasterConfig;
import com.metamx.druid.client.indexing.IndexingServiceClient;
import com.metamx.druid.master.LoadQueuePeon; import com.metamx.druid.master.LoadQueuePeon;
import com.metamx.druid.utils.PropUtils; import com.metamx.druid.utils.PropUtils;
import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.EmittingLogger;
@ -86,7 +90,7 @@ import java.util.concurrent.ScheduledExecutorService;
*/ */
public class MasterMain 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 public static void main(String[] args) throws Exception
{ {
@ -166,13 +170,14 @@ public class MasterMain
lifecycle lifecycle
); );
ServiceProvider serviceProvider = null; IndexingServiceClient indexingServiceClient = null;
if (druidMasterConfig.getMergerServiceName() != null) { if (druidMasterConfig.getMergerServiceName() != null) {
serviceProvider = Initialization.makeServiceProvider( ServiceProvider serviceProvider = Initialization.makeServiceProvider(
druidMasterConfig.getMergerServiceName(), druidMasterConfig.getMergerServiceName(),
serviceDiscovery, serviceDiscovery,
lifecycle lifecycle
); );
indexingServiceClient = new IndexingServiceClient(httpClient, jsonMapper, serviceProvider);
} }
final DruidClusterInfo druidClusterInfo = new DruidClusterInfo( final DruidClusterInfo druidClusterInfo = new DruidClusterInfo(
@ -180,10 +185,14 @@ public class MasterMain
masterYp masterYp
); );
JacksonConfigManager configManager = new JacksonConfigManager(
new ConfigManager(dbi, configFactory.build(ConfigManagerConfig.class)), jsonMapper
);
final DruidMaster master = new DruidMaster( final DruidMaster master = new DruidMaster(
druidMasterConfig, druidMasterConfig,
druidClusterInfo, druidClusterInfo,
jsonMapper, configManager,
databaseSegmentManager, databaseSegmentManager,
serverInventoryManager, serverInventoryManager,
databaseRuleManager, databaseRuleManager,
@ -191,9 +200,7 @@ public class MasterMain
emitter, emitter,
scheduledExecutorFactory, scheduledExecutorFactory,
new ConcurrentHashMap<String, LoadQueuePeon>(), new ConcurrentHashMap<String, LoadQueuePeon>(),
serviceProvider, indexingServiceClient
httpClient,
new ToStringResponseHandler(Charsets.UTF_8)
); );
lifecycle.addManagedInstance(master); lifecycle.addManagedInstance(master);
@ -226,7 +233,8 @@ public class MasterMain
databaseRuleManager, databaseRuleManager,
druidClusterInfo, druidClusterInfo,
master, 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.DruidMaster;
import com.metamx.druid.master.LoadPeonCallback; import com.metamx.druid.master.LoadPeonCallback;
import com.metamx.druid.merge.ClientKillQuery;
import javax.inject.Inject; import javax.inject.Inject;
import javax.ws.rs.Consumes; 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.DatabaseRuleManager;
import com.metamx.druid.db.DatabaseSegmentManager; import com.metamx.druid.db.DatabaseSegmentManager;
import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.DruidMaster;
import com.metamx.druid.client.indexing.IndexingServiceClient;
import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.JerseyServletModule;
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
@ -44,6 +45,7 @@ public class MasterServletModule extends JerseyServletModule
private final DruidClusterInfo druidClusterInfo; private final DruidClusterInfo druidClusterInfo;
private final DruidMaster master; private final DruidMaster master;
private final ObjectMapper jsonMapper; private final ObjectMapper jsonMapper;
private final IndexingServiceClient indexingServiceClient;
public MasterServletModule( public MasterServletModule(
ServerInventoryManager serverInventoryManager, ServerInventoryManager serverInventoryManager,
@ -51,7 +53,8 @@ public class MasterServletModule extends JerseyServletModule
DatabaseRuleManager databaseRuleManager, DatabaseRuleManager databaseRuleManager,
DruidClusterInfo druidClusterInfo, DruidClusterInfo druidClusterInfo,
DruidMaster master, DruidMaster master,
ObjectMapper jsonMapper ObjectMapper jsonMapper,
IndexingServiceClient indexingServiceClient
) )
{ {
this.serverInventoryManager = serverInventoryManager; this.serverInventoryManager = serverInventoryManager;
@ -60,6 +63,7 @@ public class MasterServletModule extends JerseyServletModule
this.druidClusterInfo = druidClusterInfo; this.druidClusterInfo = druidClusterInfo;
this.master = master; this.master = master;
this.jsonMapper = jsonMapper; this.jsonMapper = jsonMapper;
this.indexingServiceClient = indexingServiceClient;
} }
@Override @Override
@ -72,6 +76,7 @@ public class MasterServletModule extends JerseyServletModule
bind(DatabaseRuleManager.class).toInstance(databaseRuleManager); bind(DatabaseRuleManager.class).toInstance(databaseRuleManager);
bind(DruidMaster.class).toInstance(master); bind(DruidMaster.class).toInstance(master);
bind(DruidClusterInfo.class).toInstance(druidClusterInfo); bind(DruidClusterInfo.class).toInstance(druidClusterInfo);
bind(IndexingServiceClient.class).toInstance(indexingServiceClient);
serve("/*").with(GuiceContainer.class); serve("/*").with(GuiceContainer.class);
} }

View File

@ -19,17 +19,24 @@
package com.metamx.druid.initialization; package com.metamx.druid.initialization;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Supplier; import com.google.common.base.Supplier;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.metamx.common.ISE; import com.metamx.common.ISE;
import com.metamx.common.logger.Logger; import com.metamx.common.logger.Logger;
import com.metamx.druid.DruidProcessingConfig; import com.metamx.druid.DruidProcessingConfig;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.loading.DelegatingSegmentLoader; import com.metamx.druid.loading.DelegatingSegmentLoader;
import com.metamx.druid.loading.LocalDataSegmentPuller; 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.MMappedQueryableIndexFactory;
import com.metamx.druid.loading.QueryableIndexFactory; import com.metamx.druid.loading.QueryableIndexFactory;
import com.metamx.druid.loading.S3DataSegmentPuller; 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.SegmentLoaderConfig;
import com.metamx.druid.loading.SingleSegmentLoader; import com.metamx.druid.loading.SingleSegmentLoader;
import com.metamx.druid.query.group.GroupByQueryEngine; 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.timeboundary.TimeBoundaryQueryRunnerFactory;
import com.metamx.druid.query.timeseries.TimeseriesQuery; import com.metamx.druid.query.timeseries.TimeseriesQuery;
import com.metamx.druid.query.timeseries.TimeseriesQueryRunnerFactory; 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.impl.rest.httpclient.RestS3Service;
import org.jets3t.service.security.AWSCredentials;
import org.skife.config.ConfigurationObjectFactory; import org.skife.config.ConfigurationObjectFactory;
import java.lang.reflect.InvocationTargetException; import java.lang.reflect.InvocationTargetException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Map; import java.util.Map;
import java.util.Properties;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
/** /**
@ -145,6 +156,34 @@ public class ServerInit
return queryRunners; 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 class ComputeScratchPool extends StupidPool<ByteBuffer>
{ {
private static final Logger log = new Logger(ComputeScratchPool.class); 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. * 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 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) segment = segment.withSize(indexSize)
.withLoadSpec( .withLoadSpec(
ImmutableMap.<String, Object>of( ImmutableMap.<String, Object>of("type", "s3_zip", "bucket", outputBucket, "key", toPush.getKey())
"type", "s3_zip",
"bucket", outputBucket,
"key", toPush.getKey()
)
) )
.withBinaryVersion(IndexIO.getVersionFromDir(indexFilesDir)); .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; package com.metamx.druid.master;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function; import com.google.common.base.Function;
import com.google.common.base.Predicate; import com.google.common.base.Predicate;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables; 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.DruidDataSource;
import com.metamx.druid.client.DruidServer; import com.metamx.druid.client.DruidServer;
import com.metamx.druid.client.ServerInventoryManager; 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.coordination.DruidClusterInfo;
import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseRuleManager;
import com.metamx.druid.db.DatabaseSegmentManager; 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.EmittingLogger;
import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent; 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.PhoneBook;
import com.metamx.phonebook.PhoneBookPeon; import com.metamx.phonebook.PhoneBookPeon;
import com.netflix.curator.x.discovery.ServiceProvider;
import org.I0Itec.zkclient.exception.ZkNodeExistsException; import org.I0Itec.zkclient.exception.ZkNodeExistsException;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.Duration; import org.joda.time.Duration;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.net.URL;
import java.util.Arrays; import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicReference;
/** /**
*/ */
@ -83,26 +79,22 @@ public class DruidMaster
private final DruidMasterConfig config; private final DruidMasterConfig config;
private final DruidClusterInfo clusterInfo; private final DruidClusterInfo clusterInfo;
private final JacksonConfigManager configManager;
private final DatabaseSegmentManager databaseSegmentManager; private final DatabaseSegmentManager databaseSegmentManager;
private final ServerInventoryManager serverInventoryManager; private final ServerInventoryManager serverInventoryManager;
private final DatabaseRuleManager databaseRuleManager; private final DatabaseRuleManager databaseRuleManager;
private final PhoneBook yp; private final PhoneBook yp;
private final ServiceEmitter emitter; private final ServiceEmitter emitter;
private final IndexingServiceClient indexingServiceClient;
private final ScheduledExecutorService exec; private final ScheduledExecutorService exec;
private final ScheduledExecutorService peonExec; private final ScheduledExecutorService peonExec;
private final PhoneBookPeon masterPeon; private final PhoneBookPeon masterPeon;
private final Map<String, LoadQueuePeon> loadManagementPeons; private final Map<String, LoadQueuePeon> loadManagementPeons;
private final ServiceProvider serviceProvider;
private final HttpClient httpClient;
private final HttpResponseHandler responseHandler;
private final ObjectMapper jsonMapper;
public DruidMaster( public DruidMaster(
DruidMasterConfig config, DruidMasterConfig config,
DruidClusterInfo clusterInfo, DruidClusterInfo clusterInfo,
ObjectMapper jsonMapper, JacksonConfigManager configManager,
DatabaseSegmentManager databaseSegmentManager, DatabaseSegmentManager databaseSegmentManager,
ServerInventoryManager serverInventoryManager, ServerInventoryManager serverInventoryManager,
DatabaseRuleManager databaseRuleManager, DatabaseRuleManager databaseRuleManager,
@ -110,31 +102,25 @@ public class DruidMaster
ServiceEmitter emitter, ServiceEmitter emitter,
ScheduledExecutorFactory scheduledExecutorFactory, ScheduledExecutorFactory scheduledExecutorFactory,
Map<String, LoadQueuePeon> loadManagementPeons, Map<String, LoadQueuePeon> loadManagementPeons,
ServiceProvider serviceProvider, IndexingServiceClient indexingServiceClient
HttpClient httpClient,
HttpResponseHandler responseHandler
) )
{ {
this.config = config; this.config = config;
this.clusterInfo = clusterInfo; this.clusterInfo = clusterInfo;
this.configManager = configManager;
this.jsonMapper = jsonMapper;
this.databaseSegmentManager = databaseSegmentManager; this.databaseSegmentManager = databaseSegmentManager;
this.serverInventoryManager = serverInventoryManager; this.serverInventoryManager = serverInventoryManager;
this.databaseRuleManager = databaseRuleManager; this.databaseRuleManager = databaseRuleManager;
this.yp = zkPhoneBook; this.yp = zkPhoneBook;
this.emitter = emitter; this.emitter = emitter;
this.indexingServiceClient = indexingServiceClient;
this.masterPeon = new MasterListeningPeon(); this.masterPeon = new MasterListeningPeon();
this.exec = scheduledExecutorFactory.create(1, "Master-Exec--%d"); this.exec = scheduledExecutorFactory.create(1, "Master-Exec--%d");
this.peonExec = scheduledExecutorFactory.create(1, "Master-PeonExec--%d"); this.peonExec = scheduledExecutorFactory.create(1, "Master-PeonExec--%d");
this.loadManagementPeons = loadManagementPeons; this.loadManagementPeons = loadManagementPeons;
this.serviceProvider = serviceProvider;
this.httpClient = httpClient;
this.responseHandler = responseHandler;
} }
public boolean isClusterMaster() 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() public Set<DataSegment> getAvailableDataSegments()
{ {
Set<DataSegment> availableSegments = Sets.newTreeSet(Comparators.inverse(DataSegment.bucketMonthComparator())); Set<DataSegment> availableSegments = Sets.newTreeSet(Comparators.inverse(DataSegment.bucketMonthComparator()));
@ -390,7 +355,9 @@ public class DruidMaster
for (DataSegment dataSegment : dataSegments) { for (DataSegment dataSegment : dataSegments) {
if (dataSegment.getSize() < 0) { 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); availableSegments.add(dataSegment);
} }
@ -466,8 +433,14 @@ public class DruidMaster
final List<Pair<? extends MasterRunnable, Duration>> masterRunnables = Lists.newArrayList(); final List<Pair<? extends MasterRunnable, Duration>> masterRunnables = Lists.newArrayList();
masterRunnables.add(Pair.of(new MasterComputeManagerRunnable(), config.getMasterPeriod())); masterRunnables.add(Pair.of(new MasterComputeManagerRunnable(), config.getMasterPeriod()));
if (config.isMergeSegments() && serviceProvider != null) { if (config.isMergeSegments() && indexingServiceClient != null) {
masterRunnables.add(Pair.of(new MasterSegmentMergerRunnable(), config.getMasterSegmentMergerPeriod()));
masterRunnables.add(
Pair.of(
new MasterSegmentMergerRunnable(configManager.watch(MergerWhitelist.CONFIG_KEY, MergerWhitelist.class)),
config.getMasterSegmentMergerPeriod()
)
);
} }
for (final Pair<? extends MasterRunnable, Duration> masterRunnable : masterRunnables) { 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> private class MasterListeningPeon implements PhoneBookPeon<Map>
{ {
@Override @Override
@ -723,12 +729,13 @@ public class DruidMaster
private class MasterSegmentMergerRunnable extends MasterRunnable private class MasterSegmentMergerRunnable extends MasterRunnable
{ {
private MasterSegmentMergerRunnable() private MasterSegmentMergerRunnable(final AtomicReference<MergerWhitelist> whitelistRef)
{ {
super( super(
ImmutableList.of( ImmutableList.of(
new DruidMasterSegmentInfoLoader(DruidMaster.this), new DruidMasterSegmentInfoLoader(DruidMaster.this),
new DruidMasterSegmentMerger(jsonMapper, serviceProvider), new DruidMasterVersionConverter(indexingServiceClient, whitelistRef),
new DruidMasterSegmentMerger(indexingServiceClient, whitelistRef),
new DruidMasterHelper() new DruidMasterHelper()
{ {
@Override @Override
@ -739,8 +746,7 @@ public class DruidMaster
params.getEmitter().emit( params.getEmitter().emit(
new ServiceMetricEvent.Builder().build( new ServiceMetricEvent.Builder().build(
"master/merge/count", "master/merge/count", stats.getGlobalStats().get("mergedCount")
stats.getGlobalStats().get("mergedCount")
) )
); );

View File

@ -19,8 +19,6 @@
package com.metamx.druid.master; 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.Function;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.HashMultiset; import com.google.common.collect.HashMultiset;
@ -32,22 +30,19 @@ import com.google.common.collect.Multiset;
import com.google.common.collect.Ordering; import com.google.common.collect.Ordering;
import com.metamx.common.Pair; import com.metamx.common.Pair;
import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.guava.FunctionalIterable;
import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.logger.Logger; import com.metamx.common.logger.Logger;
import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.TimelineObjectHolder;
import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.VersionedIntervalTimeline;
import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DataSegment;
import com.metamx.druid.client.indexing.IndexingServiceClient;
import com.metamx.druid.partition.PartitionChunk; 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.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
import java.util.List; import java.util.List;
import java.util.Map; 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 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; this.indexingServiceClient = indexingServiceClient;
} this.whiteListRef = whitelistRef;
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
);
} }
@Override @Override
public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params) public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params)
{ {
MergerWhitelist whitelist = whiteListRef.get();
MasterStats stats = new MasterStats(); MasterStats stats = new MasterStats();
Map<String, VersionedIntervalTimeline<String, DataSegment>> dataSources = Maps.newHashMap(); Map<String, VersionedIntervalTimeline<String, DataSegment>> dataSources = Maps.newHashMap();
// Find serviced segments by using a timeline // Find serviced segments by using a timeline
for (DataSegment dataSegment : params.getAvailableSegments()) { for (DataSegment dataSegment : params.getAvailableSegments()) {
VersionedIntervalTimeline<String, DataSegment> timeline = dataSources.get(dataSegment.getDataSource()); if (whitelist == null || whitelist.contains(dataSegment.getDataSource())) {
if (timeline == null) { VersionedIntervalTimeline<String, DataSegment> timeline = dataSources.get(dataSegment.getDataSource());
timeline = new VersionedIntervalTimeline<String, DataSegment>(Ordering.<String>natural()); if (timeline == null) {
dataSources.put(dataSegment.getDataSource(), timeline); 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 // 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); log.info("[%s] Found %d segments to merge %s", dataSource, segments.size(), segmentNames);
try { try {
mergerClient.runRequest(dataSource, segments); indexingServiceClient.mergeSegments(segments);
} }
catch (Exception e) { catch (Exception e) {
log.error( 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.ImmutableSet;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DataSegment;
import com.metamx.druid.client.indexing.IndexingServiceClient;
import junit.framework.Assert; import junit.framework.Assert;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.junit.Test; import org.junit.Test;
import java.util.Collection; import java.util.Collection;
import java.util.List; import java.util.List;
import java.util.concurrent.atomic.AtomicReference;
public class DruidMasterSegmentMergerTest 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() DataSegment.builder().dataSource("foo").interval(new Interval("2012-01-06/P1D")).version("2").size(80).build()
); );
Assert.assertEquals( Assert.assertEquals(ImmutableList.of(ImmutableList.of(segments.get(4), segments.get(5))), merge(segments));
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) private static List<List<DataSegment>> merge(final Collection<DataSegment> segments)
{ {
final List<List<DataSegment>> retVal = Lists.newArrayList(); final List<List<DataSegment>> retVal = Lists.newArrayList();
final MergerClient mergerClient = new MergerClient() final IndexingServiceClient indexingServiceClient = new IndexingServiceClient(null, null, null)
{ {
@Override @Override
public void runRequest(String dataSource, List<DataSegment> segmentsToMerge) public void mergeSegments(List<DataSegment> segmentsToMerge)
{ {
retVal.add(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() final DruidMasterRuntimeParams params = DruidMasterRuntimeParams.newBuilder()
.withAvailableSegments(ImmutableSet.copyOf(segments)) .withAvailableSegments(ImmutableSet.copyOf(segments))
.withMergeBytesLimit(mergeBytesLimit) .withMergeBytesLimit(mergeBytesLimit)

View File

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