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