mirror of https://github.com/apache/druid.git
Merge pull request #97 from metamx/convert_task
Index Conversion Task and adjustment to TaskAction API
This commit is contained in:
commit
c3a871b942
|
@ -261,6 +261,7 @@ public class DataSegment implements Comparable<DataSegment>
|
|||
", loadSpec=" + loadSpec +
|
||||
", interval=" + interval +
|
||||
", dataSource='" + dataSource + '\'' +
|
||||
", binaryVersion='" + binaryVersion + '\'' +
|
||||
'}';
|
||||
}
|
||||
|
||||
|
|
|
@ -17,7 +17,7 @@
|
|||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package com.metamx.druid.merge;
|
||||
package com.metamx.druid.client.indexing;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
|
@ -29,7 +29,7 @@ import java.util.List;
|
|||
|
||||
/**
|
||||
*/
|
||||
public class ClientAppendQuery implements ClientMergeQuery
|
||||
public class ClientAppendQuery
|
||||
{
|
||||
private final String dataSource;
|
||||
private final List<DataSegment> segments;
|
||||
|
@ -45,14 +45,18 @@ public class ClientAppendQuery implements ClientMergeQuery
|
|||
}
|
||||
|
||||
@JsonProperty
|
||||
@Override
|
||||
public String getType()
|
||||
{
|
||||
return "append";
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getDataSource()
|
||||
{
|
||||
return dataSource;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Override
|
||||
public List<DataSegment> getSegments()
|
||||
{
|
||||
return segments;
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -17,21 +17,21 @@
|
|||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package com.metamx.druid.merge;
|
||||
|
||||
|
||||
package com.metamx.druid.client.indexing;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
public class ClientDeleteQuery
|
||||
/**
|
||||
*/
|
||||
public class ClientKillQuery
|
||||
{
|
||||
private final String dataSource;
|
||||
private final Interval interval;
|
||||
|
||||
@JsonCreator
|
||||
public ClientDeleteQuery(
|
||||
public ClientKillQuery(
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("interval") Interval interval
|
||||
)
|
||||
|
@ -40,22 +40,21 @@ public class ClientDeleteQuery
|
|||
this.interval = interval;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getType()
|
||||
{
|
||||
return "kill";
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getDataSource()
|
||||
{
|
||||
return dataSource;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Interval getInterval()
|
||||
{
|
||||
return interval;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "ClientDeleteQuery{" +
|
||||
"dataSource='" + dataSource + '\'' +
|
||||
", interval=" + interval +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -17,7 +17,7 @@
|
|||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package com.metamx.druid.merge;
|
||||
package com.metamx.druid.client.indexing;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
|
@ -30,14 +30,14 @@ import java.util.List;
|
|||
|
||||
/**
|
||||
*/
|
||||
public class ClientDefaultMergeQuery implements ClientMergeQuery
|
||||
public class ClientMergeQuery
|
||||
{
|
||||
private final String dataSource;
|
||||
private final List<DataSegment> segments;
|
||||
private final List<AggregatorFactory> aggregators;
|
||||
|
||||
@JsonCreator
|
||||
public ClientDefaultMergeQuery(
|
||||
public ClientMergeQuery(
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("segments") List<DataSegment> segments,
|
||||
@JsonProperty("aggregations") List<AggregatorFactory> aggregators
|
||||
|
@ -50,14 +50,18 @@ public class ClientDefaultMergeQuery implements ClientMergeQuery
|
|||
}
|
||||
|
||||
@JsonProperty
|
||||
@Override
|
||||
public String getType()
|
||||
{
|
||||
return "append";
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getDataSource()
|
||||
{
|
||||
return dataSource;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
@Override
|
||||
public List<DataSegment> getSegments()
|
||||
{
|
||||
return segments;
|
||||
|
@ -72,7 +76,7 @@ public class ClientDefaultMergeQuery implements ClientMergeQuery
|
|||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "ClientDefaultMergeQuery{" +
|
||||
return "ClientMergeQuery{" +
|
||||
"dataSource='" + dataSource + '\'' +
|
||||
", segments=" + segments +
|
||||
", aggregators=" + aggregators +
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
|
||||
}
|
|
@ -0,0 +1,9 @@
|
|||
package com.metamx.druid.config;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface ConfigSerde<T>
|
||||
{
|
||||
public byte[] serialize(T obj);
|
||||
public T deserialize(byte[] bytes);
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -65,7 +65,7 @@ public class DbConnector
|
|||
dbi,
|
||||
configTableName,
|
||||
String.format(
|
||||
"CREATE table %s (name VARCHAR(255) NOT NULL, payload LONGTEXT NOT NULL, PRIMARY KEY(name))",
|
||||
"CREATE table %s (name VARCHAR(255) NOT NULL, payload BLOB NOT NULL, PRIMARY KEY(name))",
|
||||
configTableName
|
||||
)
|
||||
);
|
||||
|
@ -84,12 +84,7 @@ public class DbConnector
|
|||
@Override
|
||||
public Void withHandle(Handle handle) throws Exception
|
||||
{
|
||||
List<Map<String, Object>> table = handle.select(
|
||||
String.format(
|
||||
"SHOW tables LIKE '%s'",
|
||||
tableName
|
||||
)
|
||||
);
|
||||
List<Map<String, Object>> table = handle.select(String.format("SHOW tables LIKE '%s'", tableName));
|
||||
|
||||
if (table.isEmpty()) {
|
||||
log.info("Creating table[%s]", tableName);
|
||||
|
|
|
@ -199,6 +199,19 @@ public class IndexIO
|
|||
}
|
||||
}
|
||||
|
||||
public static boolean convertSegment(File toConvert, File converted) throws IOException
|
||||
{
|
||||
final int version = getVersionFromDir(toConvert);
|
||||
|
||||
switch (version) {
|
||||
case 8:
|
||||
DefaultIndexIOHandler.convertV8toV9(toConvert, converted);
|
||||
return true;
|
||||
default:
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
public static interface IndexIOHandler
|
||||
{
|
||||
/**
|
||||
|
@ -229,7 +242,7 @@ public class IndexIO
|
|||
public void storeLatest(Index index, File file) throws IOException;
|
||||
}
|
||||
|
||||
static class DefaultIndexIOHandler implements IndexIOHandler
|
||||
public static class DefaultIndexIOHandler implements IndexIOHandler
|
||||
{
|
||||
private static final Logger log = new Logger(DefaultIndexIOHandler.class);
|
||||
@Override
|
||||
|
|
|
@ -25,11 +25,12 @@ import com.metamx.druid.client.DataSegment;
|
|||
import com.metamx.druid.loading.DataSegmentPusher;
|
||||
import com.metamx.druid.loading.MMappedQueryableIndexFactory;
|
||||
import com.metamx.druid.loading.S3DataSegmentPuller;
|
||||
import com.metamx.druid.loading.SegmentKiller;
|
||||
import com.metamx.druid.loading.DataSegmentKiller;
|
||||
import com.metamx.druid.loading.SegmentLoaderConfig;
|
||||
import com.metamx.druid.loading.SegmentLoadingException;
|
||||
import com.metamx.druid.loading.SingleSegmentLoader;
|
||||
import com.metamx.druid.merger.common.actions.TaskActionClient;
|
||||
import com.metamx.druid.merger.common.actions.TaskActionClientFactory;
|
||||
import com.metamx.druid.merger.common.config.TaskConfig;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
|
@ -45,29 +46,32 @@ import java.util.Map;
|
|||
public class TaskToolbox
|
||||
{
|
||||
private final TaskConfig config;
|
||||
private final TaskActionClient taskActionClient;
|
||||
private final Task task;
|
||||
private final TaskActionClientFactory taskActionClientFactory;
|
||||
private final ServiceEmitter emitter;
|
||||
private final RestS3Service s3Client;
|
||||
private final DataSegmentPusher segmentPusher;
|
||||
private final SegmentKiller segmentKiller;
|
||||
private final DataSegmentKiller dataSegmentKiller;
|
||||
private final ObjectMapper objectMapper;
|
||||
|
||||
public TaskToolbox(
|
||||
TaskConfig config,
|
||||
TaskActionClient taskActionClient,
|
||||
Task task,
|
||||
TaskActionClientFactory taskActionClientFactory,
|
||||
ServiceEmitter emitter,
|
||||
RestS3Service s3Client,
|
||||
DataSegmentPusher segmentPusher,
|
||||
SegmentKiller segmentKiller,
|
||||
DataSegmentKiller dataSegmentKiller,
|
||||
ObjectMapper objectMapper
|
||||
)
|
||||
{
|
||||
this.config = config;
|
||||
this.taskActionClient = taskActionClient;
|
||||
this.task = task;
|
||||
this.taskActionClientFactory = taskActionClientFactory;
|
||||
this.emitter = emitter;
|
||||
this.s3Client = s3Client;
|
||||
this.segmentPusher = segmentPusher;
|
||||
this.segmentKiller = segmentKiller;
|
||||
this.dataSegmentKiller = dataSegmentKiller;
|
||||
this.objectMapper = objectMapper;
|
||||
}
|
||||
|
||||
|
@ -76,9 +80,9 @@ public class TaskToolbox
|
|||
return config;
|
||||
}
|
||||
|
||||
public TaskActionClient getTaskActionClient()
|
||||
public TaskActionClient getTaskActionClientFactory()
|
||||
{
|
||||
return taskActionClient;
|
||||
return taskActionClientFactory.create(task);
|
||||
}
|
||||
|
||||
public ServiceEmitter getEmitter()
|
||||
|
@ -91,9 +95,9 @@ public class TaskToolbox
|
|||
return segmentPusher;
|
||||
}
|
||||
|
||||
public SegmentKiller getSegmentKiller()
|
||||
public DataSegmentKiller getDataSegmentKiller()
|
||||
{
|
||||
return segmentKiller;
|
||||
return dataSegmentKiller;
|
||||
}
|
||||
|
||||
public ObjectMapper getObjectMapper()
|
||||
|
@ -101,7 +105,7 @@ public class TaskToolbox
|
|||
return objectMapper;
|
||||
}
|
||||
|
||||
public Map<DataSegment, File> getSegments(final Task task, List<DataSegment> segments)
|
||||
public Map<DataSegment, File> getSegments(List<DataSegment> segments)
|
||||
throws SegmentLoadingException
|
||||
{
|
||||
final SingleSegmentLoader loader = new SingleSegmentLoader(
|
||||
|
@ -112,7 +116,7 @@ public class TaskToolbox
|
|||
@Override
|
||||
public File getCacheDirectory()
|
||||
{
|
||||
return new File(config.getTaskDir(task), "fetched_segments");
|
||||
return new File(getTaskDir(), "fetched_segments");
|
||||
}
|
||||
}
|
||||
);
|
||||
|
@ -124,4 +128,9 @@ public class TaskToolbox
|
|||
|
||||
return retVal;
|
||||
}
|
||||
|
||||
public File getTaskDir() {
|
||||
return new File(config.getBaseTaskDir(), task.getId());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
}
|
|
@ -1,17 +1,20 @@
|
|||
package com.metamx.druid.merger.common.actions;
|
||||
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.merger.coordinator.TaskStorage;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
|
||||
public class LocalTaskActionClient implements TaskActionClient
|
||||
{
|
||||
private final Task task;
|
||||
private final TaskStorage storage;
|
||||
private final TaskActionToolbox toolbox;
|
||||
|
||||
private static final EmittingLogger log = new EmittingLogger(LocalTaskActionClient.class);
|
||||
|
||||
public LocalTaskActionClient(TaskStorage storage, TaskActionToolbox toolbox)
|
||||
public LocalTaskActionClient(Task task, TaskStorage storage, TaskActionToolbox toolbox)
|
||||
{
|
||||
this.task = task;
|
||||
this.storage = storage;
|
||||
this.toolbox = toolbox;
|
||||
}
|
||||
|
@ -19,15 +22,15 @@ public class LocalTaskActionClient implements TaskActionClient
|
|||
@Override
|
||||
public <RetType> RetType submit(TaskAction<RetType> taskAction)
|
||||
{
|
||||
final RetType ret = taskAction.perform(toolbox);
|
||||
final RetType ret = taskAction.perform(task, toolbox);
|
||||
|
||||
// Add audit log
|
||||
try {
|
||||
storage.addAuditLog(taskAction);
|
||||
storage.addAuditLog(task, taskAction);
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.makeAlert(e, "Failed to record action in audit log")
|
||||
.addData("task", taskAction.getTask().getId())
|
||||
.addData("task", task.getId())
|
||||
.addData("actionClass", taskAction.getClass().getName())
|
||||
.emit();
|
||||
}
|
||||
|
|
|
@ -17,25 +17,27 @@
|
|||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package com.metamx.druid.merge;
|
||||
package com.metamx.druid.merger.common.actions;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
|
||||
|
||||
|
||||
import java.util.List;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.merger.coordinator.TaskStorage;
|
||||
|
||||
/**
|
||||
*/
|
||||
@JsonTypeInfo(use= JsonTypeInfo.Id.NAME, property="type", defaultImpl = ClientDefaultMergeQuery.class)
|
||||
@JsonSubTypes(value={
|
||||
@JsonSubTypes.Type(name="append", value=ClientAppendQuery.class)
|
||||
})
|
||||
public interface ClientMergeQuery
|
||||
public class LocalTaskActionClientFactory implements TaskActionClientFactory
|
||||
{
|
||||
public String getDataSource();
|
||||
private final TaskStorage storage;
|
||||
private final TaskActionToolbox toolbox;
|
||||
|
||||
public List<DataSegment> getSegments();
|
||||
public LocalTaskActionClientFactory(TaskStorage storage, TaskActionToolbox toolbox)
|
||||
{
|
||||
this.storage = storage;
|
||||
this.toolbox = toolbox;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskActionClient create(Task task)
|
||||
{
|
||||
return new LocalTaskActionClient(task, storage, toolbox);
|
||||
}
|
||||
}
|
|
@ -11,25 +11,16 @@ import org.joda.time.Interval;
|
|||
|
||||
public class LockAcquireAction implements TaskAction<Optional<TaskLock>>
|
||||
{
|
||||
private final Task task;
|
||||
private final Interval interval;
|
||||
|
||||
@JsonCreator
|
||||
public LockAcquireAction(
|
||||
@JsonProperty("task") Task task,
|
||||
@JsonProperty("interval") Interval interval
|
||||
)
|
||||
{
|
||||
this.task = task;
|
||||
this.interval = interval;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Task getTask()
|
||||
{
|
||||
return task;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Interval getInterval()
|
||||
{
|
||||
|
@ -42,7 +33,7 @@ public class LockAcquireAction implements TaskAction<Optional<TaskLock>>
|
|||
}
|
||||
|
||||
@Override
|
||||
public Optional<TaskLock> perform(TaskActionToolbox toolbox)
|
||||
public Optional<TaskLock> perform(Task task, TaskActionToolbox toolbox)
|
||||
{
|
||||
try {
|
||||
return toolbox.getTaskLockbox().tryLock(task, interval);
|
||||
|
|
|
@ -12,29 +12,13 @@ import java.util.List;
|
|||
|
||||
public class LockListAction implements TaskAction<List<TaskLock>>
|
||||
{
|
||||
private final Task task;
|
||||
|
||||
@JsonCreator
|
||||
public LockListAction(
|
||||
@JsonProperty("task") Task task
|
||||
)
|
||||
{
|
||||
this.task = task;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Task getTask()
|
||||
{
|
||||
return task;
|
||||
}
|
||||
|
||||
public TypeReference<List<TaskLock>> getReturnTypeReference()
|
||||
{
|
||||
return new TypeReference<List<TaskLock>>() {};
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<TaskLock> perform(TaskActionToolbox toolbox)
|
||||
public List<TaskLock> perform(Task task, TaskActionToolbox toolbox)
|
||||
{
|
||||
try {
|
||||
return toolbox.getTaskLockbox().findLocksForTask(task);
|
||||
|
|
|
@ -12,25 +12,16 @@ import java.util.List;
|
|||
|
||||
public class LockReleaseAction implements TaskAction<Void>
|
||||
{
|
||||
private final Task task;
|
||||
private final Interval interval;
|
||||
|
||||
@JsonCreator
|
||||
public LockReleaseAction(
|
||||
@JsonProperty("task") Task task,
|
||||
@JsonProperty("interval") Interval interval
|
||||
)
|
||||
{
|
||||
this.task = task;
|
||||
this.interval = interval;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Task getTask()
|
||||
{
|
||||
return task;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Interval getInterval()
|
||||
{
|
||||
|
@ -43,7 +34,7 @@ public class LockReleaseAction implements TaskAction<Void>
|
|||
}
|
||||
|
||||
@Override
|
||||
public Void perform(TaskActionToolbox toolbox)
|
||||
public Void perform(Task task, TaskActionToolbox toolbox)
|
||||
{
|
||||
try {
|
||||
toolbox.getTaskLockbox().unlock(task, interval);
|
||||
|
|
|
@ -4,6 +4,7 @@ import com.fasterxml.jackson.core.type.TypeReference;
|
|||
import com.google.common.base.Charsets;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.http.client.HttpClient;
|
||||
import com.metamx.http.client.response.ToStringResponseHandler;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
@ -16,14 +17,16 @@ import java.util.Map;
|
|||
|
||||
public class RemoteTaskActionClient implements TaskActionClient
|
||||
{
|
||||
private final Task task;
|
||||
private final HttpClient httpClient;
|
||||
private final ServiceProvider serviceProvider;
|
||||
private final ObjectMapper jsonMapper;
|
||||
|
||||
private static final Logger log = new Logger(RemoteTaskActionClient.class);
|
||||
|
||||
public RemoteTaskActionClient(HttpClient httpClient, ServiceProvider serviceProvider, ObjectMapper jsonMapper)
|
||||
public RemoteTaskActionClient(Task task, HttpClient httpClient, ServiceProvider serviceProvider, ObjectMapper jsonMapper)
|
||||
{
|
||||
this.task = task;
|
||||
this.httpClient = httpClient;
|
||||
this.serviceProvider = serviceProvider;
|
||||
this.jsonMapper = jsonMapper;
|
||||
|
@ -33,7 +36,7 @@ public class RemoteTaskActionClient implements TaskActionClient
|
|||
public <RetType> RetType submit(TaskAction<RetType> taskAction)
|
||||
{
|
||||
try {
|
||||
byte[] dataToSend = jsonMapper.writeValueAsBytes(taskAction);
|
||||
byte[] dataToSend = jsonMapper.writeValueAsBytes(new TaskActionHolder(task, taskAction));
|
||||
|
||||
final String response = httpClient.post(getServiceUri().toURL())
|
||||
.setContent("application/json", dataToSend)
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -18,25 +18,16 @@ import java.util.Set;
|
|||
|
||||
public class SegmentInsertAction implements TaskAction<Void>
|
||||
{
|
||||
private final Task task;
|
||||
private final Set<DataSegment> segments;
|
||||
|
||||
@JsonCreator
|
||||
public SegmentInsertAction(
|
||||
@JsonProperty("task") Task task,
|
||||
@JsonProperty("segments") Set<DataSegment> segments
|
||||
)
|
||||
{
|
||||
this.task = task;
|
||||
this.segments = ImmutableSet.copyOf(segments);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Task getTask()
|
||||
{
|
||||
return task;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Set<DataSegment> getSegments()
|
||||
{
|
||||
|
@ -49,7 +40,7 @@ public class SegmentInsertAction implements TaskAction<Void>
|
|||
}
|
||||
|
||||
@Override
|
||||
public Void perform(TaskActionToolbox toolbox)
|
||||
public Void perform(Task task, TaskActionToolbox toolbox)
|
||||
{
|
||||
if(!toolbox.taskLockCoversSegments(task, segments, false)) {
|
||||
throw new ISE("Segments not covered by locks for task: %s", task.getId());
|
||||
|
|
|
@ -12,28 +12,19 @@ import java.util.List;
|
|||
|
||||
public class SegmentListUnusedAction implements TaskAction<List<DataSegment>>
|
||||
{
|
||||
private final Task task;
|
||||
private final String dataSource;
|
||||
private final Interval interval;
|
||||
|
||||
@JsonCreator
|
||||
public SegmentListUnusedAction(
|
||||
@JsonProperty("task") Task task,
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("interval") Interval interval
|
||||
)
|
||||
{
|
||||
this.task = task;
|
||||
this.dataSource = dataSource;
|
||||
this.interval = interval;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Task getTask()
|
||||
{
|
||||
return task;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getDataSource()
|
||||
{
|
||||
|
@ -52,7 +43,7 @@ public class SegmentListUnusedAction implements TaskAction<List<DataSegment>>
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<DataSegment> perform(TaskActionToolbox toolbox)
|
||||
public List<DataSegment> perform(Task task, TaskActionToolbox toolbox)
|
||||
{
|
||||
try {
|
||||
return toolbox.getMergerDBCoordinator().getUnusedSegmentsForInterval(dataSource, interval);
|
||||
|
|
|
@ -12,28 +12,19 @@ import java.util.List;
|
|||
|
||||
public class SegmentListUsedAction implements TaskAction<List<DataSegment>>
|
||||
{
|
||||
private final Task task;
|
||||
private final String dataSource;
|
||||
private final Interval interval;
|
||||
|
||||
@JsonCreator
|
||||
public SegmentListUsedAction(
|
||||
@JsonProperty("task") Task task,
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("interval") Interval interval
|
||||
)
|
||||
{
|
||||
this.task = task;
|
||||
this.dataSource = dataSource;
|
||||
this.interval = interval;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Task getTask()
|
||||
{
|
||||
return task;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getDataSource()
|
||||
{
|
||||
|
@ -52,7 +43,7 @@ public class SegmentListUsedAction implements TaskAction<List<DataSegment>>
|
|||
}
|
||||
|
||||
@Override
|
||||
public List<DataSegment> perform(TaskActionToolbox toolbox)
|
||||
public List<DataSegment> perform(Task task, TaskActionToolbox toolbox)
|
||||
{
|
||||
try {
|
||||
return toolbox.getMergerDBCoordinator().getUsedSegmentsForInterval(dataSource, interval);
|
||||
|
|
|
@ -18,25 +18,16 @@ import java.util.Set;
|
|||
|
||||
public class SegmentNukeAction implements TaskAction<Void>
|
||||
{
|
||||
private final Task task;
|
||||
private final Set<DataSegment> segments;
|
||||
|
||||
@JsonCreator
|
||||
public SegmentNukeAction(
|
||||
@JsonProperty("task") Task task,
|
||||
@JsonProperty("segments") Set<DataSegment> segments
|
||||
)
|
||||
{
|
||||
this.task = task;
|
||||
this.segments = ImmutableSet.copyOf(segments);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Task getTask()
|
||||
{
|
||||
return task;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Set<DataSegment> getSegments()
|
||||
{
|
||||
|
@ -49,7 +40,7 @@ public class SegmentNukeAction implements TaskAction<Void>
|
|||
}
|
||||
|
||||
@Override
|
||||
public Void perform(TaskActionToolbox toolbox)
|
||||
public Void perform(Task task, TaskActionToolbox toolbox)
|
||||
{
|
||||
if(!toolbox.taskLockCoversSegments(task, segments, true)) {
|
||||
throw new ISE("Segments not covered by locks for task: %s", task.getId());
|
||||
|
|
|
@ -11,25 +11,16 @@ import java.util.List;
|
|||
|
||||
public class SpawnTasksAction implements TaskAction<Void>
|
||||
{
|
||||
private final Task task;
|
||||
private final List<Task> newTasks;
|
||||
|
||||
@JsonCreator
|
||||
public SpawnTasksAction(
|
||||
@JsonProperty("task") Task task,
|
||||
@JsonProperty("newTasks") List<Task> newTasks
|
||||
)
|
||||
{
|
||||
this.task = task;
|
||||
this.newTasks = ImmutableList.copyOf(newTasks);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Task getTask()
|
||||
{
|
||||
return task;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public List<Task> getNewTasks()
|
||||
{
|
||||
|
@ -42,7 +33,7 @@ public class SpawnTasksAction implements TaskAction<Void>
|
|||
}
|
||||
|
||||
@Override
|
||||
public Void perform(TaskActionToolbox toolbox)
|
||||
public Void perform(Task task, TaskActionToolbox toolbox)
|
||||
{
|
||||
try {
|
||||
for(final Task newTask : newTasks) {
|
||||
|
|
|
@ -18,7 +18,6 @@ import com.fasterxml.jackson.core.type.TypeReference;
|
|||
})
|
||||
public interface TaskAction<RetType>
|
||||
{
|
||||
public Task getTask(); // TODO Look into replacing this with task ID so stuff serializes smaller
|
||||
public TypeReference<RetType> getReturnTypeReference(); // T_T
|
||||
public RetType perform(TaskActionToolbox toolbox);
|
||||
public RetType perform(Task task, TaskActionToolbox toolbox);
|
||||
}
|
||||
|
|
|
@ -17,23 +17,13 @@
|
|||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package com.metamx.druid.merger.coordinator.config;
|
||||
package com.metamx.druid.merger.common.actions;
|
||||
|
||||
import org.joda.time.Duration;
|
||||
import org.skife.config.Config;
|
||||
import org.skife.config.Default;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
|
||||
/**
|
||||
*/
|
||||
public abstract class WorkerSetupManagerConfig
|
||||
public interface TaskActionClientFactory
|
||||
{
|
||||
@Config("druid.indexer.configTable")
|
||||
public abstract String getConfigTable();
|
||||
|
||||
@Config("druid.indexer.workerSetupConfigName")
|
||||
public abstract String getWorkerSetupConfigName();
|
||||
|
||||
@Config("druid.indexer.poll.duration")
|
||||
@Default("PT1M")
|
||||
public abstract Duration getPollDuration();
|
||||
public TaskActionClient create(Task task);
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -1,3 +1,22 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package com.metamx.druid.merger.common.config;
|
||||
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
|
@ -17,8 +36,4 @@ public abstract class TaskConfig
|
|||
|
||||
@Config("druid.merger.hadoopWorkingPath")
|
||||
public abstract String getHadoopWorkingPath();
|
||||
|
||||
public File getTaskDir(final Task task) {
|
||||
return new File(getBaseTaskDir(), task.getId());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,15 +20,19 @@
|
|||
package com.metamx.druid.merger.common.task;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Objects;
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.common.actions.SegmentListUsedAction;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
public abstract class AbstractTask implements Task
|
||||
{
|
||||
private static final Joiner ID_JOINER = Joiner.on("_");
|
||||
|
||||
private final String id;
|
||||
private final String groupId;
|
||||
private final String dataSource;
|
||||
|
@ -91,4 +95,20 @@ public abstract class AbstractTask implements Task
|
|||
.add("interval", getImplicitLockInterval())
|
||||
.toString();
|
||||
}
|
||||
|
||||
/** Start helper methods **/
|
||||
public static String joinId(Object... objects)
|
||||
{
|
||||
return ID_JOINER.join(objects);
|
||||
}
|
||||
|
||||
public SegmentListUsedAction defaultListUsedAction()
|
||||
{
|
||||
return new SegmentListUsedAction(getDataSource(), getImplicitLockInterval().get());
|
||||
}
|
||||
|
||||
public TaskStatus success()
|
||||
{
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -44,7 +44,7 @@ import java.util.Map;
|
|||
|
||||
/**
|
||||
*/
|
||||
public class AppendTask extends MergeTask
|
||||
public class AppendTask extends MergeTaskBase
|
||||
{
|
||||
@JsonCreator
|
||||
public AppendTask(
|
||||
|
|
|
@ -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";
|
||||
}
|
||||
}
|
|
@ -77,7 +77,7 @@ public class DeleteTask extends AbstractTask
|
|||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
// Strategy: Create an empty segment covering the interval to be deleted
|
||||
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this)));
|
||||
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction()));
|
||||
final Interval interval = this.getImplicitLockInterval().get();
|
||||
final IncrementalIndex empty = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]);
|
||||
final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(interval, empty);
|
||||
|
@ -91,7 +91,7 @@ public class DeleteTask extends AbstractTask
|
|||
.shardSpec(new NoneShardSpec())
|
||||
.build();
|
||||
|
||||
final File outDir = new File(toolbox.getConfig().getTaskDir(this), segment.getIdentifier());
|
||||
final File outDir = new File(toolbox.getTaskDir(), segment.getIdentifier());
|
||||
final File fileToUpload = IndexMerger.merge(Lists.newArrayList(emptyAdapter), new AggregatorFactory[0], outDir);
|
||||
|
||||
// Upload the segment
|
||||
|
@ -104,7 +104,7 @@ public class DeleteTask extends AbstractTask
|
|||
segment.getVersion()
|
||||
);
|
||||
|
||||
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(uploadedSegment)));
|
||||
toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(uploadedSegment)));
|
||||
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
|
|
|
@ -1,3 +1,22 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package com.metamx.druid.merger.common.task;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
|
@ -74,7 +93,7 @@ public class HadoopIndexTask extends AbstractTask
|
|||
);
|
||||
|
||||
// We should have a lock from before we started running
|
||||
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this)));
|
||||
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction()));
|
||||
log.info("Setting version to: %s", myLock.getVersion());
|
||||
configCopy.setVersion(myLock.getVersion());
|
||||
|
||||
|
@ -105,7 +124,7 @@ public class HadoopIndexTask extends AbstractTask
|
|||
List<DataSegment> publishedSegments = job.getPublishedSegments();
|
||||
|
||||
// Request segment pushes
|
||||
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.copyOf(publishedSegments)));
|
||||
toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.copyOf(publishedSegments)));
|
||||
|
||||
// Done
|
||||
return TaskStatus.success(getId());
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -100,24 +100,21 @@ public class IndexGeneratorTask extends AbstractTask
|
|||
public TaskStatus run(final TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
// We should have a lock from before we started running
|
||||
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this)));
|
||||
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction()));
|
||||
|
||||
// We know this exists
|
||||
final Interval interval = getImplicitLockInterval().get();
|
||||
|
||||
// Set up temporary directory for indexing
|
||||
final File tmpDir = new File(
|
||||
toolbox.getTaskDir(),
|
||||
String.format(
|
||||
"%s/%s",
|
||||
toolbox.getConfig().getTaskDir(this).toString(),
|
||||
String.format(
|
||||
"%s_%s_%s_%s_%s",
|
||||
this.getDataSource(),
|
||||
interval.getStart(),
|
||||
interval.getEnd(),
|
||||
myLock.getVersion(),
|
||||
schema.getShardSpec().getPartitionNum()
|
||||
)
|
||||
"%s_%s_%s_%s_%s",
|
||||
this.getDataSource(),
|
||||
interval.getStart(),
|
||||
interval.getEnd(),
|
||||
myLock.getVersion(),
|
||||
schema.getShardSpec().getPartitionNum()
|
||||
)
|
||||
);
|
||||
|
||||
|
@ -193,7 +190,7 @@ public class IndexGeneratorTask extends AbstractTask
|
|||
);
|
||||
|
||||
// Request segment pushes
|
||||
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.copyOf(pushedSegments)));
|
||||
toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.copyOf(pushedSegments)));
|
||||
|
||||
// Done
|
||||
return TaskStatus.success(getId());
|
||||
|
|
|
@ -142,7 +142,7 @@ public class IndexTask extends AbstractTask
|
|||
@Override
|
||||
public TaskStatus preflight(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, toSubtasks()));
|
||||
toolbox.getTaskActionClientFactory().submit(new SpawnTasksAction(toSubtasks()));
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
|
||||
|
|
|
@ -1,3 +1,22 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package com.metamx.druid.merger.common.task;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
|
@ -53,7 +72,7 @@ public class KillTask extends AbstractTask
|
|||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
// Confirm we have a lock (will throw if there isn't exactly one element)
|
||||
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this)));
|
||||
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction()));
|
||||
|
||||
if(!myLock.getDataSource().equals(getDataSource())) {
|
||||
throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource());
|
||||
|
@ -64,14 +83,9 @@ public class KillTask extends AbstractTask
|
|||
}
|
||||
|
||||
// List unused segments
|
||||
final List<DataSegment> unusedSegments = toolbox.getTaskActionClient()
|
||||
.submit(
|
||||
new SegmentListUnusedAction(
|
||||
this,
|
||||
myLock.getDataSource(),
|
||||
myLock.getInterval()
|
||||
)
|
||||
);
|
||||
final List<DataSegment> unusedSegments = toolbox
|
||||
.getTaskActionClientFactory()
|
||||
.submit(new SegmentListUnusedAction(myLock.getDataSource(), myLock.getInterval()));
|
||||
|
||||
// Verify none of these segments have versions > lock version
|
||||
for(final DataSegment unusedSegment : unusedSegments) {
|
||||
|
@ -88,10 +102,12 @@ public class KillTask extends AbstractTask
|
|||
}
|
||||
|
||||
// Kill segments
|
||||
toolbox.getSegmentKiller().kill(unusedSegments);
|
||||
for (DataSegment segment : unusedSegments) {
|
||||
toolbox.getDataSegmentKiller().kill(segment);
|
||||
}
|
||||
|
||||
// Remove metadata for these segments
|
||||
toolbox.getTaskActionClient().submit(new SegmentNukeAction(this, ImmutableSet.copyOf(unusedSegments)));
|
||||
toolbox.getTaskActionClientFactory().submit(new SegmentNukeAction(ImmutableSet.copyOf(unusedSegments)));
|
||||
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
|
|
|
@ -19,318 +19,71 @@
|
|||
|
||||
package com.metamx.druid.merger.common.task;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
import com.google.common.base.Charsets;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Objects;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.hash.Hashing;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.merger.common.TaskLock;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.common.actions.LockListAction;
|
||||
import com.metamx.druid.merger.common.actions.SegmentInsertAction;
|
||||
import com.metamx.druid.merger.common.actions.SegmentListUsedAction;
|
||||
import com.metamx.druid.shard.NoneShardSpec;
|
||||
import com.metamx.emitter.service.AlertEvent;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
import com.metamx.druid.index.QueryableIndex;
|
||||
import com.metamx.druid.index.v1.IndexIO;
|
||||
import com.metamx.druid.index.v1.IndexMerger;
|
||||
|
||||
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
*/
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultMergeTask.class)
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "append", value = AppendTask.class)
|
||||
})
|
||||
public abstract class MergeTask extends AbstractTask
|
||||
public class MergeTask extends MergeTaskBase
|
||||
{
|
||||
private final List<DataSegment> segments;
|
||||
private final List<AggregatorFactory> aggregators;
|
||||
|
||||
private static final Logger log = new Logger(MergeTask.class);
|
||||
|
||||
protected MergeTask(final String dataSource, final List<DataSegment> segments)
|
||||
{
|
||||
super(
|
||||
// _not_ the version, just something uniqueish
|
||||
String.format("merge_%s_%s", computeProcessingID(dataSource, segments), new DateTime().toString()),
|
||||
dataSource,
|
||||
computeMergedInterval(segments)
|
||||
);
|
||||
|
||||
// Verify segment list is nonempty
|
||||
Preconditions.checkArgument(segments.size() > 0, "segments nonempty");
|
||||
// Verify segments are all in the correct datasource
|
||||
Preconditions.checkArgument(
|
||||
Iterables.size(
|
||||
Iterables.filter(
|
||||
segments,
|
||||
new Predicate<DataSegment>()
|
||||
{
|
||||
@Override
|
||||
public boolean apply(@Nullable DataSegment segment)
|
||||
{
|
||||
return segment == null || !segment.getDataSource().equalsIgnoreCase(dataSource);
|
||||
}
|
||||
}
|
||||
)
|
||||
) == 0, "segments in the wrong datasource"
|
||||
);
|
||||
// Verify segments are all unsharded
|
||||
Preconditions.checkArgument(
|
||||
Iterables.size(
|
||||
Iterables.filter(
|
||||
segments,
|
||||
new Predicate<DataSegment>()
|
||||
{
|
||||
@Override
|
||||
public boolean apply(@Nullable DataSegment segment)
|
||||
{
|
||||
return segment == null || !(segment.getShardSpec() instanceof NoneShardSpec);
|
||||
}
|
||||
}
|
||||
)
|
||||
) == 0, "segments without NoneShardSpec"
|
||||
);
|
||||
|
||||
this.segments = segments;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this)));
|
||||
final ServiceEmitter emitter = toolbox.getEmitter();
|
||||
final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder();
|
||||
final DataSegment mergedSegment = computeMergedSegment(getDataSource(), myLock.getVersion(), segments);
|
||||
final File taskDir = toolbox.getConfig().getTaskDir(this);
|
||||
|
||||
try {
|
||||
|
||||
final long startTime = System.currentTimeMillis();
|
||||
|
||||
log.info(
|
||||
"Starting merge of id[%s], segments: %s",
|
||||
getId(),
|
||||
Lists.transform(
|
||||
segments,
|
||||
new Function<DataSegment, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(@Nullable DataSegment input)
|
||||
{
|
||||
return input.getIdentifier();
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
|
||||
|
||||
// download segments to merge
|
||||
final Map<DataSegment, File> gettedSegments = toolbox.getSegments(this, segments);
|
||||
|
||||
// merge files together
|
||||
final File fileToUpload = merge(gettedSegments, new File(taskDir, "merged"));
|
||||
|
||||
emitter.emit(builder.build("merger/numMerged", segments.size()));
|
||||
emitter.emit(builder.build("merger/mergeTime", System.currentTimeMillis() - startTime));
|
||||
|
||||
log.info(
|
||||
"[%s] : Merged %d segments in %,d millis",
|
||||
mergedSegment.getDataSource(),
|
||||
segments.size(),
|
||||
System.currentTimeMillis() - startTime
|
||||
);
|
||||
|
||||
long uploadStart = System.currentTimeMillis();
|
||||
|
||||
// Upload file
|
||||
final DataSegment uploadedSegment = toolbox.getSegmentPusher().push(fileToUpload, mergedSegment);
|
||||
|
||||
emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart));
|
||||
emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize()));
|
||||
|
||||
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(uploadedSegment)));
|
||||
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.error(
|
||||
e,
|
||||
String.format(
|
||||
"Exception merging %s[%s] segments",
|
||||
mergedSegment.getDataSource(),
|
||||
mergedSegment.getInterval()
|
||||
)
|
||||
);
|
||||
emitter.emit(
|
||||
new AlertEvent.Builder().build(
|
||||
"Exception merging",
|
||||
ImmutableMap.<String, Object>builder()
|
||||
.put("exception", e.toString())
|
||||
.build()
|
||||
)
|
||||
);
|
||||
|
||||
return TaskStatus.failure(getId());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks pre-existing segments in "context" to confirm that this merge query is valid. Specifically, confirm that
|
||||
* we are operating on every segment that overlaps the chosen interval.
|
||||
*/
|
||||
@Override
|
||||
public TaskStatus preflight(TaskToolbox toolbox)
|
||||
{
|
||||
final Function<DataSegment, String> toIdentifier = new Function<DataSegment, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(DataSegment dataSegment)
|
||||
{
|
||||
return dataSegment.getIdentifier();
|
||||
}
|
||||
};
|
||||
|
||||
final Set<String> current = ImmutableSet.copyOf(
|
||||
Iterables.transform(
|
||||
toolbox.getTaskActionClient()
|
||||
.submit(new SegmentListUsedAction(this, getDataSource(), getImplicitLockInterval().get())),
|
||||
toIdentifier
|
||||
)
|
||||
);
|
||||
final Set<String> requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier));
|
||||
|
||||
final Set<String> missingFromRequested = Sets.difference(current, requested);
|
||||
if (!missingFromRequested.isEmpty()) {
|
||||
throw new ISE(
|
||||
"Merge is invalid: current segment(s) are not in the requested set: %s",
|
||||
Joiner.on(", ").join(missingFromRequested)
|
||||
);
|
||||
}
|
||||
|
||||
final Set<String> missingFromCurrent = Sets.difference(requested, current);
|
||||
if (!missingFromCurrent.isEmpty()) {
|
||||
throw new ISE(
|
||||
"Merge is invalid: requested segment(s) are not in the current set: %s",
|
||||
Joiner.on(", ").join(missingFromCurrent)
|
||||
);
|
||||
}
|
||||
|
||||
return TaskStatus.running(getId());
|
||||
|
||||
}
|
||||
|
||||
protected abstract File merge(Map<DataSegment, File> segments, File outDir)
|
||||
throws Exception;
|
||||
|
||||
@JsonProperty
|
||||
public List<DataSegment> getSegments()
|
||||
{
|
||||
return segments;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return Objects.toStringHelper(this)
|
||||
.add("id", getId())
|
||||
.add("dataSource", getDataSource())
|
||||
.add("interval", getImplicitLockInterval())
|
||||
.add("segments", segments)
|
||||
.toString();
|
||||
}
|
||||
|
||||
private static String computeProcessingID(final String dataSource, final List<DataSegment> segments)
|
||||
{
|
||||
final String segmentIDs = Joiner.on("_").join(
|
||||
Iterables.transform(
|
||||
Ordering.natural().sortedCopy(segments), new Function<DataSegment, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(DataSegment x)
|
||||
{
|
||||
return String.format(
|
||||
"%s_%s_%s_%s",
|
||||
x.getInterval().getStart(),
|
||||
x.getInterval().getEnd(),
|
||||
x.getVersion(),
|
||||
x.getShardSpec().getPartitionNum()
|
||||
);
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
|
||||
return String.format(
|
||||
"%s_%s",
|
||||
dataSource,
|
||||
Hashing.sha1().hashString(segmentIDs, Charsets.UTF_8).toString().toLowerCase()
|
||||
);
|
||||
}
|
||||
|
||||
private static Interval computeMergedInterval(final List<DataSegment> segments)
|
||||
{
|
||||
Preconditions.checkArgument(segments.size() > 0, "segments.size() > 0");
|
||||
|
||||
DateTime start = null;
|
||||
DateTime end = null;
|
||||
|
||||
for(final DataSegment segment : segments) {
|
||||
if(start == null || segment.getInterval().getStart().isBefore(start)) {
|
||||
start = segment.getInterval().getStart();
|
||||
}
|
||||
|
||||
if(end == null || segment.getInterval().getEnd().isAfter(end)) {
|
||||
end = segment.getInterval().getEnd();
|
||||
}
|
||||
}
|
||||
|
||||
return new Interval(start, end);
|
||||
}
|
||||
|
||||
private static DataSegment computeMergedSegment(
|
||||
final String dataSource,
|
||||
final String version,
|
||||
final List<DataSegment> segments
|
||||
@JsonCreator
|
||||
public MergeTask(
|
||||
@JsonProperty("dataSource") String dataSource,
|
||||
@JsonProperty("segments") List<DataSegment> segments,
|
||||
@JsonProperty("aggregations") List<AggregatorFactory> aggregators
|
||||
)
|
||||
{
|
||||
final Interval mergedInterval = computeMergedInterval(segments);
|
||||
final Set<String> mergedDimensions = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER);
|
||||
final Set<String> mergedMetrics = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER);
|
||||
super(dataSource, segments);
|
||||
this.aggregators = aggregators;
|
||||
}
|
||||
|
||||
for (DataSegment segment : segments) {
|
||||
mergedDimensions.addAll(segment.getDimensions());
|
||||
mergedMetrics.addAll(segment.getMetrics());
|
||||
}
|
||||
@Override
|
||||
public File merge(final Map<DataSegment, File> segments, final File outDir)
|
||||
throws Exception
|
||||
{
|
||||
return IndexMerger.mergeQueryableIndex(
|
||||
Lists.transform(
|
||||
ImmutableList.copyOf(segments.values()),
|
||||
new Function<File, QueryableIndex>()
|
||||
{
|
||||
@Override
|
||||
public QueryableIndex apply(@Nullable File input)
|
||||
{
|
||||
try {
|
||||
return IndexIO.loadIndex(input);
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
),
|
||||
aggregators.toArray(new AggregatorFactory[aggregators.size()]),
|
||||
outDir
|
||||
);
|
||||
}
|
||||
|
||||
return DataSegment.builder()
|
||||
.dataSource(dataSource)
|
||||
.interval(mergedInterval)
|
||||
.version(version)
|
||||
.shardSpec(new NoneShardSpec())
|
||||
.dimensions(Lists.newArrayList(mergedDimensions))
|
||||
.metrics(Lists.newArrayList(mergedMetrics))
|
||||
.build();
|
||||
@Override
|
||||
public String getType()
|
||||
{
|
||||
return "merge";
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -41,16 +41,18 @@ import org.joda.time.Interval;
|
|||
* to release locks early if they desire.</li>
|
||||
* </ul>
|
||||
*/
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultMergeTask.class)
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "append", value = AppendTask.class),
|
||||
@JsonSubTypes.Type(name = "merge", value = DefaultMergeTask.class),
|
||||
@JsonSubTypes.Type(name = "merge", value = MergeTask.class),
|
||||
@JsonSubTypes.Type(name = "delete", value = DeleteTask.class),
|
||||
@JsonSubTypes.Type(name = "kill", value = KillTask.class),
|
||||
@JsonSubTypes.Type(name = "index", value = IndexTask.class),
|
||||
@JsonSubTypes.Type(name = "index_partitions", value = IndexDeterminePartitionsTask.class),
|
||||
@JsonSubTypes.Type(name = "index_generator", value = IndexGeneratorTask.class),
|
||||
@JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class)
|
||||
@JsonSubTypes.Type(name = "index_hadoop", value = HadoopIndexTask.class),
|
||||
@JsonSubTypes.Type(name = "version_converter", value = VersionConverterTask.class),
|
||||
@JsonSubTypes.Type(name = "version_converter_sub", value = VersionConverterTask.SubTask.class)
|
||||
})
|
||||
public interface Task
|
||||
{
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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.");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -327,11 +327,11 @@ public class DbTaskStorage implements TaskStorage
|
|||
}
|
||||
|
||||
@Override
|
||||
public <T> void addAuditLog(final TaskAction<T> taskAction)
|
||||
public <T> void addAuditLog(final Task task, final TaskAction<T> taskAction)
|
||||
{
|
||||
Preconditions.checkNotNull(taskAction, "taskAction");
|
||||
|
||||
log.info("Logging action for task[%s]: %s", taskAction.getTask().getId(), taskAction);
|
||||
log.info("Logging action for task[%s]: %s", task.getId(), taskAction);
|
||||
|
||||
dbi.withHandle(
|
||||
new HandleCallback<Integer>()
|
||||
|
@ -345,7 +345,7 @@ public class DbTaskStorage implements TaskStorage
|
|||
dbConnectorConfig.getTaskLogTable()
|
||||
)
|
||||
)
|
||||
.bind("task_id", taskAction.getTask().getId())
|
||||
.bind("task_id", task.getId())
|
||||
.bind("log_payload", jsonMapper.writeValueAsString(taskAction))
|
||||
.execute();
|
||||
}
|
||||
|
|
|
@ -40,14 +40,14 @@ import java.util.concurrent.locks.ReentrantLock;
|
|||
* Implements an in-heap TaskStorage facility, with no persistence across restarts. This class is not
|
||||
* thread safe.
|
||||
*/
|
||||
public class LocalTaskStorage implements TaskStorage
|
||||
public class HeapMemoryTaskStorage implements TaskStorage
|
||||
{
|
||||
private final ReentrantLock giant = new ReentrantLock();
|
||||
private final Map<String, TaskStuff> tasks = Maps.newHashMap();
|
||||
private final Multimap<String, TaskLock> taskLocks = HashMultimap.create();
|
||||
private final Multimap<String, TaskAction> taskActions = ArrayListMultimap.create();
|
||||
|
||||
private static final Logger log = new Logger(LocalTaskStorage.class);
|
||||
private static final Logger log = new Logger(HeapMemoryTaskStorage.class);
|
||||
|
||||
@Override
|
||||
public void insert(Task task, TaskStatus status)
|
||||
|
@ -185,12 +185,12 @@ public class LocalTaskStorage implements TaskStorage
|
|||
}
|
||||
|
||||
@Override
|
||||
public <T> void addAuditLog(TaskAction<T> taskAction)
|
||||
public <T> void addAuditLog(Task task, TaskAction<T> taskAction)
|
||||
{
|
||||
giant.lock();
|
||||
|
||||
try {
|
||||
taskActions.put(taskAction.getTask().getId(), taskAction);
|
||||
taskActions.put(task.getId(), taskAction);
|
||||
} finally {
|
||||
giant.unlock();
|
||||
}
|
|
@ -26,6 +26,7 @@ import com.metamx.common.logger.Logger;
|
|||
import com.metamx.druid.merger.common.TaskCallback;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.common.TaskToolboxFactory;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
|
||||
|
@ -38,17 +39,17 @@ import java.util.concurrent.ExecutorService;
|
|||
*/
|
||||
public class LocalTaskRunner implements TaskRunner
|
||||
{
|
||||
private final TaskToolbox toolbox;
|
||||
private final TaskToolboxFactory toolboxFactory;
|
||||
private final ExecutorService exec;
|
||||
|
||||
private static final Logger log = new Logger(LocalTaskRunner.class);
|
||||
|
||||
public LocalTaskRunner(
|
||||
TaskToolbox toolbox,
|
||||
TaskToolboxFactory toolboxFactory,
|
||||
ExecutorService exec
|
||||
)
|
||||
{
|
||||
this.toolbox = toolbox;
|
||||
this.toolboxFactory = toolboxFactory;
|
||||
this.exec = exec;
|
||||
}
|
||||
|
||||
|
@ -61,6 +62,8 @@ public class LocalTaskRunner implements TaskRunner
|
|||
@Override
|
||||
public void run(final Task task, final TaskCallback callback)
|
||||
{
|
||||
final TaskToolbox toolbox = toolboxFactory.build(task);
|
||||
|
||||
exec.submit(
|
||||
new Runnable()
|
||||
{
|
||||
|
@ -89,7 +92,7 @@ public class LocalTaskRunner implements TaskRunner
|
|||
}
|
||||
|
||||
try {
|
||||
final File taskDir = toolbox.getConfig().getTaskDir(task);
|
||||
final File taskDir = toolbox.getTaskDir();
|
||||
|
||||
if (taskDir.exists()) {
|
||||
log.info("Removing task directory: %s", taskDir);
|
||||
|
|
|
@ -35,7 +35,7 @@ import com.metamx.druid.merger.common.TaskCallback;
|
|||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig;
|
||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
|
||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
|
||||
import com.metamx.druid.merger.worker.Worker;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import com.netflix.curator.framework.CuratorFramework;
|
||||
|
@ -57,6 +57,7 @@ import java.util.concurrent.ExecutorService;
|
|||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
/**
|
||||
* The RemoteTaskRunner's primary responsibility is to assign tasks to worker nodes and manage retries in failure
|
||||
|
@ -82,7 +83,7 @@ public class RemoteTaskRunner implements TaskRunner
|
|||
private final PathChildrenCache workerPathCache;
|
||||
private final ScheduledExecutorService scheduledExec;
|
||||
private final RetryPolicyFactory retryPolicyFactory;
|
||||
private final WorkerSetupManager workerSetupManager;
|
||||
private final AtomicReference<WorkerSetupData> workerSetupData;
|
||||
|
||||
// all workers that exist in ZK
|
||||
private final Map<String, ZkWorker> zkWorkers = new ConcurrentHashMap<String, ZkWorker>();
|
||||
|
@ -104,7 +105,7 @@ public class RemoteTaskRunner implements TaskRunner
|
|||
PathChildrenCache workerPathCache,
|
||||
ScheduledExecutorService scheduledExec,
|
||||
RetryPolicyFactory retryPolicyFactory,
|
||||
WorkerSetupManager workerSetupManager
|
||||
AtomicReference<WorkerSetupData> workerSetupData
|
||||
)
|
||||
{
|
||||
this.jsonMapper = jsonMapper;
|
||||
|
@ -113,7 +114,7 @@ public class RemoteTaskRunner implements TaskRunner
|
|||
this.workerPathCache = workerPathCache;
|
||||
this.scheduledExec = scheduledExec;
|
||||
this.retryPolicyFactory = retryPolicyFactory;
|
||||
this.workerSetupManager = workerSetupManager;
|
||||
this.workerSetupData = workerSetupData;
|
||||
}
|
||||
|
||||
@LifecycleStart
|
||||
|
@ -548,7 +549,7 @@ public class RemoteTaskRunner implements TaskRunner
|
|||
return (!input.isAtCapacity() &&
|
||||
input.getWorker()
|
||||
.getVersion()
|
||||
.compareTo(workerSetupManager.getWorkerSetupData().getMinVersion()) >= 0);
|
||||
.compareTo(workerSetupData.get().getMinVersion()) >= 0);
|
||||
}
|
||||
}
|
||||
)
|
||||
|
|
|
@ -1,3 +1,22 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package com.metamx.druid.merger.coordinator;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
|
|
|
@ -26,6 +26,8 @@ import com.metamx.common.lifecycle.LifecycleStop;
|
|||
import com.metamx.druid.initialization.Initialization;
|
||||
import com.metamx.druid.initialization.ServiceDiscoveryConfig;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.common.TaskToolboxFactory;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
|
||||
import com.metamx.druid.merger.coordinator.exec.TaskConsumer;
|
||||
import com.metamx.druid.merger.coordinator.scaling.ResourceManagementScheduler;
|
||||
|
@ -49,7 +51,7 @@ public class TaskMasterLifecycle
|
|||
private final ReentrantLock giant = new ReentrantLock();
|
||||
private final Condition mayBeStopped = giant.newCondition();
|
||||
private final TaskQueue taskQueue;
|
||||
private final TaskToolbox taskToolbox;
|
||||
private final TaskToolboxFactory taskToolboxFactory;
|
||||
|
||||
private volatile boolean leading = false;
|
||||
private volatile TaskRunner taskRunner;
|
||||
|
@ -59,7 +61,7 @@ public class TaskMasterLifecycle
|
|||
|
||||
public TaskMasterLifecycle(
|
||||
final TaskQueue taskQueue,
|
||||
final TaskToolbox taskToolbox,
|
||||
final TaskToolboxFactory taskToolboxFactory,
|
||||
final IndexerCoordinatorConfig indexerCoordinatorConfig,
|
||||
final ServiceDiscoveryConfig serviceDiscoveryConfig,
|
||||
final TaskRunnerFactory runnerFactory,
|
||||
|
@ -69,7 +71,7 @@ public class TaskMasterLifecycle
|
|||
)
|
||||
{
|
||||
this.taskQueue = taskQueue;
|
||||
this.taskToolbox = taskToolbox;
|
||||
this.taskToolboxFactory = taskToolboxFactory;
|
||||
|
||||
this.leaderSelector = new LeaderSelector(
|
||||
curator, indexerCoordinatorConfig.getLeaderLatchPath(), new LeaderSelectorListener()
|
||||
|
@ -87,7 +89,7 @@ public class TaskMasterLifecycle
|
|||
final TaskConsumer taskConsumer = new TaskConsumer(
|
||||
taskQueue,
|
||||
taskRunner,
|
||||
taskToolbox,
|
||||
taskToolboxFactory,
|
||||
emitter
|
||||
);
|
||||
|
||||
|
@ -221,9 +223,4 @@ public class TaskMasterLifecycle
|
|||
{
|
||||
return taskToolbox;
|
||||
}
|
||||
|
||||
public ResourceManagementScheduler getResourceManagementScheduler()
|
||||
{
|
||||
return resourceManagementScheduler;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -69,7 +69,7 @@ public interface TaskStorage
|
|||
/**
|
||||
* Add an action taken by a task to the audit log.
|
||||
*/
|
||||
public <T> void addAuditLog(TaskAction<T> taskAction);
|
||||
public <T> void addAuditLog(Task task, TaskAction<T> taskAction);
|
||||
|
||||
/**
|
||||
* Returns all actions taken by a task.
|
||||
|
|
|
@ -24,7 +24,7 @@ import com.metamx.common.lifecycle.LifecycleStart;
|
|||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import com.metamx.druid.merger.common.TaskCallback;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.common.TaskToolboxFactory;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.merger.coordinator.TaskQueue;
|
||||
import com.metamx.druid.merger.coordinator.TaskRunner;
|
||||
|
@ -36,7 +36,7 @@ public class TaskConsumer implements Runnable
|
|||
{
|
||||
private final TaskQueue queue;
|
||||
private final TaskRunner runner;
|
||||
private final TaskToolbox toolbox;
|
||||
private final TaskToolboxFactory toolboxFactory;
|
||||
private final ServiceEmitter emitter;
|
||||
private final Thread thready;
|
||||
|
||||
|
@ -47,13 +47,13 @@ public class TaskConsumer implements Runnable
|
|||
public TaskConsumer(
|
||||
TaskQueue queue,
|
||||
TaskRunner runner,
|
||||
TaskToolbox toolbox,
|
||||
TaskToolboxFactory toolboxFactory,
|
||||
ServiceEmitter emitter
|
||||
)
|
||||
{
|
||||
this.queue = queue;
|
||||
this.runner = runner;
|
||||
this.toolbox = toolbox;
|
||||
this.toolboxFactory = toolboxFactory;
|
||||
this.emitter = emitter;
|
||||
this.thready = new Thread(this);
|
||||
}
|
||||
|
@ -123,7 +123,7 @@ public class TaskConsumer implements Runnable
|
|||
// Run preflight checks
|
||||
TaskStatus preflightStatus;
|
||||
try {
|
||||
preflightStatus = task.preflight(toolbox);
|
||||
preflightStatus = task.preflight(toolboxFactory.build(task));
|
||||
log.info("Preflight done for task: %s", task.getId());
|
||||
}
|
||||
catch (Exception e) {
|
||||
|
|
|
@ -39,6 +39,9 @@ import com.metamx.common.lifecycle.LifecycleStart;
|
|||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.RegisteringNode;
|
||||
import com.metamx.druid.config.ConfigManager;
|
||||
import com.metamx.druid.config.ConfigManagerConfig;
|
||||
import com.metamx.druid.config.JacksonConfigManager;
|
||||
import com.metamx.druid.db.DbConnector;
|
||||
import com.metamx.druid.db.DbConnectorConfig;
|
||||
import com.metamx.druid.http.GuiceServletConfig;
|
||||
|
@ -48,22 +51,21 @@ import com.metamx.druid.http.RedirectInfo;
|
|||
import com.metamx.druid.http.StatusServlet;
|
||||
import com.metamx.druid.initialization.Initialization;
|
||||
import com.metamx.druid.initialization.ServerConfig;
|
||||
import com.metamx.druid.initialization.ServerInit;
|
||||
import com.metamx.druid.initialization.ServiceDiscoveryConfig;
|
||||
import com.metamx.druid.jackson.DefaultObjectMapper;
|
||||
import com.metamx.druid.loading.DataSegmentKiller;
|
||||
import com.metamx.druid.loading.DataSegmentPusher;
|
||||
import com.metamx.druid.loading.S3DataSegmentPusher;
|
||||
import com.metamx.druid.loading.S3DataSegmentPusherConfig;
|
||||
import com.metamx.druid.loading.S3SegmentKiller;
|
||||
import com.metamx.druid.loading.SegmentKiller;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.common.actions.LocalTaskActionClient;
|
||||
import com.metamx.druid.loading.S3DataSegmentKiller;
|
||||
import com.metamx.druid.merger.common.TaskToolboxFactory;
|
||||
import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory;
|
||||
import com.metamx.druid.merger.common.actions.TaskActionToolbox;
|
||||
import com.metamx.druid.merger.common.config.IndexerZkConfig;
|
||||
import com.metamx.druid.merger.common.config.TaskConfig;
|
||||
import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory;
|
||||
import com.metamx.druid.merger.coordinator.DbTaskStorage;
|
||||
import com.metamx.druid.merger.coordinator.HeapMemoryTaskStorage;
|
||||
import com.metamx.druid.merger.coordinator.LocalTaskRunner;
|
||||
import com.metamx.druid.merger.coordinator.LocalTaskStorage;
|
||||
import com.metamx.druid.merger.coordinator.MergerDBCoordinator;
|
||||
import com.metamx.druid.merger.coordinator.RemoteTaskRunner;
|
||||
import com.metamx.druid.merger.coordinator.RetryPolicyFactory;
|
||||
|
@ -79,7 +81,6 @@ import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
|
|||
import com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig;
|
||||
import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig;
|
||||
import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig;
|
||||
import com.metamx.druid.merger.coordinator.config.WorkerSetupManagerConfig;
|
||||
import com.metamx.druid.merger.coordinator.scaling.AutoScalingStrategy;
|
||||
import com.metamx.druid.merger.coordinator.scaling.EC2AutoScalingStrategy;
|
||||
import com.metamx.druid.merger.coordinator.scaling.NoopAutoScalingStrategy;
|
||||
|
@ -88,7 +89,7 @@ import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerCo
|
|||
import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerFactory;
|
||||
import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagementStrategy;
|
||||
import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagmentConfig;
|
||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
|
||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
|
||||
import com.metamx.druid.utils.PropUtils;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import com.metamx.emitter.core.Emitters;
|
||||
|
@ -123,6 +124,7 @@ import java.util.Properties;
|
|||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -147,7 +149,8 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
private RestS3Service s3Service = null;
|
||||
private IndexerCoordinatorConfig config = null;
|
||||
private TaskConfig taskConfig = null;
|
||||
private TaskToolbox taskToolbox = null;
|
||||
private DataSegmentPusher segmentPusher = null;
|
||||
private TaskToolboxFactory taskToolboxFactory = null;
|
||||
private MergerDBCoordinator mergerDBCoordinator = null;
|
||||
private TaskStorage taskStorage = null;
|
||||
private TaskQueue taskQueue = null;
|
||||
|
@ -155,7 +158,6 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
private CuratorFramework curatorFramework = null;
|
||||
private ScheduledExecutorFactory scheduledExecutorFactory = null;
|
||||
private IndexerZkConfig indexerZkConfig;
|
||||
private WorkerSetupManager workerSetupManager = null;
|
||||
private TaskRunnerFactory taskRunnerFactory = null;
|
||||
private ResourceManagementSchedulerFactory resourceManagementSchedulerFactory = null;
|
||||
private TaskMasterLifecycle taskMasterLifecycle = null;
|
||||
|
@ -208,6 +210,12 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
return this;
|
||||
}
|
||||
|
||||
public IndexerCoordinatorNode setSegmentPusher(DataSegmentPusher segmentPusher)
|
||||
{
|
||||
this.segmentPusher = segmentPusher;
|
||||
return this;
|
||||
}
|
||||
|
||||
public IndexerCoordinatorNode setMergeDbCoordinator(MergerDBCoordinator mergeDbCoordinator)
|
||||
{
|
||||
this.mergerDBCoordinator = mergeDbCoordinator;
|
||||
|
@ -220,12 +228,6 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
return this;
|
||||
}
|
||||
|
||||
public IndexerCoordinatorNode setWorkerSetupManager(WorkerSetupManager workerSetupManager)
|
||||
{
|
||||
this.workerSetupManager = workerSetupManager;
|
||||
return this;
|
||||
}
|
||||
|
||||
public IndexerCoordinatorNode setTaskRunnerFactory(TaskRunnerFactory taskRunnerFactory)
|
||||
{
|
||||
this.taskRunnerFactory = taskRunnerFactory;
|
||||
|
@ -242,6 +244,10 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
{
|
||||
scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle);
|
||||
|
||||
final ConfigManagerConfig managerConfig = configFactory.build(ConfigManagerConfig.class);
|
||||
DbConnector.createConfigTable(dbi, managerConfig.getConfigTable());
|
||||
JacksonConfigManager configManager = new JacksonConfigManager(new ConfigManager(dbi, managerConfig), jsonMapper);
|
||||
|
||||
initializeEmitter();
|
||||
initializeMonitors();
|
||||
initializeDB();
|
||||
|
@ -252,14 +258,14 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
initializeTaskStorage();
|
||||
initializeTaskLockbox();
|
||||
initializeTaskQueue();
|
||||
initializeDataSegmentPusher();
|
||||
initializeTaskToolbox();
|
||||
initializeJacksonInjections();
|
||||
initializeJacksonSubtypes();
|
||||
initializeCurator();
|
||||
initializeIndexerZkConfig();
|
||||
initializeWorkerSetupManager();
|
||||
initializeTaskRunnerFactory();
|
||||
initializeResourceManagement();
|
||||
initializeTaskRunnerFactory(configManager);
|
||||
initializeResourceManagement(configManager);
|
||||
initializeTaskMasterLifecycle();
|
||||
initializeServer();
|
||||
|
||||
|
@ -279,7 +285,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
emitter,
|
||||
taskMasterLifecycle,
|
||||
new TaskStorageQueryAdapter(taskStorage),
|
||||
workerSetupManager
|
||||
configManager
|
||||
)
|
||||
);
|
||||
|
||||
|
@ -339,7 +345,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class);
|
||||
taskMasterLifecycle = new TaskMasterLifecycle(
|
||||
taskQueue,
|
||||
taskToolbox,
|
||||
taskToolboxFactory,
|
||||
config,
|
||||
serviceDiscoveryConfig,
|
||||
taskRunnerFactory,
|
||||
|
@ -403,7 +409,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
InjectableValues.Std injectables = new InjectableValues.Std();
|
||||
|
||||
injectables.addValue("s3Client", s3Service)
|
||||
.addValue("segmentPusher", taskToolbox.getSegmentPusher());
|
||||
.addValue("segmentPusher", segmentPusher);
|
||||
|
||||
jsonMapper.setInjectableValues(injectables);
|
||||
}
|
||||
|
@ -472,27 +478,27 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
);
|
||||
}
|
||||
|
||||
public void initializeDataSegmentPusher()
|
||||
{
|
||||
if (segmentPusher == null) {
|
||||
segmentPusher = ServerInit.getSegmentPusher(props, configFactory, jsonMapper);
|
||||
}
|
||||
}
|
||||
|
||||
public void initializeTaskToolbox()
|
||||
{
|
||||
if (taskToolbox == null) {
|
||||
final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher(
|
||||
s3Service,
|
||||
configFactory.build(S3DataSegmentPusherConfig.class),
|
||||
jsonMapper
|
||||
);
|
||||
final SegmentKiller segmentKiller = new S3SegmentKiller(
|
||||
s3Service
|
||||
);
|
||||
taskToolbox = new TaskToolbox(
|
||||
if (taskToolboxFactory == null) {
|
||||
final DataSegmentKiller dataSegmentKiller = new S3DataSegmentKiller(s3Service);
|
||||
taskToolboxFactory = new TaskToolboxFactory(
|
||||
taskConfig,
|
||||
new LocalTaskActionClient(
|
||||
new LocalTaskActionClientFactory(
|
||||
taskStorage,
|
||||
new TaskActionToolbox(taskQueue, taskLockbox, mergerDBCoordinator, emitter)
|
||||
),
|
||||
emitter,
|
||||
s3Service,
|
||||
dataSegmentPusher,
|
||||
segmentKiller,
|
||||
segmentPusher,
|
||||
dataSegmentKiller,
|
||||
jsonMapper
|
||||
);
|
||||
}
|
||||
|
@ -546,7 +552,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
{
|
||||
if (taskStorage == null) {
|
||||
if (config.getStorageImpl().equals("local")) {
|
||||
taskStorage = new LocalTaskStorage();
|
||||
taskStorage = new HeapMemoryTaskStorage();
|
||||
} else if (config.getStorageImpl().equals("db")) {
|
||||
final IndexerDbConnectorConfig dbConnectorConfig = configFactory.build(IndexerDbConnectorConfig.class);
|
||||
taskStorage = new DbTaskStorage(jsonMapper, dbConnectorConfig, new DbConnector(dbConnectorConfig).getDBI());
|
||||
|
@ -556,26 +562,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
}
|
||||
}
|
||||
|
||||
public void initializeWorkerSetupManager()
|
||||
{
|
||||
if (workerSetupManager == null) {
|
||||
final WorkerSetupManagerConfig workerSetupManagerConfig = configFactory.build(WorkerSetupManagerConfig.class);
|
||||
|
||||
DbConnector.createConfigTable(dbi, workerSetupManagerConfig.getConfigTable());
|
||||
workerSetupManager = new WorkerSetupManager(
|
||||
dbi, Executors.newScheduledThreadPool(
|
||||
1,
|
||||
new ThreadFactoryBuilder()
|
||||
.setDaemon(true)
|
||||
.setNameFormat("WorkerSetupManagerExec--%d")
|
||||
.build()
|
||||
), jsonMapper, workerSetupManagerConfig
|
||||
);
|
||||
}
|
||||
lifecycle.addManagedInstance(workerSetupManager);
|
||||
}
|
||||
|
||||
public void initializeTaskRunnerFactory()
|
||||
private void initializeTaskRunnerFactory(final JacksonConfigManager configManager)
|
||||
{
|
||||
if (taskRunnerFactory == null) {
|
||||
if (config.getRunnerImpl().equals("remote")) {
|
||||
|
@ -601,7 +588,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
new PathChildrenCache(curatorFramework, indexerZkConfig.getAnnouncementPath(), true),
|
||||
retryScheduledExec,
|
||||
new RetryPolicyFactory(configFactory.build(RetryPolicyConfig.class)),
|
||||
workerSetupManager
|
||||
configManager.watch(WorkerSetupData.CONFIG_KEY, WorkerSetupData.class)
|
||||
);
|
||||
|
||||
return remoteTaskRunner;
|
||||
|
@ -615,7 +602,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
public TaskRunner build()
|
||||
{
|
||||
final ExecutorService runnerExec = Executors.newFixedThreadPool(config.getNumLocalThreads());
|
||||
return new LocalTaskRunner(taskToolbox, runnerExec);
|
||||
return new LocalTaskRunner(taskToolboxFactory, runnerExec);
|
||||
}
|
||||
};
|
||||
} else {
|
||||
|
@ -624,7 +611,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
}
|
||||
}
|
||||
|
||||
private void initializeResourceManagement()
|
||||
private void initializeResourceManagement(final JacksonConfigManager configManager)
|
||||
{
|
||||
if (resourceManagementSchedulerFactory == null) {
|
||||
resourceManagementSchedulerFactory = new ResourceManagementSchedulerFactory()
|
||||
|
@ -639,6 +626,9 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
.setNameFormat("ScalingExec--%d")
|
||||
.build()
|
||||
);
|
||||
final AtomicReference<WorkerSetupData> workerSetupData = configManager.watch(
|
||||
WorkerSetupData.CONFIG_KEY, WorkerSetupData.class
|
||||
);
|
||||
|
||||
AutoScalingStrategy strategy;
|
||||
if (config.getStrategyImpl().equalsIgnoreCase("ec2")) {
|
||||
|
@ -651,7 +641,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
)
|
||||
),
|
||||
configFactory.build(EC2AutoScalingStrategyConfig.class),
|
||||
workerSetupManager
|
||||
workerSetupData
|
||||
);
|
||||
} else if (config.getStrategyImpl().equalsIgnoreCase("noop")) {
|
||||
strategy = new NoopAutoScalingStrategy();
|
||||
|
@ -664,7 +654,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
|
|||
new SimpleResourceManagementStrategy(
|
||||
strategy,
|
||||
configFactory.build(SimpleResourceManagmentConfig.class),
|
||||
workerSetupManager
|
||||
workerSetupData
|
||||
),
|
||||
configFactory.build(ResourceManagementSchedulerConfig.class),
|
||||
scalingScheduledExec
|
||||
|
|
|
@ -19,6 +19,8 @@
|
|||
|
||||
package com.metamx.druid.merger.coordinator.http;
|
||||
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
|
@ -26,8 +28,9 @@ import com.google.common.collect.Maps;
|
|||
import com.google.inject.Inject;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.config.JacksonConfigManager;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.actions.TaskAction;
|
||||
import com.metamx.druid.merger.common.actions.TaskActionHolder;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.druid.merger.coordinator.TaskMasterLifecycle;
|
||||
import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter;
|
||||
|
@ -35,10 +38,7 @@ import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
|
|||
import com.metamx.druid.merger.coordinator.scaling.AutoScalingData;
|
||||
import com.metamx.druid.merger.coordinator.scaling.ScalingStats;
|
||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
|
||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
|
||||
import javax.ws.rs.Consumes;
|
||||
import javax.ws.rs.GET;
|
||||
|
@ -49,6 +49,7 @@ import javax.ws.rs.Produces;
|
|||
import javax.ws.rs.core.Response;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -61,16 +62,18 @@ public class IndexerCoordinatorResource
|
|||
private final ServiceEmitter emitter;
|
||||
private final TaskMasterLifecycle taskMasterLifecycle;
|
||||
private final TaskStorageQueryAdapter taskStorageQueryAdapter;
|
||||
private final WorkerSetupManager workerSetupManager;
|
||||
private final JacksonConfigManager configManager;
|
||||
private final ObjectMapper jsonMapper;
|
||||
|
||||
private AtomicReference<WorkerSetupData> workerSetupDataRef = null;
|
||||
|
||||
@Inject
|
||||
public IndexerCoordinatorResource(
|
||||
IndexerCoordinatorConfig config,
|
||||
ServiceEmitter emitter,
|
||||
TaskMasterLifecycle taskMasterLifecycle,
|
||||
TaskStorageQueryAdapter taskStorageQueryAdapter,
|
||||
WorkerSetupManager workerSetupManager,
|
||||
JacksonConfigManager configManager,
|
||||
ObjectMapper jsonMapper
|
||||
) throws Exception
|
||||
{
|
||||
|
@ -78,7 +81,7 @@ public class IndexerCoordinatorResource
|
|||
this.emitter = emitter;
|
||||
this.taskMasterLifecycle = taskMasterLifecycle;
|
||||
this.taskStorageQueryAdapter = taskStorageQueryAdapter;
|
||||
this.workerSetupManager = workerSetupManager;
|
||||
this.configManager = configManager;
|
||||
this.jsonMapper = jsonMapper;
|
||||
}
|
||||
|
||||
|
@ -98,18 +101,15 @@ public class IndexerCoordinatorResource
|
|||
@Produces("application/json")
|
||||
public Response doIndex(final Task task)
|
||||
{
|
||||
// verify against whitelist
|
||||
if (config.isWhitelistEnabled() && !config.getWhitelistDatasources().contains(task.getDataSource())) {
|
||||
return Response.status(Response.Status.BAD_REQUEST)
|
||||
.entity(
|
||||
ImmutableMap.of(
|
||||
"error",
|
||||
String.format("dataSource[%s] is not whitelisted", task.getDataSource())
|
||||
)
|
||||
)
|
||||
.build();
|
||||
}
|
||||
return taskPost(task);
|
||||
}
|
||||
|
||||
@POST
|
||||
@Path("/task")
|
||||
@Consumes("application/json")
|
||||
@Produces("application/json")
|
||||
public Response taskPost(final Task task)
|
||||
{
|
||||
taskMasterLifecycle.getTaskQueue().add(task);
|
||||
return Response.ok(ImmutableMap.of("task", task.getId())).build();
|
||||
}
|
||||
|
@ -162,7 +162,11 @@ public class IndexerCoordinatorResource
|
|||
@Produces("application/json")
|
||||
public Response getWorkerSetupData()
|
||||
{
|
||||
return Response.ok(workerSetupManager.getWorkerSetupData()).build();
|
||||
if (workerSetupDataRef == null) {
|
||||
workerSetupDataRef = configManager.watch(WorkerSetupData.CONFIG_KEY, WorkerSetupData.class);
|
||||
}
|
||||
|
||||
return Response.ok(workerSetupDataRef.get()).build();
|
||||
}
|
||||
|
||||
@POST
|
||||
|
@ -172,7 +176,7 @@ public class IndexerCoordinatorResource
|
|||
final WorkerSetupData workerSetupData
|
||||
)
|
||||
{
|
||||
if (!workerSetupManager.setWorkerSetupData(workerSetupData)) {
|
||||
if (!configManager.set(WorkerSetupData.CONFIG_KEY, workerSetupData)) {
|
||||
return Response.status(Response.Status.BAD_REQUEST).build();
|
||||
}
|
||||
return Response.ok().build();
|
||||
|
@ -181,9 +185,12 @@ public class IndexerCoordinatorResource
|
|||
@POST
|
||||
@Path("/action")
|
||||
@Produces("application/json")
|
||||
public <T> Response doAction(final TaskAction<T> action)
|
||||
public <T> Response doAction(final TaskActionHolder<T> holder)
|
||||
{
|
||||
final T ret = taskMasterLifecycle.getTaskToolbox().getTaskActionClient().submit(action);
|
||||
final T ret = taskMasterLifecycle.getTaskToolbox(holder.getTask())
|
||||
.getTaskActionClientFactory()
|
||||
.submit(holder.getAction());
|
||||
|
||||
final Map<String, Object> retMap = Maps.newHashMap();
|
||||
retMap.put("result", ret);
|
||||
|
||||
|
|
|
@ -22,10 +22,10 @@ package com.metamx.druid.merger.coordinator.http;
|
|||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider;
|
||||
import com.google.inject.Provides;
|
||||
import com.metamx.druid.config.JacksonConfigManager;
|
||||
import com.metamx.druid.merger.coordinator.TaskMasterLifecycle;
|
||||
import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter;
|
||||
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
|
||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.sun.jersey.guice.JerseyServletModule;
|
||||
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
|
||||
|
@ -41,7 +41,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
|
|||
private final ServiceEmitter emitter;
|
||||
private final TaskMasterLifecycle taskMasterLifecycle;
|
||||
private final TaskStorageQueryAdapter taskStorageQueryAdapter;
|
||||
private final WorkerSetupManager workerSetupManager;
|
||||
private final JacksonConfigManager configManager;
|
||||
|
||||
public IndexerCoordinatorServletModule(
|
||||
ObjectMapper jsonMapper,
|
||||
|
@ -49,7 +49,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
|
|||
ServiceEmitter emitter,
|
||||
TaskMasterLifecycle taskMasterLifecycle,
|
||||
TaskStorageQueryAdapter taskStorageQueryAdapter,
|
||||
WorkerSetupManager workerSetupManager
|
||||
JacksonConfigManager configManager
|
||||
)
|
||||
{
|
||||
this.jsonMapper = jsonMapper;
|
||||
|
@ -57,7 +57,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
|
|||
this.emitter = emitter;
|
||||
this.taskMasterLifecycle = taskMasterLifecycle;
|
||||
this.taskStorageQueryAdapter = taskStorageQueryAdapter;
|
||||
this.workerSetupManager = workerSetupManager;
|
||||
this.configManager = configManager;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -69,7 +69,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
|
|||
bind(ServiceEmitter.class).toInstance(emitter);
|
||||
bind(TaskMasterLifecycle.class).toInstance(taskMasterLifecycle);
|
||||
bind(TaskStorageQueryAdapter.class).toInstance(taskStorageQueryAdapter);
|
||||
bind(WorkerSetupManager.class).toInstance(workerSetupManager);
|
||||
bind(JacksonConfigManager.class).toInstance(configManager);
|
||||
|
||||
serve("/*").with(GuiceContainer.class);
|
||||
}
|
||||
|
|
|
@ -34,13 +34,13 @@ import com.google.common.collect.Lists;
|
|||
import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig;
|
||||
import com.metamx.druid.merger.coordinator.setup.EC2NodeData;
|
||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
|
||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import org.apache.commons.codec.binary.Base64;
|
||||
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -51,26 +51,26 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy<Instance>
|
|||
private final ObjectMapper jsonMapper;
|
||||
private final AmazonEC2Client amazonEC2Client;
|
||||
private final EC2AutoScalingStrategyConfig config;
|
||||
private final WorkerSetupManager workerSetupManager;
|
||||
private final AtomicReference<WorkerSetupData> workerSetupDataRef;
|
||||
|
||||
public EC2AutoScalingStrategy(
|
||||
ObjectMapper jsonMapper,
|
||||
AmazonEC2Client amazonEC2Client,
|
||||
EC2AutoScalingStrategyConfig config,
|
||||
WorkerSetupManager workerSetupManager
|
||||
AtomicReference<WorkerSetupData> workerSetupDataRef
|
||||
)
|
||||
{
|
||||
this.jsonMapper = jsonMapper;
|
||||
this.amazonEC2Client = amazonEC2Client;
|
||||
this.config = config;
|
||||
this.workerSetupManager = workerSetupManager;
|
||||
this.workerSetupDataRef = workerSetupDataRef;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AutoScalingData<Instance> provision()
|
||||
{
|
||||
try {
|
||||
WorkerSetupData setupData = workerSetupManager.getWorkerSetupData();
|
||||
WorkerSetupData setupData = workerSetupDataRef.get();
|
||||
EC2NodeData workerConfig = setupData.getNodeData();
|
||||
|
||||
RunInstancesResult result = amazonEC2Client.runInstances(
|
||||
|
|
|
@ -1,3 +1,22 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package com.metamx.druid.merger.coordinator.scaling;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
|
|
|
@ -27,16 +27,16 @@ import com.google.common.collect.Sets;
|
|||
import com.metamx.common.guava.FunctionalIterable;
|
||||
import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem;
|
||||
import com.metamx.druid.merger.coordinator.ZkWorker;
|
||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
|
||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentSkipListSet;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -46,7 +46,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat
|
|||
|
||||
private final AutoScalingStrategy autoScalingStrategy;
|
||||
private final SimpleResourceManagmentConfig config;
|
||||
private final WorkerSetupManager workerSetupManager;
|
||||
private final AtomicReference<WorkerSetupData> workerSetupdDataRef;
|
||||
private final ScalingStats scalingStats;
|
||||
|
||||
private final ConcurrentSkipListSet<String> currentlyProvisioning = new ConcurrentSkipListSet<String>();
|
||||
|
@ -58,12 +58,12 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat
|
|||
public SimpleResourceManagementStrategy(
|
||||
AutoScalingStrategy autoScalingStrategy,
|
||||
SimpleResourceManagmentConfig config,
|
||||
WorkerSetupManager workerSetupManager
|
||||
AtomicReference<WorkerSetupData> workerSetupdDataRef
|
||||
)
|
||||
{
|
||||
this.autoScalingStrategy = autoScalingStrategy;
|
||||
this.config = config;
|
||||
this.workerSetupManager = workerSetupManager;
|
||||
this.workerSetupdDataRef = workerSetupdDataRef;
|
||||
this.scalingStats = new ScalingStats(config.getNumEventsToTrack());
|
||||
}
|
||||
|
||||
|
@ -151,7 +151,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat
|
|||
boolean nothingTerminating = currentlyTerminating.isEmpty();
|
||||
|
||||
if (nothingTerminating) {
|
||||
final int minNumWorkers = workerSetupManager.getWorkerSetupData().getMinNumWorkers();
|
||||
final int minNumWorkers = workerSetupdDataRef.get().getMinNumWorkers();
|
||||
if (zkWorkers.size() <= minNumWorkers) {
|
||||
log.info("Only [%d <= %d] nodes in the cluster, not terminating anything.", zkWorkers.size(), minNumWorkers);
|
||||
return false;
|
||||
|
|
|
@ -31,6 +31,8 @@ import java.util.List;
|
|||
*/
|
||||
public class WorkerSetupData
|
||||
{
|
||||
public static final String CONFIG_KEY = "worker.setup";
|
||||
|
||||
private final String minVersion;
|
||||
private final int minNumWorkers;
|
||||
private final EC2NodeData nodeData;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -23,6 +23,7 @@ import com.metamx.common.lifecycle.LifecycleStart;
|
|||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.common.TaskToolboxFactory;
|
||||
import com.metamx.druid.merger.common.task.Task;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import com.netflix.curator.framework.CuratorFramework;
|
||||
|
@ -45,21 +46,21 @@ public class TaskMonitor
|
|||
private final PathChildrenCache pathChildrenCache;
|
||||
private final CuratorFramework cf;
|
||||
private final WorkerCuratorCoordinator workerCuratorCoordinator;
|
||||
private final TaskToolbox toolbox;
|
||||
private final TaskToolboxFactory toolboxFactory;
|
||||
private final ExecutorService exec;
|
||||
|
||||
public TaskMonitor(
|
||||
PathChildrenCache pathChildrenCache,
|
||||
CuratorFramework cf,
|
||||
WorkerCuratorCoordinator workerCuratorCoordinator,
|
||||
TaskToolbox toolbox,
|
||||
TaskToolboxFactory toolboxFactory,
|
||||
ExecutorService exec
|
||||
)
|
||||
{
|
||||
this.pathChildrenCache = pathChildrenCache;
|
||||
this.cf = cf;
|
||||
this.workerCuratorCoordinator = workerCuratorCoordinator;
|
||||
this.toolbox = toolbox;
|
||||
this.toolboxFactory = toolboxFactory;
|
||||
this.exec = exec;
|
||||
}
|
||||
|
||||
|
@ -81,10 +82,11 @@ public class TaskMonitor
|
|||
throws Exception
|
||||
{
|
||||
if (pathChildrenCacheEvent.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) {
|
||||
final Task task = toolbox.getObjectMapper().readValue(
|
||||
final Task task = toolboxFactory.getObjectMapper().readValue(
|
||||
cf.getData().forPath(pathChildrenCacheEvent.getData().getPath()),
|
||||
Task.class
|
||||
);
|
||||
final TaskToolbox toolbox = toolboxFactory.build(task);
|
||||
|
||||
if (workerCuratorCoordinator.statusExists(task.getId())) {
|
||||
log.warn("Got task %s that I am already running...", task.getId());
|
||||
|
@ -99,7 +101,7 @@ public class TaskMonitor
|
|||
public void run()
|
||||
{
|
||||
final long startTime = System.currentTimeMillis();
|
||||
final File taskDir = toolbox.getConfig().getTaskDir(task);
|
||||
final File taskDir = toolbox.getTaskDir();
|
||||
|
||||
log.info("Running task [%s]", task.getId());
|
||||
|
||||
|
|
|
@ -35,15 +35,14 @@ import com.metamx.druid.http.StatusServlet;
|
|||
import com.metamx.druid.initialization.CuratorConfig;
|
||||
import com.metamx.druid.initialization.Initialization;
|
||||
import com.metamx.druid.initialization.ServerConfig;
|
||||
import com.metamx.druid.initialization.ServerInit;
|
||||
import com.metamx.druid.initialization.ServiceDiscoveryConfig;
|
||||
import com.metamx.druid.jackson.DefaultObjectMapper;
|
||||
import com.metamx.druid.loading.DataSegmentPusher;
|
||||
import com.metamx.druid.loading.S3DataSegmentPusher;
|
||||
import com.metamx.druid.loading.S3DataSegmentPusherConfig;
|
||||
import com.metamx.druid.loading.S3SegmentKiller;
|
||||
import com.metamx.druid.loading.SegmentKiller;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.common.actions.RemoteTaskActionClient;
|
||||
import com.metamx.druid.loading.S3DataSegmentKiller;
|
||||
import com.metamx.druid.loading.DataSegmentKiller;
|
||||
import com.metamx.druid.merger.common.TaskToolboxFactory;
|
||||
import com.metamx.druid.merger.common.actions.RemoteTaskActionClientFactory;
|
||||
import com.metamx.druid.merger.common.config.IndexerZkConfig;
|
||||
import com.metamx.druid.merger.common.config.TaskConfig;
|
||||
import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory;
|
||||
|
@ -106,7 +105,8 @@ public class WorkerNode extends RegisteringNode
|
|||
private ServiceEmitter emitter = null;
|
||||
private TaskConfig taskConfig = null;
|
||||
private WorkerConfig workerConfig = null;
|
||||
private TaskToolbox taskToolbox = null;
|
||||
private DataSegmentPusher segmentPusher = null;
|
||||
private TaskToolboxFactory taskToolboxFactory = null;
|
||||
private CuratorFramework curatorFramework = null;
|
||||
private ServiceDiscovery serviceDiscovery = null;
|
||||
private ServiceProvider coordinatorServiceProvider = null;
|
||||
|
@ -149,9 +149,15 @@ public class WorkerNode extends RegisteringNode
|
|||
return this;
|
||||
}
|
||||
|
||||
public WorkerNode setTaskToolbox(TaskToolbox taskToolbox)
|
||||
public WorkerNode setSegmentPusher(DataSegmentPusher segmentPusher)
|
||||
{
|
||||
this.taskToolbox = taskToolbox;
|
||||
this.segmentPusher = segmentPusher;
|
||||
return this;
|
||||
}
|
||||
|
||||
public WorkerNode setTaskToolboxFactory(TaskToolboxFactory taskToolboxFactory)
|
||||
{
|
||||
this.taskToolboxFactory = taskToolboxFactory;
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -195,6 +201,7 @@ public class WorkerNode extends RegisteringNode
|
|||
initializeCuratorFramework();
|
||||
initializeServiceDiscovery();
|
||||
initializeCoordinatorServiceProvider();
|
||||
initializeDataSegmentPusher();
|
||||
initializeTaskToolbox();
|
||||
initializeJacksonInjections();
|
||||
initializeJacksonSubtypes();
|
||||
|
@ -271,7 +278,7 @@ public class WorkerNode extends RegisteringNode
|
|||
InjectableValues.Std injectables = new InjectableValues.Std();
|
||||
|
||||
injectables.addValue("s3Client", s3Service)
|
||||
.addValue("segmentPusher", taskToolbox.getSegmentPusher());
|
||||
.addValue("segmentPusher", segmentPusher);
|
||||
|
||||
jsonMapper.setInjectableValues(injectables);
|
||||
}
|
||||
|
@ -334,24 +341,24 @@ public class WorkerNode extends RegisteringNode
|
|||
}
|
||||
}
|
||||
|
||||
public void initializeDataSegmentPusher()
|
||||
{
|
||||
if (segmentPusher == null) {
|
||||
segmentPusher = ServerInit.getSegmentPusher(props, configFactory, jsonMapper);
|
||||
}
|
||||
}
|
||||
|
||||
public void initializeTaskToolbox() throws S3ServiceException
|
||||
{
|
||||
if (taskToolbox == null) {
|
||||
final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher(
|
||||
s3Service,
|
||||
configFactory.build(S3DataSegmentPusherConfig.class),
|
||||
jsonMapper
|
||||
);
|
||||
final SegmentKiller segmentKiller = new S3SegmentKiller(
|
||||
s3Service
|
||||
);
|
||||
taskToolbox = new TaskToolbox(
|
||||
if (taskToolboxFactory == null) {
|
||||
final DataSegmentKiller dataSegmentKiller = new S3DataSegmentKiller(s3Service);
|
||||
taskToolboxFactory = new TaskToolboxFactory(
|
||||
taskConfig,
|
||||
new RemoteTaskActionClient(httpClient, coordinatorServiceProvider, jsonMapper),
|
||||
new RemoteTaskActionClientFactory(httpClient, coordinatorServiceProvider, jsonMapper),
|
||||
emitter,
|
||||
s3Service,
|
||||
dataSegmentPusher,
|
||||
segmentKiller,
|
||||
segmentPusher,
|
||||
dataSegmentKiller,
|
||||
jsonMapper
|
||||
);
|
||||
}
|
||||
|
@ -417,7 +424,7 @@ public class WorkerNode extends RegisteringNode
|
|||
pathChildrenCache,
|
||||
curatorFramework,
|
||||
workerCuratorCoordinator,
|
||||
taskToolbox,
|
||||
taskToolboxFactory,
|
||||
workerExec
|
||||
);
|
||||
lifecycle.addManagedInstance(taskMonitor);
|
||||
|
|
|
@ -26,14 +26,14 @@ import com.metamx.druid.aggregation.AggregatorFactory;
|
|||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.common.task.DefaultMergeTask;
|
||||
import com.metamx.druid.merger.common.task.MergeTask;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
@JsonTypeName("test")
|
||||
public class TestTask extends DefaultMergeTask
|
||||
public class TestTask extends MergeTask
|
||||
{
|
||||
private final String id;
|
||||
private final TaskStatus status;
|
||||
|
|
|
@ -31,7 +31,7 @@ import java.io.File;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class MergeTaskTest
|
||||
public class MergeTaskBaseTest
|
||||
{
|
||||
private final DataSegment.Builder segmentBuilder = DataSegment.builder()
|
||||
.dataSource("foo")
|
||||
|
@ -43,7 +43,7 @@ public class MergeTaskTest
|
|||
.add(segmentBuilder.interval(new Interval("2012-01-03/2012-01-05")).build())
|
||||
.build();
|
||||
|
||||
final MergeTask testMergeTask = new MergeTask("foo", segments)
|
||||
final MergeTaskBase testMergeTaskBase = new MergeTaskBase("foo", segments)
|
||||
{
|
||||
@Override
|
||||
protected File merge(Map<DataSegment, File> segments, File outDir) throws Exception
|
||||
|
@ -61,13 +61,13 @@ public class MergeTaskTest
|
|||
@Test
|
||||
public void testDataSource()
|
||||
{
|
||||
Assert.assertEquals("foo", testMergeTask.getDataSource());
|
||||
Assert.assertEquals("foo", testMergeTaskBase.getDataSource());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInterval()
|
||||
{
|
||||
Assert.assertEquals(new Interval("2012-01-03/2012-01-07"), testMergeTask.getImplicitLockInterval().get());
|
||||
Assert.assertEquals(new Interval("2012-01-03/2012-01-07"), testMergeTaskBase.getImplicitLockInterval().get());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -81,7 +81,7 @@ public class MergeTaskTest
|
|||
).toString().toLowerCase() + "_";
|
||||
Assert.assertEquals(
|
||||
desiredPrefix,
|
||||
testMergeTask.getId().substring(0, desiredPrefix.length())
|
||||
testMergeTaskBase.getId().substring(0, desiredPrefix.length())
|
||||
);
|
||||
}
|
||||
}
|
|
@ -11,13 +11,12 @@ import com.metamx.druid.jackson.DefaultObjectMapper;
|
|||
import com.metamx.druid.merger.TestTask;
|
||||
import com.metamx.druid.merger.common.TaskCallback;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.common.TaskToolboxFactory;
|
||||
import com.metamx.druid.merger.common.config.IndexerZkConfig;
|
||||
import com.metamx.druid.merger.common.config.TaskConfig;
|
||||
import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig;
|
||||
import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig;
|
||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
|
||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
|
||||
import com.metamx.druid.merger.worker.TaskMonitor;
|
||||
import com.metamx.druid.merger.worker.Worker;
|
||||
import com.metamx.druid.merger.worker.WorkerCuratorCoordinator;
|
||||
|
@ -42,6 +41,7 @@ import org.junit.Test;
|
|||
import java.io.File;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import static junit.framework.Assert.fail;
|
||||
|
||||
|
@ -60,7 +60,6 @@ public class RemoteTaskRunnerTest
|
|||
private PathChildrenCache pathChildrenCache;
|
||||
private RemoteTaskRunner remoteTaskRunner;
|
||||
private TaskMonitor taskMonitor;
|
||||
private WorkerSetupManager workerSetupManager;
|
||||
|
||||
private ScheduledExecutorService scheduledExec;
|
||||
|
||||
|
@ -280,7 +279,7 @@ public class RemoteTaskRunnerTest
|
|||
new PathChildrenCache(cf, String.format("%s/worker1", tasksPath), true),
|
||||
cf,
|
||||
workerCuratorCoordinator,
|
||||
new TaskToolbox(
|
||||
new TaskToolboxFactory(
|
||||
new TaskConfig()
|
||||
{
|
||||
@Override
|
||||
|
@ -316,17 +315,6 @@ public class RemoteTaskRunnerTest
|
|||
private void makeRemoteTaskRunner() throws Exception
|
||||
{
|
||||
scheduledExec = EasyMock.createMock(ScheduledExecutorService.class);
|
||||
workerSetupManager = EasyMock.createMock(WorkerSetupManager.class);
|
||||
|
||||
EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn(
|
||||
new WorkerSetupData(
|
||||
"0",
|
||||
0,
|
||||
null,
|
||||
null
|
||||
)
|
||||
).atLeastOnce();
|
||||
EasyMock.replay(workerSetupManager);
|
||||
|
||||
remoteTaskRunner = new RemoteTaskRunner(
|
||||
jsonMapper,
|
||||
|
@ -335,7 +323,7 @@ public class RemoteTaskRunnerTest
|
|||
pathChildrenCache,
|
||||
scheduledExec,
|
||||
new RetryPolicyFactory(new TestRetryPolicyConfig()),
|
||||
workerSetupManager
|
||||
new AtomicReference<WorkerSetupData>(new WorkerSetupData("0", 0, null, null))
|
||||
);
|
||||
|
||||
// Create a single worker and wait for things for be ready
|
||||
|
|
|
@ -1,3 +1,22 @@
|
|||
/*
|
||||
* Druid - a distributed column store.
|
||||
* Copyright (C) 2012 Metamarkets Group Inc.
|
||||
*
|
||||
* This program is free software; you can redistribute it and/or
|
||||
* modify it under the terms of the GNU General Public License
|
||||
* as published by the Free Software Foundation; either version 2
|
||||
* of the License, or (at your option) any later version.
|
||||
*
|
||||
* This program is distributed in the hope that it will be useful,
|
||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
||||
* GNU General Public License for more details.
|
||||
*
|
||||
* You should have received a copy of the GNU General Public License
|
||||
* along with this program; if not, write to the Free Software
|
||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package com.metamx.druid.merger.coordinator;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
|
@ -21,12 +40,13 @@ import com.metamx.druid.input.InputRow;
|
|||
import com.metamx.druid.input.MapBasedInputRow;
|
||||
import com.metamx.druid.jackson.DefaultObjectMapper;
|
||||
import com.metamx.druid.loading.DataSegmentPusher;
|
||||
import com.metamx.druid.loading.SegmentKiller;
|
||||
import com.metamx.druid.loading.DataSegmentKiller;
|
||||
import com.metamx.druid.loading.SegmentLoadingException;
|
||||
import com.metamx.druid.merger.common.TaskLock;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.common.actions.LocalTaskActionClient;
|
||||
import com.metamx.druid.merger.common.TaskToolboxFactory;
|
||||
import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory;
|
||||
import com.metamx.druid.merger.common.actions.LockAcquireAction;
|
||||
import com.metamx.druid.merger.common.actions.LockListAction;
|
||||
import com.metamx.druid.merger.common.actions.LockReleaseAction;
|
||||
|
@ -56,10 +76,8 @@ import org.junit.Test;
|
|||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Executors;
|
||||
|
||||
|
@ -71,7 +89,7 @@ public class TaskLifecycleTest
|
|||
private TaskQueue tq = null;
|
||||
private TaskRunner tr = null;
|
||||
private MockMergerDBCoordinator mdc = null;
|
||||
private TaskToolbox tb = null;
|
||||
private TaskToolboxFactory tb = null;
|
||||
private TaskConsumer tc = null;
|
||||
TaskStorageQueryAdapter tsqa = null;
|
||||
|
||||
|
@ -91,12 +109,12 @@ public class TaskLifecycleTest
|
|||
|
||||
tmp = Files.createTempDir();
|
||||
|
||||
ts = new LocalTaskStorage();
|
||||
ts = new HeapMemoryTaskStorage();
|
||||
tl = new TaskLockbox(ts);
|
||||
tq = new TaskQueue(ts, tl);
|
||||
mdc = newMockMDC();
|
||||
|
||||
tb = new TaskToolbox(
|
||||
tb = new TaskToolboxFactory(
|
||||
new TaskConfig()
|
||||
{
|
||||
@Override
|
||||
|
@ -117,7 +135,7 @@ public class TaskLifecycleTest
|
|||
return null;
|
||||
}
|
||||
},
|
||||
new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, mdc, newMockEmitter())),
|
||||
new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, mdc, newMockEmitter())),
|
||||
newMockEmitter(),
|
||||
null, // s3 client
|
||||
new DataSegmentPusher()
|
||||
|
@ -128,25 +146,16 @@ public class TaskLifecycleTest
|
|||
return segment;
|
||||
}
|
||||
},
|
||||
new SegmentKiller()
|
||||
new DataSegmentKiller()
|
||||
{
|
||||
@Override
|
||||
public void kill(Collection<DataSegment> segments) throws ServiceException
|
||||
public void kill(DataSegment segments) throws SegmentLoadingException
|
||||
{
|
||||
|
||||
}
|
||||
},
|
||||
new DefaultObjectMapper()
|
||||
)
|
||||
{
|
||||
@Override
|
||||
public Map<DataSegment, File> getSegments(
|
||||
Task task, List<DataSegment> segments
|
||||
) throws SegmentLoadingException
|
||||
{
|
||||
return ImmutableMap.of();
|
||||
}
|
||||
};
|
||||
);
|
||||
|
||||
tr = new LocalTaskRunner(
|
||||
tb,
|
||||
|
@ -239,11 +248,12 @@ public class TaskLifecycleTest
|
|||
@Test
|
||||
public void testKillTask() throws Exception
|
||||
{
|
||||
// TODO: Worst test ever
|
||||
// This test doesn't actually do anything right now. We should actually put things into the Mocked coordinator
|
||||
// Such that this test can test things...
|
||||
final Task killTask = new KillTask("foo", new Interval("2010-01-02/P2D"));
|
||||
|
||||
final TaskStatus mergedStatus = runTask(killTask);
|
||||
Assert.assertEquals("merged statusCode", TaskStatus.Status.SUCCESS, mergedStatus.getStatusCode());
|
||||
final TaskStatus status = runTask(killTask);
|
||||
Assert.assertEquals("merged statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode());
|
||||
Assert.assertEquals("num segments published", 0, mdc.getPublished().size());
|
||||
Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size());
|
||||
}
|
||||
|
@ -273,8 +283,8 @@ public class TaskLifecycleTest
|
|||
// Sort of similar to what realtime tasks do:
|
||||
|
||||
// Acquire lock for first interval
|
||||
final Optional<TaskLock> lock1 = toolbox.getTaskActionClient().submit(new LockAcquireAction(this, interval1));
|
||||
final List<TaskLock> locks1 = toolbox.getTaskActionClient().submit(new LockListAction(this));
|
||||
final Optional<TaskLock> lock1 = toolbox.getTaskActionClientFactory().submit(new LockAcquireAction(interval1));
|
||||
final List<TaskLock> locks1 = toolbox.getTaskActionClientFactory().submit(new LockListAction());
|
||||
|
||||
// (Confirm lock sanity)
|
||||
Assert.assertTrue("lock1 present", lock1.isPresent());
|
||||
|
@ -282,8 +292,8 @@ public class TaskLifecycleTest
|
|||
Assert.assertEquals("locks1", ImmutableList.of(lock1.get()), locks1);
|
||||
|
||||
// Acquire lock for second interval
|
||||
final Optional<TaskLock> lock2 = toolbox.getTaskActionClient().submit(new LockAcquireAction(this, interval2));
|
||||
final List<TaskLock> locks2 = toolbox.getTaskActionClient().submit(new LockListAction(this));
|
||||
final Optional<TaskLock> lock2 = toolbox.getTaskActionClientFactory().submit(new LockAcquireAction(interval2));
|
||||
final List<TaskLock> locks2 = toolbox.getTaskActionClientFactory().submit(new LockListAction());
|
||||
|
||||
// (Confirm lock sanity)
|
||||
Assert.assertTrue("lock2 present", lock2.isPresent());
|
||||
|
@ -291,10 +301,9 @@ public class TaskLifecycleTest
|
|||
Assert.assertEquals("locks2", ImmutableList.of(lock1.get(), lock2.get()), locks2);
|
||||
|
||||
// Push first segment
|
||||
toolbox.getTaskActionClient()
|
||||
toolbox.getTaskActionClientFactory()
|
||||
.submit(
|
||||
new SegmentInsertAction(
|
||||
this,
|
||||
ImmutableSet.of(
|
||||
DataSegment.builder()
|
||||
.dataSource("foo")
|
||||
|
@ -306,17 +315,16 @@ public class TaskLifecycleTest
|
|||
);
|
||||
|
||||
// Release first lock
|
||||
toolbox.getTaskActionClient().submit(new LockReleaseAction(this, interval1));
|
||||
final List<TaskLock> locks3 = toolbox.getTaskActionClient().submit(new LockListAction(this));
|
||||
toolbox.getTaskActionClientFactory().submit(new LockReleaseAction(interval1));
|
||||
final List<TaskLock> locks3 = toolbox.getTaskActionClientFactory().submit(new LockListAction());
|
||||
|
||||
// (Confirm lock sanity)
|
||||
Assert.assertEquals("locks3", ImmutableList.of(lock2.get()), locks3);
|
||||
|
||||
// Push second segment
|
||||
toolbox.getTaskActionClient()
|
||||
toolbox.getTaskActionClientFactory()
|
||||
.submit(
|
||||
new SegmentInsertAction(
|
||||
this,
|
||||
ImmutableSet.of(
|
||||
DataSegment.builder()
|
||||
.dataSource("foo")
|
||||
|
@ -328,8 +336,8 @@ public class TaskLifecycleTest
|
|||
);
|
||||
|
||||
// Release second lock
|
||||
toolbox.getTaskActionClient().submit(new LockReleaseAction(this, interval2));
|
||||
final List<TaskLock> locks4 = toolbox.getTaskActionClient().submit(new LockListAction(this));
|
||||
toolbox.getTaskActionClientFactory().submit(new LockReleaseAction(interval2));
|
||||
final List<TaskLock> locks4 = toolbox.getTaskActionClientFactory().submit(new LockListAction());
|
||||
|
||||
// (Confirm lock sanity)
|
||||
Assert.assertEquals("locks4", ImmutableList.<TaskLock>of(), locks4);
|
||||
|
@ -362,8 +370,8 @@ public class TaskLifecycleTest
|
|||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
final TaskLock myLock = Iterables.getOnlyElement(
|
||||
toolbox.getTaskActionClient()
|
||||
.submit(new LockListAction(this))
|
||||
toolbox.getTaskActionClientFactory()
|
||||
.submit(new LockListAction())
|
||||
);
|
||||
|
||||
final DataSegment segment = DataSegment.builder()
|
||||
|
@ -372,7 +380,7 @@ public class TaskLifecycleTest
|
|||
.version(myLock.getVersion())
|
||||
.build();
|
||||
|
||||
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment)));
|
||||
toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(segment)));
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
};
|
||||
|
@ -398,10 +406,7 @@ public class TaskLifecycleTest
|
|||
@Override
|
||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
final TaskLock myLock = Iterables.getOnlyElement(
|
||||
toolbox.getTaskActionClient()
|
||||
.submit(new LockListAction(this))
|
||||
);
|
||||
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction()));
|
||||
|
||||
final DataSegment segment = DataSegment.builder()
|
||||
.dataSource("ds")
|
||||
|
@ -409,7 +414,7 @@ public class TaskLifecycleTest
|
|||
.version(myLock.getVersion())
|
||||
.build();
|
||||
|
||||
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment)));
|
||||
toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(segment)));
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
};
|
||||
|
@ -435,10 +440,7 @@ public class TaskLifecycleTest
|
|||
@Override
|
||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
final TaskLock myLock = Iterables.getOnlyElement(
|
||||
toolbox.getTaskActionClient()
|
||||
.submit(new LockListAction(this))
|
||||
);
|
||||
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClientFactory().submit(new LockListAction()));
|
||||
|
||||
final DataSegment segment = DataSegment.builder()
|
||||
.dataSource("ds")
|
||||
|
@ -446,7 +448,7 @@ public class TaskLifecycleTest
|
|||
.version(myLock.getVersion() + "1!!!1!!")
|
||||
.build();
|
||||
|
||||
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment)));
|
||||
toolbox.getTaskActionClientFactory().submit(new SegmentInsertAction(ImmutableSet.of(segment)));
|
||||
return TaskStatus.success(getId());
|
||||
}
|
||||
};
|
||||
|
|
|
@ -26,7 +26,8 @@ import com.google.common.collect.Sets;
|
|||
import com.metamx.druid.merger.common.TaskLock;
|
||||
import com.metamx.druid.merger.common.TaskStatus;
|
||||
import com.metamx.druid.merger.common.TaskToolbox;
|
||||
import com.metamx.druid.merger.common.actions.LocalTaskActionClient;
|
||||
import com.metamx.druid.merger.common.TaskToolboxFactory;
|
||||
import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory;
|
||||
import com.metamx.druid.merger.common.actions.SpawnTasksAction;
|
||||
import com.metamx.druid.merger.common.actions.TaskActionToolbox;
|
||||
import com.metamx.druid.merger.common.task.AbstractTask;
|
||||
|
@ -43,7 +44,7 @@ public class TaskQueueTest
|
|||
@Test
|
||||
public void testEmptyQueue() throws Exception
|
||||
{
|
||||
final TaskStorage ts = new LocalTaskStorage();
|
||||
final TaskStorage ts = new HeapMemoryTaskStorage();
|
||||
final TaskLockbox tl = new TaskLockbox(ts);
|
||||
final TaskQueue tq = newTaskQueue(ts, tl);
|
||||
|
||||
|
@ -65,7 +66,7 @@ public class TaskQueueTest
|
|||
@Test
|
||||
public void testAddRemove() throws Exception
|
||||
{
|
||||
final TaskStorage ts = new LocalTaskStorage();
|
||||
final TaskStorage ts = new HeapMemoryTaskStorage();
|
||||
final TaskLockbox tl = new TaskLockbox(ts);
|
||||
final TaskQueue tq = newTaskQueue(ts, tl);
|
||||
|
||||
|
@ -154,12 +155,12 @@ public class TaskQueueTest
|
|||
@Test
|
||||
public void testContinues() throws Exception
|
||||
{
|
||||
final TaskStorage ts = new LocalTaskStorage();
|
||||
final TaskStorage ts = new HeapMemoryTaskStorage();
|
||||
final TaskLockbox tl = new TaskLockbox(ts);
|
||||
final TaskQueue tq = newTaskQueue(ts, tl);
|
||||
final TaskToolbox tb = new TaskToolbox(
|
||||
final TaskToolboxFactory tb = new TaskToolboxFactory(
|
||||
null,
|
||||
new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, null, null)),
|
||||
new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, null, null)),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
|
@ -181,7 +182,7 @@ public class TaskQueueTest
|
|||
Assert.assertNull("poll #2", tq.poll());
|
||||
|
||||
// report T1 done. Should cause T0 to be created
|
||||
tq.notify(t1, t1.run(tb));
|
||||
tq.notify(t1, t1.run(tb.build(t1)));
|
||||
|
||||
Assert.assertTrue("T0 isPresent (#2)", ts.getStatus("T0").isPresent());
|
||||
Assert.assertTrue("T0 isRunnable (#2)", ts.getStatus("T0").get().isRunnable());
|
||||
|
@ -195,7 +196,7 @@ public class TaskQueueTest
|
|||
Assert.assertNull("poll #4", tq.poll());
|
||||
|
||||
// report T0 done. Should cause T0, T1 to be marked complete
|
||||
tq.notify(t0, t0.run(tb));
|
||||
tq.notify(t0, t0.run(tb.build(t0)));
|
||||
|
||||
Assert.assertTrue("T0 isPresent (#3)", ts.getStatus("T0").isPresent());
|
||||
Assert.assertTrue("T0 isRunnable (#3)", !ts.getStatus("T0").get().isRunnable());
|
||||
|
@ -211,12 +212,12 @@ public class TaskQueueTest
|
|||
@Test
|
||||
public void testConcurrency() throws Exception
|
||||
{
|
||||
final TaskStorage ts = new LocalTaskStorage();
|
||||
final TaskStorage ts = new HeapMemoryTaskStorage();
|
||||
final TaskLockbox tl = new TaskLockbox(ts);
|
||||
final TaskQueue tq = newTaskQueue(ts, tl);
|
||||
final TaskToolbox tb = new TaskToolbox(
|
||||
final TaskToolboxFactory tb = new TaskToolboxFactory(
|
||||
null,
|
||||
new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, null, null)),
|
||||
new LocalTaskActionClientFactory(ts, new TaskActionToolbox(tq, tl, null, null)),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
|
@ -248,7 +249,7 @@ public class TaskQueueTest
|
|||
Thread.sleep(5);
|
||||
|
||||
// Finish t0
|
||||
tq.notify(t0, t0.run(tb));
|
||||
tq.notify(t0, t0.run(tb.build(t0)));
|
||||
|
||||
// take max number of tasks
|
||||
final Set<String> taken = Sets.newHashSet();
|
||||
|
@ -280,7 +281,7 @@ public class TaskQueueTest
|
|||
Assert.assertNull("null poll #2", tq.poll());
|
||||
|
||||
// Finish t3
|
||||
tq.notify(t3, t3.run(tb));
|
||||
tq.notify(t3, t3.run(tb.build(t3)));
|
||||
|
||||
// We should be able to get t2 now
|
||||
final Task wt2 = tq.poll();
|
||||
|
@ -291,7 +292,7 @@ public class TaskQueueTest
|
|||
Assert.assertNull("null poll #3", tq.poll());
|
||||
|
||||
// Finish t2
|
||||
tq.notify(t2, t2.run(tb));
|
||||
tq.notify(t2, t2.run(tb.build(t2)));
|
||||
|
||||
// We should be able to get t4
|
||||
// And it should be in group G0, but that group should have a different version than last time
|
||||
|
@ -305,14 +306,14 @@ public class TaskQueueTest
|
|||
Assert.assertNotSame("wt4 version", wt2Lock.getVersion(), wt4Lock.getVersion());
|
||||
|
||||
// Kind of done testing at this point, but let's finish t4 anyway
|
||||
tq.notify(t4, t4.run(tb));
|
||||
tq.notify(t4, t4.run(tb.build(t4)));
|
||||
Assert.assertNull("null poll #4", tq.poll());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBootstrap() throws Exception
|
||||
{
|
||||
final TaskStorage storage = new LocalTaskStorage();
|
||||
final TaskStorage storage = new HeapMemoryTaskStorage();
|
||||
final TaskLockbox lockbox = new TaskLockbox(storage);
|
||||
|
||||
storage.insert(newTask("T1", "G1", "bar", new Interval("2011-01-01/P1D")), TaskStatus.running("T1"));
|
||||
|
@ -374,7 +375,7 @@ public class TaskQueueTest
|
|||
@Override
|
||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, nextTasks));
|
||||
toolbox.getTaskActionClientFactory().submit(new SpawnTasksAction(nextTasks));
|
||||
return TaskStatus.success(id);
|
||||
}
|
||||
};
|
||||
|
|
|
@ -33,7 +33,6 @@ import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig;
|
|||
import com.metamx.druid.merger.coordinator.setup.EC2NodeData;
|
||||
import com.metamx.druid.merger.coordinator.setup.GalaxyUserData;
|
||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
|
||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
|
||||
import org.easymock.EasyMock;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
|
@ -42,6 +41,7 @@ import org.junit.Test;
|
|||
|
||||
import java.util.Arrays;
|
||||
import java.util.Date;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -57,7 +57,7 @@ public class EC2AutoScalingStrategyTest
|
|||
private Reservation reservation;
|
||||
private Instance instance;
|
||||
private EC2AutoScalingStrategy strategy;
|
||||
private WorkerSetupManager workerSetupManager;
|
||||
private AtomicReference<WorkerSetupData> workerSetupData;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception
|
||||
|
@ -66,7 +66,7 @@ public class EC2AutoScalingStrategyTest
|
|||
runInstancesResult = EasyMock.createMock(RunInstancesResult.class);
|
||||
describeInstancesResult = EasyMock.createMock(DescribeInstancesResult.class);
|
||||
reservation = EasyMock.createMock(Reservation.class);
|
||||
workerSetupManager = EasyMock.createMock(WorkerSetupManager.class);
|
||||
workerSetupData = new AtomicReference<WorkerSetupData>(null);
|
||||
|
||||
instance = new Instance()
|
||||
.withInstanceId(INSTANCE_ID)
|
||||
|
@ -85,7 +85,7 @@ public class EC2AutoScalingStrategyTest
|
|||
return "8080";
|
||||
}
|
||||
},
|
||||
workerSetupManager
|
||||
workerSetupData
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -96,13 +96,12 @@ public class EC2AutoScalingStrategyTest
|
|||
EasyMock.verify(runInstancesResult);
|
||||
EasyMock.verify(describeInstancesResult);
|
||||
EasyMock.verify(reservation);
|
||||
EasyMock.verify(workerSetupManager);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testScale()
|
||||
{
|
||||
EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn(
|
||||
workerSetupData.set(
|
||||
new WorkerSetupData(
|
||||
"0",
|
||||
0,
|
||||
|
@ -110,7 +109,6 @@ public class EC2AutoScalingStrategyTest
|
|||
new GalaxyUserData("env", "version", "type")
|
||||
)
|
||||
);
|
||||
EasyMock.replay(workerSetupManager);
|
||||
|
||||
EasyMock.expect(amazonEC2Client.runInstances(EasyMock.anyObject(RunInstancesRequest.class))).andReturn(
|
||||
runInstancesResult
|
||||
|
|
|
@ -29,7 +29,6 @@ import com.metamx.druid.merger.common.task.Task;
|
|||
import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem;
|
||||
import com.metamx.druid.merger.coordinator.ZkWorker;
|
||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
|
||||
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
|
||||
import com.metamx.druid.merger.worker.Worker;
|
||||
import junit.framework.Assert;
|
||||
import org.easymock.EasyMock;
|
||||
|
@ -42,21 +41,22 @@ import org.junit.Test;
|
|||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class SimpleResourceManagementStrategyTest
|
||||
{
|
||||
private AutoScalingStrategy autoScalingStrategy;
|
||||
private WorkerSetupManager workerSetupManager;
|
||||
private Task testTask;
|
||||
private SimpleResourceManagementStrategy simpleResourceManagementStrategy;
|
||||
private AtomicReference<WorkerSetupData> workerSetupData;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception
|
||||
{
|
||||
workerSetupManager = EasyMock.createMock(WorkerSetupManager.class);
|
||||
autoScalingStrategy = EasyMock.createMock(AutoScalingStrategy.class);
|
||||
workerSetupData = new AtomicReference<WorkerSetupData>(null);
|
||||
|
||||
testTask = new TestTask(
|
||||
"task1",
|
||||
|
@ -105,7 +105,7 @@ public class SimpleResourceManagementStrategyTest
|
|||
return new Duration(0);
|
||||
}
|
||||
},
|
||||
workerSetupManager
|
||||
workerSetupData
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -187,8 +187,7 @@ public class SimpleResourceManagementStrategyTest
|
|||
@Test
|
||||
public void testDoSuccessfulTerminate() throws Exception
|
||||
{
|
||||
EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn(new WorkerSetupData("0", 0, null, null));
|
||||
EasyMock.replay(workerSetupManager);
|
||||
workerSetupData.set(new WorkerSetupData("0", 0, null, null));
|
||||
|
||||
EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.<List<String>>anyObject()))
|
||||
.andReturn(Lists.<String>newArrayList());
|
||||
|
@ -212,15 +211,13 @@ public class SimpleResourceManagementStrategyTest
|
|||
simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.TERMINATE
|
||||
);
|
||||
|
||||
EasyMock.verify(workerSetupManager);
|
||||
EasyMock.verify(autoScalingStrategy);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSomethingTerminating() throws Exception
|
||||
{
|
||||
EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn(new WorkerSetupData("0", 0, null, null));
|
||||
EasyMock.replay(workerSetupManager);
|
||||
workerSetupData.set(new WorkerSetupData("0", 0, null, null));
|
||||
|
||||
EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.<List<String>>anyObject()))
|
||||
.andReturn(Lists.<String>newArrayList("ip")).times(2);
|
||||
|
@ -259,7 +256,6 @@ public class SimpleResourceManagementStrategyTest
|
|||
simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.TERMINATE
|
||||
);
|
||||
|
||||
EasyMock.verify(workerSetupManager);
|
||||
EasyMock.verify(autoScalingStrategy);
|
||||
}
|
||||
|
||||
|
|
12
pom.xml
12
pom.xml
|
@ -38,7 +38,7 @@
|
|||
|
||||
<properties>
|
||||
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
|
||||
<metamx.java-util.version>0.20.0</metamx.java-util.version>
|
||||
<metamx.java-util.version>0.21.0</metamx.java-util.version>
|
||||
</properties>
|
||||
|
||||
<modules>
|
||||
|
@ -165,17 +165,17 @@
|
|||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-annotations</artifactId>
|
||||
<version>2.1.2</version>
|
||||
<version>2.1.4</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-core</artifactId>
|
||||
<version>2.1.3</version>
|
||||
<version>2.1.4</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
<version>2.1.4-mmx-2</version>
|
||||
<version>2.1.4</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.datatype</groupId>
|
||||
|
@ -190,12 +190,12 @@
|
|||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.dataformat</groupId>
|
||||
<artifactId>jackson-dataformat-smile</artifactId>
|
||||
<version>2.1.3</version>
|
||||
<version>2.1.4</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.jaxrs</groupId>
|
||||
<artifactId>jackson-jaxrs-json-provider</artifactId>
|
||||
<version>2.1.3</version>
|
||||
<version>2.1.4</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.codehaus.jackson</groupId>
|
||||
|
|
|
@ -45,25 +45,15 @@ import com.metamx.druid.db.DbConnectorConfig;
|
|||
import com.metamx.druid.http.QueryServlet;
|
||||
import com.metamx.druid.http.StatusServlet;
|
||||
import com.metamx.druid.initialization.Initialization;
|
||||
import com.metamx.druid.initialization.ServerInit;
|
||||
import com.metamx.druid.jackson.DefaultObjectMapper;
|
||||
import com.metamx.druid.loading.DataSegmentPusher;
|
||||
import com.metamx.druid.loading.LocalDataSegmentPusher;
|
||||
import com.metamx.druid.loading.LocalDataSegmentPusherConfig;
|
||||
import com.metamx.druid.loading.S3DataSegmentPusher;
|
||||
import com.metamx.druid.loading.S3DataSegmentPusherConfig;
|
||||
import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
|
||||
import com.metamx.druid.utils.PropUtils;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.metamx.metrics.Monitor;
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
import org.jets3t.service.S3ServiceException;
|
||||
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
|
||||
import org.jets3t.service.security.AWSCredentials;
|
||||
import org.mortbay.jetty.servlet.Context;
|
||||
import org.mortbay.jetty.servlet.ServletHolder;
|
||||
import org.skife.config.ConfigurationObjectFactory;
|
||||
|
@ -259,31 +249,7 @@ public class RealtimeNode extends BaseServerNode<RealtimeNode>
|
|||
private void initializeSegmentPusher()
|
||||
{
|
||||
if (dataSegmentPusher == null) {
|
||||
final Properties props = getProps();
|
||||
if (Boolean.parseBoolean(props.getProperty("druid.pusher.local", "false"))) {
|
||||
dataSegmentPusher = new LocalDataSegmentPusher(
|
||||
getConfigFactory().build(LocalDataSegmentPusherConfig.class), getJsonMapper()
|
||||
);
|
||||
}
|
||||
else {
|
||||
|
||||
final RestS3Service s3Client;
|
||||
try {
|
||||
s3Client = new RestS3Service(
|
||||
new AWSCredentials(
|
||||
PropUtils.getProperty(props, "com.metamx.aws.accessKey"),
|
||||
PropUtils.getProperty(props, "com.metamx.aws.secretKey")
|
||||
)
|
||||
);
|
||||
}
|
||||
catch (S3ServiceException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
dataSegmentPusher = new S3DataSegmentPusher(
|
||||
s3Client, getConfigFactory().build(S3DataSegmentPusherConfig.class), getJsonMapper()
|
||||
);
|
||||
}
|
||||
dataSegmentPusher = ServerInit.getSegmentPusher(getProps(), getConfigFactory(), getJsonMapper());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -34,8 +34,8 @@ import com.metamx.druid.coordination.DruidClusterInfo;
|
|||
import com.metamx.druid.db.DatabaseRuleManager;
|
||||
import com.metamx.druid.db.DatabaseSegmentManager;
|
||||
import com.metamx.druid.master.DruidMaster;
|
||||
import com.metamx.druid.client.indexing.IndexingServiceClient;
|
||||
import com.metamx.druid.master.rules.Rule;
|
||||
import com.metamx.druid.merge.ClientKillQuery;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -66,6 +66,7 @@ public class InfoResource
|
|||
private final DatabaseSegmentManager databaseSegmentManager;
|
||||
private final DatabaseRuleManager databaseRuleManager;
|
||||
private final DruidClusterInfo druidClusterInfo;
|
||||
private final IndexingServiceClient indexingServiceClient;
|
||||
|
||||
@Inject
|
||||
public InfoResource(
|
||||
|
@ -73,7 +74,8 @@ public class InfoResource
|
|||
ServerInventoryManager serverInventoryManager,
|
||||
DatabaseSegmentManager databaseSegmentManager,
|
||||
DatabaseRuleManager databaseRuleManager,
|
||||
DruidClusterInfo druidClusterInfo
|
||||
DruidClusterInfo druidClusterInfo,
|
||||
IndexingServiceClient indexingServiceClient
|
||||
)
|
||||
{
|
||||
this.master = master;
|
||||
|
@ -81,6 +83,7 @@ public class InfoResource
|
|||
this.databaseSegmentManager = databaseSegmentManager;
|
||||
this.databaseRuleManager = databaseRuleManager;
|
||||
this.druidClusterInfo = druidClusterInfo;
|
||||
this.indexingServiceClient = indexingServiceClient;
|
||||
}
|
||||
|
||||
@GET
|
||||
|
@ -374,7 +377,7 @@ public class InfoResource
|
|||
)
|
||||
{
|
||||
if (kill != null && Boolean.valueOf(kill)) {
|
||||
master.killSegments(new ClientKillQuery(dataSourceName, new Interval(interval)));
|
||||
indexingServiceClient.killSegments(dataSourceName, new Interval(interval));
|
||||
} else {
|
||||
if (!databaseSegmentManager.removeDatasource(dataSourceName)) {
|
||||
return Response.status(Response.Status.NOT_FOUND).build();
|
||||
|
|
|
@ -33,6 +33,9 @@ import com.metamx.common.lifecycle.Lifecycle;
|
|||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.client.ServerInventoryManager;
|
||||
import com.metamx.druid.client.ServerInventoryManagerConfig;
|
||||
import com.metamx.druid.config.ConfigManager;
|
||||
import com.metamx.druid.config.ConfigManagerConfig;
|
||||
import com.metamx.druid.config.JacksonConfigManager;
|
||||
import com.metamx.druid.coordination.DruidClusterInfo;
|
||||
import com.metamx.druid.coordination.DruidClusterInfoConfig;
|
||||
import com.metamx.druid.db.DatabaseRuleManager;
|
||||
|
@ -49,6 +52,7 @@ import com.metamx.druid.jackson.DefaultObjectMapper;
|
|||
import com.metamx.druid.log.LogLevelAdjuster;
|
||||
import com.metamx.druid.master.DruidMaster;
|
||||
import com.metamx.druid.master.DruidMasterConfig;
|
||||
import com.metamx.druid.client.indexing.IndexingServiceClient;
|
||||
import com.metamx.druid.master.LoadQueuePeon;
|
||||
import com.metamx.druid.utils.PropUtils;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
|
@ -86,7 +90,7 @@ import java.util.concurrent.ScheduledExecutorService;
|
|||
*/
|
||||
public class MasterMain
|
||||
{
|
||||
private static final Logger log = new Logger(ServerMain.class);
|
||||
private static final Logger log = new Logger(MasterMain.class);
|
||||
|
||||
public static void main(String[] args) throws Exception
|
||||
{
|
||||
|
@ -166,13 +170,14 @@ public class MasterMain
|
|||
lifecycle
|
||||
);
|
||||
|
||||
ServiceProvider serviceProvider = null;
|
||||
IndexingServiceClient indexingServiceClient = null;
|
||||
if (druidMasterConfig.getMergerServiceName() != null) {
|
||||
serviceProvider = Initialization.makeServiceProvider(
|
||||
ServiceProvider serviceProvider = Initialization.makeServiceProvider(
|
||||
druidMasterConfig.getMergerServiceName(),
|
||||
serviceDiscovery,
|
||||
lifecycle
|
||||
);
|
||||
indexingServiceClient = new IndexingServiceClient(httpClient, jsonMapper, serviceProvider);
|
||||
}
|
||||
|
||||
final DruidClusterInfo druidClusterInfo = new DruidClusterInfo(
|
||||
|
@ -180,10 +185,14 @@ public class MasterMain
|
|||
masterYp
|
||||
);
|
||||
|
||||
JacksonConfigManager configManager = new JacksonConfigManager(
|
||||
new ConfigManager(dbi, configFactory.build(ConfigManagerConfig.class)), jsonMapper
|
||||
);
|
||||
|
||||
final DruidMaster master = new DruidMaster(
|
||||
druidMasterConfig,
|
||||
druidClusterInfo,
|
||||
jsonMapper,
|
||||
configManager,
|
||||
databaseSegmentManager,
|
||||
serverInventoryManager,
|
||||
databaseRuleManager,
|
||||
|
@ -191,9 +200,7 @@ public class MasterMain
|
|||
emitter,
|
||||
scheduledExecutorFactory,
|
||||
new ConcurrentHashMap<String, LoadQueuePeon>(),
|
||||
serviceProvider,
|
||||
httpClient,
|
||||
new ToStringResponseHandler(Charsets.UTF_8)
|
||||
indexingServiceClient
|
||||
);
|
||||
lifecycle.addManagedInstance(master);
|
||||
|
||||
|
@ -226,7 +233,8 @@ public class MasterMain
|
|||
databaseRuleManager,
|
||||
druidClusterInfo,
|
||||
master,
|
||||
jsonMapper
|
||||
jsonMapper,
|
||||
indexingServiceClient
|
||||
)
|
||||
);
|
||||
|
||||
|
|
|
@ -21,7 +21,6 @@ package com.metamx.druid.http;
|
|||
|
||||
import com.metamx.druid.master.DruidMaster;
|
||||
import com.metamx.druid.master.LoadPeonCallback;
|
||||
import com.metamx.druid.merge.ClientKillQuery;
|
||||
|
||||
import javax.inject.Inject;
|
||||
import javax.ws.rs.Consumes;
|
||||
|
|
|
@ -27,6 +27,7 @@ import com.metamx.druid.coordination.DruidClusterInfo;
|
|||
import com.metamx.druid.db.DatabaseRuleManager;
|
||||
import com.metamx.druid.db.DatabaseSegmentManager;
|
||||
import com.metamx.druid.master.DruidMaster;
|
||||
import com.metamx.druid.client.indexing.IndexingServiceClient;
|
||||
import com.sun.jersey.guice.JerseyServletModule;
|
||||
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
|
||||
|
||||
|
@ -44,6 +45,7 @@ public class MasterServletModule extends JerseyServletModule
|
|||
private final DruidClusterInfo druidClusterInfo;
|
||||
private final DruidMaster master;
|
||||
private final ObjectMapper jsonMapper;
|
||||
private final IndexingServiceClient indexingServiceClient;
|
||||
|
||||
public MasterServletModule(
|
||||
ServerInventoryManager serverInventoryManager,
|
||||
|
@ -51,7 +53,8 @@ public class MasterServletModule extends JerseyServletModule
|
|||
DatabaseRuleManager databaseRuleManager,
|
||||
DruidClusterInfo druidClusterInfo,
|
||||
DruidMaster master,
|
||||
ObjectMapper jsonMapper
|
||||
ObjectMapper jsonMapper,
|
||||
IndexingServiceClient indexingServiceClient
|
||||
)
|
||||
{
|
||||
this.serverInventoryManager = serverInventoryManager;
|
||||
|
@ -60,6 +63,7 @@ public class MasterServletModule extends JerseyServletModule
|
|||
this.druidClusterInfo = druidClusterInfo;
|
||||
this.master = master;
|
||||
this.jsonMapper = jsonMapper;
|
||||
this.indexingServiceClient = indexingServiceClient;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -72,6 +76,7 @@ public class MasterServletModule extends JerseyServletModule
|
|||
bind(DatabaseRuleManager.class).toInstance(databaseRuleManager);
|
||||
bind(DruidMaster.class).toInstance(master);
|
||||
bind(DruidClusterInfo.class).toInstance(druidClusterInfo);
|
||||
bind(IndexingServiceClient.class).toInstance(indexingServiceClient);
|
||||
|
||||
serve("/*").with(GuiceContainer.class);
|
||||
}
|
||||
|
|
|
@ -19,17 +19,24 @@
|
|||
|
||||
package com.metamx.druid.initialization;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.metamx.common.ISE;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.DruidProcessingConfig;
|
||||
import com.metamx.druid.loading.DataSegmentPusher;
|
||||
import com.metamx.druid.loading.DelegatingSegmentLoader;
|
||||
import com.metamx.druid.loading.LocalDataSegmentPuller;
|
||||
import com.metamx.druid.loading.LocalDataSegmentPusher;
|
||||
import com.metamx.druid.loading.LocalDataSegmentPusherConfig;
|
||||
import com.metamx.druid.loading.MMappedQueryableIndexFactory;
|
||||
import com.metamx.druid.loading.QueryableIndexFactory;
|
||||
import com.metamx.druid.loading.S3DataSegmentPuller;
|
||||
import com.metamx.druid.loading.S3DataSegmentPusher;
|
||||
import com.metamx.druid.loading.S3DataSegmentPusherConfig;
|
||||
import com.metamx.druid.loading.SegmentLoaderConfig;
|
||||
import com.metamx.druid.loading.SingleSegmentLoader;
|
||||
import com.metamx.druid.query.group.GroupByQueryEngine;
|
||||
|
@ -48,12 +55,16 @@ import com.metamx.druid.query.timeboundary.TimeBoundaryQuery;
|
|||
import com.metamx.druid.query.timeboundary.TimeBoundaryQueryRunnerFactory;
|
||||
import com.metamx.druid.query.timeseries.TimeseriesQuery;
|
||||
import com.metamx.druid.query.timeseries.TimeseriesQueryRunnerFactory;
|
||||
import com.metamx.druid.utils.PropUtils;
|
||||
import org.jets3t.service.S3ServiceException;
|
||||
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
|
||||
import org.jets3t.service.security.AWSCredentials;
|
||||
import org.skife.config.ConfigurationObjectFactory;
|
||||
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
/**
|
||||
|
@ -145,6 +156,34 @@ public class ServerInit
|
|||
return queryRunners;
|
||||
}
|
||||
|
||||
public static DataSegmentPusher getSegmentPusher(
|
||||
final Properties props,
|
||||
final ConfigurationObjectFactory configFactory,
|
||||
final ObjectMapper jsonMapper
|
||||
)
|
||||
{
|
||||
if (Boolean.parseBoolean(props.getProperty("druid.pusher.local", "false"))) {
|
||||
return new LocalDataSegmentPusher(configFactory.build(LocalDataSegmentPusherConfig.class), jsonMapper);
|
||||
}
|
||||
else {
|
||||
|
||||
final RestS3Service s3Client;
|
||||
try {
|
||||
s3Client = new RestS3Service(
|
||||
new AWSCredentials(
|
||||
PropUtils.getProperty(props, "com.metamx.aws.accessKey"),
|
||||
PropUtils.getProperty(props, "com.metamx.aws.secretKey")
|
||||
)
|
||||
);
|
||||
}
|
||||
catch (S3ServiceException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
|
||||
return new S3DataSegmentPusher(s3Client, configFactory.build(S3DataSegmentPusherConfig.class), jsonMapper);
|
||||
}
|
||||
}
|
||||
|
||||
private static class ComputeScratchPool extends StupidPool<ByteBuffer>
|
||||
{
|
||||
private static final Logger log = new Logger(ComputeScratchPool.class);
|
||||
|
|
|
@ -17,15 +17,13 @@
|
|||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||
*/
|
||||
|
||||
package com.metamx.druid.master;
|
||||
package com.metamx.druid.loading;
|
||||
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface MergerClient
|
||||
public interface DataSegmentKiller
|
||||
{
|
||||
public void runRequest(String dataSource, List<DataSegment> segments);
|
||||
public void kill(DataSegment segments) throws SegmentLoadingException;
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -87,11 +87,7 @@ public class S3DataSegmentPusher implements DataSegmentPusher
|
|||
|
||||
segment = segment.withSize(indexSize)
|
||||
.withLoadSpec(
|
||||
ImmutableMap.<String, Object>of(
|
||||
"type", "s3_zip",
|
||||
"bucket", outputBucket,
|
||||
"key", toPush.getKey()
|
||||
)
|
||||
ImmutableMap.<String, Object>of("type", "s3_zip", "bucket", outputBucket, "key", toPush.getKey())
|
||||
)
|
||||
.withBinaryVersion(IndexIO.getVersionFromDir(indexFilesDir));
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -19,10 +19,8 @@
|
|||
|
||||
package com.metamx.druid.master;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
|
@ -41,32 +39,30 @@ import com.metamx.druid.client.DataSegment;
|
|||
import com.metamx.druid.client.DruidDataSource;
|
||||
import com.metamx.druid.client.DruidServer;
|
||||
import com.metamx.druid.client.ServerInventoryManager;
|
||||
import com.metamx.druid.client.indexing.IndexingServiceClient;
|
||||
import com.metamx.druid.config.JacksonConfigManager;
|
||||
import com.metamx.druid.coordination.DruidClusterInfo;
|
||||
import com.metamx.druid.db.DatabaseRuleManager;
|
||||
import com.metamx.druid.db.DatabaseSegmentManager;
|
||||
import com.metamx.druid.merge.ClientKillQuery;
|
||||
import com.metamx.druid.index.v1.IndexIO;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import com.metamx.emitter.service.ServiceEmitter;
|
||||
import com.metamx.emitter.service.ServiceMetricEvent;
|
||||
import com.metamx.http.client.HttpClient;
|
||||
import com.metamx.http.client.response.HttpResponseHandler;
|
||||
import com.metamx.phonebook.PhoneBook;
|
||||
import com.metamx.phonebook.PhoneBookPeon;
|
||||
import com.netflix.curator.x.discovery.ServiceProvider;
|
||||
import org.I0Itec.zkclient.exception.ZkNodeExistsException;
|
||||
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Duration;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.net.URL;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -83,26 +79,22 @@ public class DruidMaster
|
|||
|
||||
private final DruidMasterConfig config;
|
||||
private final DruidClusterInfo clusterInfo;
|
||||
private final JacksonConfigManager configManager;
|
||||
private final DatabaseSegmentManager databaseSegmentManager;
|
||||
private final ServerInventoryManager serverInventoryManager;
|
||||
private final DatabaseRuleManager databaseRuleManager;
|
||||
private final PhoneBook yp;
|
||||
private final ServiceEmitter emitter;
|
||||
private final IndexingServiceClient indexingServiceClient;
|
||||
private final ScheduledExecutorService exec;
|
||||
private final ScheduledExecutorService peonExec;
|
||||
private final PhoneBookPeon masterPeon;
|
||||
private final Map<String, LoadQueuePeon> loadManagementPeons;
|
||||
private final ServiceProvider serviceProvider;
|
||||
|
||||
private final HttpClient httpClient;
|
||||
private final HttpResponseHandler responseHandler;
|
||||
|
||||
private final ObjectMapper jsonMapper;
|
||||
|
||||
public DruidMaster(
|
||||
DruidMasterConfig config,
|
||||
DruidClusterInfo clusterInfo,
|
||||
ObjectMapper jsonMapper,
|
||||
JacksonConfigManager configManager,
|
||||
DatabaseSegmentManager databaseSegmentManager,
|
||||
ServerInventoryManager serverInventoryManager,
|
||||
DatabaseRuleManager databaseRuleManager,
|
||||
|
@ -110,31 +102,25 @@ public class DruidMaster
|
|||
ServiceEmitter emitter,
|
||||
ScheduledExecutorFactory scheduledExecutorFactory,
|
||||
Map<String, LoadQueuePeon> loadManagementPeons,
|
||||
ServiceProvider serviceProvider,
|
||||
HttpClient httpClient,
|
||||
HttpResponseHandler responseHandler
|
||||
IndexingServiceClient indexingServiceClient
|
||||
)
|
||||
{
|
||||
this.config = config;
|
||||
this.clusterInfo = clusterInfo;
|
||||
|
||||
this.jsonMapper = jsonMapper;
|
||||
this.configManager = configManager;
|
||||
|
||||
this.databaseSegmentManager = databaseSegmentManager;
|
||||
this.serverInventoryManager = serverInventoryManager;
|
||||
this.databaseRuleManager = databaseRuleManager;
|
||||
this.yp = zkPhoneBook;
|
||||
this.emitter = emitter;
|
||||
this.indexingServiceClient = indexingServiceClient;
|
||||
|
||||
this.masterPeon = new MasterListeningPeon();
|
||||
this.exec = scheduledExecutorFactory.create(1, "Master-Exec--%d");
|
||||
this.peonExec = scheduledExecutorFactory.create(1, "Master-PeonExec--%d");
|
||||
|
||||
this.loadManagementPeons = loadManagementPeons;
|
||||
|
||||
this.serviceProvider = serviceProvider;
|
||||
this.httpClient = httpClient;
|
||||
this.responseHandler = responseHandler;
|
||||
}
|
||||
|
||||
public boolean isClusterMaster()
|
||||
|
@ -349,27 +335,6 @@ public class DruidMaster
|
|||
}
|
||||
}
|
||||
|
||||
public void killSegments(ClientKillQuery killQuery)
|
||||
{
|
||||
try {
|
||||
httpClient.post(
|
||||
new URL(
|
||||
String.format(
|
||||
"http://%s:%s/mmx/merger/v1/index",
|
||||
serviceProvider.getInstance().getAddress(),
|
||||
serviceProvider.getInstance().getPort()
|
||||
)
|
||||
)
|
||||
)
|
||||
.setContent("application/json", jsonMapper.writeValueAsBytes(killQuery))
|
||||
.go(responseHandler)
|
||||
.get();
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
public Set<DataSegment> getAvailableDataSegments()
|
||||
{
|
||||
Set<DataSegment> availableSegments = Sets.newTreeSet(Comparators.inverse(DataSegment.bucketMonthComparator()));
|
||||
|
@ -390,7 +355,9 @@ public class DruidMaster
|
|||
|
||||
for (DataSegment dataSegment : dataSegments) {
|
||||
if (dataSegment.getSize() < 0) {
|
||||
log.warn("No size on Segment[%s], wtf?", dataSegment);
|
||||
log.makeAlert("No size on Segment, wtf?")
|
||||
.addData("segment", dataSegment)
|
||||
.emit();
|
||||
}
|
||||
availableSegments.add(dataSegment);
|
||||
}
|
||||
|
@ -466,8 +433,14 @@ public class DruidMaster
|
|||
final List<Pair<? extends MasterRunnable, Duration>> masterRunnables = Lists.newArrayList();
|
||||
|
||||
masterRunnables.add(Pair.of(new MasterComputeManagerRunnable(), config.getMasterPeriod()));
|
||||
if (config.isMergeSegments() && serviceProvider != null) {
|
||||
masterRunnables.add(Pair.of(new MasterSegmentMergerRunnable(), config.getMasterSegmentMergerPeriod()));
|
||||
if (config.isMergeSegments() && indexingServiceClient != null) {
|
||||
|
||||
masterRunnables.add(
|
||||
Pair.of(
|
||||
new MasterSegmentMergerRunnable(configManager.watch(MergerWhitelist.CONFIG_KEY, MergerWhitelist.class)),
|
||||
config.getMasterSegmentMergerPeriod()
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
for (final Pair<? extends MasterRunnable, Duration> masterRunnable : masterRunnables) {
|
||||
|
@ -529,6 +502,39 @@ public class DruidMaster
|
|||
}
|
||||
}
|
||||
|
||||
public static class DruidMasterVersionConverter implements DruidMasterHelper
|
||||
{
|
||||
private final IndexingServiceClient indexingServiceClient;
|
||||
private final AtomicReference<MergerWhitelist> whitelistRef;
|
||||
|
||||
public DruidMasterVersionConverter(
|
||||
IndexingServiceClient indexingServiceClient,
|
||||
AtomicReference<MergerWhitelist> whitelistRef
|
||||
)
|
||||
{
|
||||
this.indexingServiceClient = indexingServiceClient;
|
||||
this.whitelistRef = whitelistRef;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params)
|
||||
{
|
||||
MergerWhitelist whitelist = whitelistRef.get();
|
||||
|
||||
for (DataSegment dataSegment : params.getAvailableSegments()) {
|
||||
if (whitelist == null || whitelist.contains(dataSegment.getDataSource())) {
|
||||
final Integer binaryVersion = dataSegment.getBinaryVersion();
|
||||
|
||||
if (binaryVersion == null || binaryVersion < IndexIO.CURRENT_VERSION_ID) {
|
||||
indexingServiceClient.upgradeSegment(dataSegment);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return params;
|
||||
}
|
||||
}
|
||||
|
||||
private class MasterListeningPeon implements PhoneBookPeon<Map>
|
||||
{
|
||||
@Override
|
||||
|
@ -723,12 +729,13 @@ public class DruidMaster
|
|||
|
||||
private class MasterSegmentMergerRunnable extends MasterRunnable
|
||||
{
|
||||
private MasterSegmentMergerRunnable()
|
||||
private MasterSegmentMergerRunnable(final AtomicReference<MergerWhitelist> whitelistRef)
|
||||
{
|
||||
super(
|
||||
ImmutableList.of(
|
||||
new DruidMasterSegmentInfoLoader(DruidMaster.this),
|
||||
new DruidMasterSegmentMerger(jsonMapper, serviceProvider),
|
||||
new DruidMasterVersionConverter(indexingServiceClient, whitelistRef),
|
||||
new DruidMasterSegmentMerger(indexingServiceClient, whitelistRef),
|
||||
new DruidMasterHelper()
|
||||
{
|
||||
@Override
|
||||
|
@ -739,8 +746,7 @@ public class DruidMaster
|
|||
|
||||
params.getEmitter().emit(
|
||||
new ServiceMetricEvent.Builder().build(
|
||||
"master/merge/count",
|
||||
stats.getGlobalStats().get("mergedCount")
|
||||
"master/merge/count", stats.getGlobalStats().get("mergedCount")
|
||||
)
|
||||
);
|
||||
|
||||
|
|
|
@ -19,8 +19,6 @@
|
|||
|
||||
package com.metamx.druid.master;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Charsets;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.HashMultiset;
|
||||
|
@ -32,22 +30,19 @@ import com.google.common.collect.Multiset;
|
|||
import com.google.common.collect.Ordering;
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.guava.FunctionalIterable;
|
||||
import com.metamx.common.lifecycle.Lifecycle;
|
||||
import com.metamx.common.logger.Logger;
|
||||
import com.metamx.druid.TimelineObjectHolder;
|
||||
import com.metamx.druid.VersionedIntervalTimeline;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.client.indexing.IndexingServiceClient;
|
||||
import com.metamx.druid.partition.PartitionChunk;
|
||||
import com.metamx.http.client.HttpClientConfig;
|
||||
import com.metamx.http.client.HttpClientInit;
|
||||
import com.metamx.http.client.response.ToStringResponseHandler;
|
||||
import com.netflix.curator.x.discovery.ServiceProvider;
|
||||
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -55,44 +50,40 @@ public class DruidMasterSegmentMerger implements DruidMasterHelper
|
|||
{
|
||||
private static final Logger log = new Logger(DruidMasterSegmentMerger.class);
|
||||
|
||||
private final MergerClient mergerClient;
|
||||
private final IndexingServiceClient indexingServiceClient;
|
||||
private final AtomicReference<MergerWhitelist> whiteListRef;
|
||||
|
||||
public DruidMasterSegmentMerger(MergerClient mergerClient)
|
||||
public DruidMasterSegmentMerger(
|
||||
IndexingServiceClient indexingServiceClient,
|
||||
AtomicReference<MergerWhitelist> whitelistRef
|
||||
)
|
||||
{
|
||||
this.mergerClient = mergerClient;
|
||||
}
|
||||
|
||||
public DruidMasterSegmentMerger(ObjectMapper jsonMapper, ServiceProvider serviceProvider)
|
||||
{
|
||||
this.mergerClient = new HttpMergerClient(
|
||||
HttpClientInit.createClient(
|
||||
HttpClientConfig.builder().withNumConnections(1).build(),
|
||||
new Lifecycle()
|
||||
),
|
||||
new ToStringResponseHandler(Charsets.UTF_8),
|
||||
jsonMapper,
|
||||
serviceProvider
|
||||
);
|
||||
this.indexingServiceClient = indexingServiceClient;
|
||||
this.whiteListRef = whitelistRef;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params)
|
||||
{
|
||||
MergerWhitelist whitelist = whiteListRef.get();
|
||||
|
||||
MasterStats stats = new MasterStats();
|
||||
Map<String, VersionedIntervalTimeline<String, DataSegment>> dataSources = Maps.newHashMap();
|
||||
|
||||
// Find serviced segments by using a timeline
|
||||
for (DataSegment dataSegment : params.getAvailableSegments()) {
|
||||
VersionedIntervalTimeline<String, DataSegment> timeline = dataSources.get(dataSegment.getDataSource());
|
||||
if (timeline == null) {
|
||||
timeline = new VersionedIntervalTimeline<String, DataSegment>(Ordering.<String>natural());
|
||||
dataSources.put(dataSegment.getDataSource(), timeline);
|
||||
if (whitelist == null || whitelist.contains(dataSegment.getDataSource())) {
|
||||
VersionedIntervalTimeline<String, DataSegment> timeline = dataSources.get(dataSegment.getDataSource());
|
||||
if (timeline == null) {
|
||||
timeline = new VersionedIntervalTimeline<String, DataSegment>(Ordering.<String>natural());
|
||||
dataSources.put(dataSegment.getDataSource(), timeline);
|
||||
}
|
||||
timeline.add(
|
||||
dataSegment.getInterval(),
|
||||
dataSegment.getVersion(),
|
||||
dataSegment.getShardSpec().createChunk(dataSegment)
|
||||
);
|
||||
}
|
||||
timeline.add(
|
||||
dataSegment.getInterval(),
|
||||
dataSegment.getVersion(),
|
||||
dataSegment.getShardSpec().createChunk(dataSegment)
|
||||
);
|
||||
}
|
||||
|
||||
// Find segments to merge
|
||||
|
@ -161,7 +152,7 @@ public class DruidMasterSegmentMerger implements DruidMasterHelper
|
|||
log.info("[%s] Found %d segments to merge %s", dataSource, segments.size(), segmentNames);
|
||||
|
||||
try {
|
||||
mergerClient.runRequest(dataSource, segments);
|
||||
indexingServiceClient.mergeSegments(segments);
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.error(
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -23,12 +23,14 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.druid.client.DataSegment;
|
||||
import com.metamx.druid.client.indexing.IndexingServiceClient;
|
||||
import junit.framework.Assert;
|
||||
import org.joda.time.Interval;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
public class DruidMasterSegmentMergerTest
|
||||
{
|
||||
|
@ -367,11 +369,7 @@ public class DruidMasterSegmentMergerTest
|
|||
DataSegment.builder().dataSource("foo").interval(new Interval("2012-01-06/P1D")).version("2").size(80).build()
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
ImmutableList.of(segments.get(4), segments.get(5))
|
||||
), merge(segments)
|
||||
);
|
||||
Assert.assertEquals(ImmutableList.of(ImmutableList.of(segments.get(4), segments.get(5))), merge(segments));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -380,16 +378,17 @@ public class DruidMasterSegmentMergerTest
|
|||
private static List<List<DataSegment>> merge(final Collection<DataSegment> segments)
|
||||
{
|
||||
final List<List<DataSegment>> retVal = Lists.newArrayList();
|
||||
final MergerClient mergerClient = new MergerClient()
|
||||
final IndexingServiceClient indexingServiceClient = new IndexingServiceClient(null, null, null)
|
||||
{
|
||||
@Override
|
||||
public void runRequest(String dataSource, List<DataSegment> segmentsToMerge)
|
||||
public void mergeSegments(List<DataSegment> segmentsToMerge)
|
||||
{
|
||||
retVal.add(segmentsToMerge);
|
||||
}
|
||||
};
|
||||
|
||||
final DruidMasterSegmentMerger merger = new DruidMasterSegmentMerger(mergerClient);
|
||||
final AtomicReference<MergerWhitelist> whitelistRef = new AtomicReference<MergerWhitelist>(null);
|
||||
final DruidMasterSegmentMerger merger = new DruidMasterSegmentMerger(indexingServiceClient, whitelistRef);
|
||||
final DruidMasterRuntimeParams params = DruidMasterRuntimeParams.newBuilder()
|
||||
.withAvailableSegments(ImmutableSet.copyOf(segments))
|
||||
.withMergeBytesLimit(mergeBytesLimit)
|
||||
|
|
|
@ -152,8 +152,6 @@ public class DruidMasterTest
|
|||
new NoopServiceEmitter(),
|
||||
scheduledExecutorFactory,
|
||||
loadManagementPeons,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue