Merge remote-tracking branch 'origin/master' into new_balancer

Conflicts:
	server/src/main/java/com/metamx/druid/master/DruidMaster.java
This commit is contained in:
Nelson Ray 2013-02-26 14:18:07 -08:00
commit e67a8757ba
157 changed files with 7194 additions and 2893 deletions

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.2.10-SNAPSHOT</version>
<version>0.3.4-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -139,10 +139,17 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
// build set of segments to query
Set<Pair<ServerSelector, SegmentDescriptor>> segments = Sets.newLinkedHashSet();
for (Interval interval : rewrittenQuery.getIntervals()) {
List<TimelineObjectHolder<String, ServerSelector>> serversLookup = timeline.lookup(interval);
List<TimelineObjectHolder<String, ServerSelector>> serversLookup = Lists.newLinkedList();
for (TimelineObjectHolder<String, ServerSelector> holder : serversLookup) {
for (Interval interval : rewrittenQuery.getIntervals()) {
serversLookup.addAll(timeline.lookup(interval));
}
// Let tool chest filter out unneeded segments
final List<TimelineObjectHolder<String, ServerSelector>> filteredServersLookup =
toolChest.filterSegments(query, serversLookup);
for (TimelineObjectHolder<String, ServerSelector> holder : filteredServersLookup) {
for (PartitionChunk<ServerSelector> chunk : holder.getObject()) {
ServerSelector selector = chunk.getObject();
final SegmentDescriptor descriptor = new SegmentDescriptor(
@ -152,7 +159,6 @@ public class CachingClusteredClient<T> implements QueryRunner<T>
segments.add(Pair.of(selector, descriptor));
}
}
}
final byte[] queryCacheKey;
if(strategy != null) {

View File

@ -19,7 +19,6 @@
package com.metamx.druid.client;
import com.google.common.collect.Maps;
import com.metamx.common.Pair;
import com.metamx.common.logger.Logger;
import com.metamx.phonebook.PhoneBook;
@ -27,12 +26,13 @@ import com.metamx.phonebook.PhoneBookPeon;
import java.util.Iterator;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
/**
*/
public class ServerInventoryManager extends InventoryManager<DruidServer>
{
private static final Map<String, Integer> removedSegments = Maps.newHashMap();
private static final Map<String, Integer> removedSegments = new ConcurrentHashMap<String, Integer>();
public ServerInventoryManager(
ServerInventoryManagerConfig config,

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Throwables;
import com.google.common.io.Closeables;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.metamx.common.ISE;
import com.metamx.common.concurrent.ScheduledExecutorFactory;
import com.metamx.common.lifecycle.Lifecycle;
import com.metamx.common.logger.Logger;
@ -31,8 +30,8 @@ import com.metamx.druid.client.ZKPhoneBook;
import com.metamx.druid.http.FileRequestLogger;
import com.metamx.druid.http.RequestLogger;
import com.metamx.druid.utils.PropUtils;
import com.metamx.druid.zk.StringZkSerializer;
import com.metamx.druid.zk.PropertiesZkSerializer;
import com.metamx.druid.zk.StringZkSerializer;
import com.netflix.curator.framework.CuratorFramework;
import com.netflix.curator.framework.CuratorFrameworkFactory;
import com.netflix.curator.retry.ExponentialBackoffRetry;
@ -42,7 +41,6 @@ import com.netflix.curator.x.discovery.ServiceInstance;
import com.netflix.curator.x.discovery.ServiceProvider;
import org.I0Itec.zkclient.ZkClient;
import org.I0Itec.zkclient.ZkConnection;
import org.mortbay.jetty.Connector;
import org.mortbay.jetty.Server;
import org.mortbay.jetty.nio.SelectChannelConnector;
@ -53,7 +51,6 @@ import java.io.IOException;
import java.io.InputStream;
import java.util.Properties;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
/**
*/

View File

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

View File

@ -22,18 +22,19 @@ package com.metamx.druid.query;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.base.Function;
import com.metamx.common.guava.Sequence;
import com.metamx.druid.LogicalSegment;
import com.metamx.druid.Query;
import com.metamx.emitter.service.ServiceMetricEvent;
import java.util.List;
/**
* The broker-side (also used by server in some cases) API for a specific Query type. This API is still undergoing
* evolution and is only semi-stable, so proprietary Query implementations should be ready for the potential
* maintenance burden when upgrading versions.
*/
public interface QueryToolChest<ResultType, QueryType extends Query<ResultType>>
public abstract class QueryToolChest<ResultType, QueryType extends Query<ResultType>>
{
public QueryRunner<ResultType> mergeResults(QueryRunner<ResultType> runner);
public abstract QueryRunner<ResultType> mergeResults(QueryRunner<ResultType> runner);
/**
* This method doesn't belong here, but it's here for now just to make it work.
@ -41,11 +42,24 @@ public interface QueryToolChest<ResultType, QueryType extends Query<ResultType>>
* @param seqOfSequences
* @return
*/
public Sequence<ResultType> mergeSequences(Sequence<Sequence<ResultType>> seqOfSequences);
public ServiceMetricEvent.Builder makeMetricBuilder(QueryType query);
public Function<ResultType, ResultType> makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn);
public TypeReference<ResultType> getResultTypeReference();
public <T> CacheStrategy<ResultType, T, QueryType> getCacheStrategy(QueryType query);
public QueryRunner<ResultType> preMergeQueryDecoration(QueryRunner<ResultType> runner);
public QueryRunner<ResultType> postMergeQueryDecoration(QueryRunner<ResultType> runner);
public abstract Sequence<ResultType> mergeSequences(Sequence<Sequence<ResultType>> seqOfSequences);
public abstract ServiceMetricEvent.Builder makeMetricBuilder(QueryType query);
public abstract Function<ResultType, ResultType> makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn);
public abstract TypeReference<ResultType> getResultTypeReference();
public <T> CacheStrategy<ResultType, T, QueryType> getCacheStrategy(QueryType query) {
return null;
}
public QueryRunner<ResultType> preMergeQueryDecoration(QueryRunner<ResultType> runner) {
return runner;
}
public QueryRunner<ResultType> postMergeQueryDecoration(QueryRunner<ResultType> runner) {
return runner;
}
public <T extends LogicalSegment> List<T> filterSegments(QueryType query, List<T> segments) {
return segments;
}
}

View File

@ -54,7 +54,7 @@ import java.util.Properties;
/**
*/
public class GroupByQueryQueryToolChest implements QueryToolChest<Row, GroupByQuery>
public class GroupByQueryQueryToolChest extends QueryToolChest<Row, GroupByQuery>
{
private static final TypeReference<Row> TYPE_REFERENCE = new TypeReference<Row>(){};
@ -177,22 +177,4 @@ public class GroupByQueryQueryToolChest implements QueryToolChest<Row, GroupByQu
{
return TYPE_REFERENCE;
}
@Override
public CacheStrategy<Row, Object, GroupByQuery> getCacheStrategy(GroupByQuery query)
{
return null;
}
@Override
public QueryRunner<Row> preMergeQueryDecoration(QueryRunner<Row> runner)
{
return runner;
}
@Override
public QueryRunner<Row> postMergeQueryDecoration(QueryRunner<Row> runner)
{
return runner;
}
}

View File

@ -51,7 +51,7 @@ import java.util.Map;
import java.util.Set;
public class SegmentMetadataQueryQueryToolChest implements QueryToolChest<SegmentAnalysis, SegmentMetadataQuery>
public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAnalysis, SegmentMetadataQuery>
{
private static final TypeReference<SegmentAnalysis> TYPE_REFERENCE = new TypeReference<SegmentAnalysis>(){};
private static final byte[] SEGMENT_METADATA_CACHE_PREFIX = new byte[]{0x4};
@ -220,18 +220,6 @@ public class SegmentMetadataQueryQueryToolChest implements QueryToolChest<Segmen
};
}
@Override
public QueryRunner<SegmentAnalysis> preMergeQueryDecoration(QueryRunner<SegmentAnalysis> runner)
{
return runner;
}
@Override
public QueryRunner<SegmentAnalysis> postMergeQueryDecoration(QueryRunner<SegmentAnalysis> runner)
{
return runner;
}
private Ordering<SegmentAnalysis> getOrdering()
{
return new Ordering<SegmentAnalysis>()

View File

@ -75,7 +75,7 @@ public class FragmentSearchQuerySpec implements SearchQuerySpec
public boolean accept(String dimVal)
{
for (String value : values) {
if (!dimVal.toLowerCase().contains(value)) {
if (dimVal == null || !dimVal.toLowerCase().contains(value)) {
return false;
}
}

View File

@ -62,6 +62,9 @@ public class InsensitiveContainsSearchQuerySpec implements SearchQuerySpec
@Override
public boolean accept(String dimVal)
{
if (dimVal == null) {
return false;
}
return dimVal.toLowerCase().contains(value);
}

View File

@ -65,7 +65,7 @@ import java.util.Set;
/**
*/
public class SearchQueryQueryToolChest implements QueryToolChest<Result<SearchResultValue>, SearchQuery>
public class SearchQueryQueryToolChest extends QueryToolChest<Result<SearchResultValue>, SearchQuery>
{
private static final byte SEARCH_QUERY = 0x2;
@ -263,12 +263,6 @@ public class SearchQueryQueryToolChest implements QueryToolChest<Result<SearchRe
);
}
@Override
public QueryRunner<Result<SearchResultValue>> postMergeQueryDecoration(final QueryRunner<Result<SearchResultValue>> runner)
{
return runner;
}
private static class SearchThresholdAdjustingQueryRunner implements QueryRunner<Result<SearchResultValue>>
{
private final QueryRunner<Result<SearchResultValue>> runner;

View File

@ -27,6 +27,7 @@ import com.google.common.collect.Ordering;
import com.metamx.common.guava.MergeSequence;
import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences;
import com.metamx.druid.LogicalSegment;
import com.metamx.druid.Query;
import com.metamx.druid.collect.OrderedMergeSequence;
import com.metamx.druid.query.BySegmentSkippingQueryRunner;
@ -37,7 +38,6 @@ import com.metamx.druid.query.QueryToolChest;
import com.metamx.druid.result.Result;
import com.metamx.druid.result.TimeBoundaryResultValue;
import com.metamx.emitter.service.ServiceMetricEvent;
import org.joda.time.DateTime;
import javax.annotation.Nullable;
@ -47,7 +47,7 @@ import java.util.List;
/**
*/
public class TimeBoundaryQueryQueryToolChest
implements QueryToolChest<Result<TimeBoundaryResultValue>, TimeBoundaryQuery>
extends QueryToolChest<Result<TimeBoundaryResultValue>, TimeBoundaryQuery>
{
private static final byte TIMEBOUNDARY_QUERY = 0x3;
@ -58,6 +58,16 @@ public class TimeBoundaryQueryQueryToolChest
{
};
@Override
public <T extends LogicalSegment> List<T> filterSegments(TimeBoundaryQuery query, List<T> input)
{
if(input.size() <= 1) {
return input;
}
return Lists.newArrayList(input.get(0), input.get(input.size() - 1));
}
@Override
public QueryRunner<Result<TimeBoundaryResultValue>> mergeResults(
final QueryRunner<Result<TimeBoundaryResultValue>> runner
@ -169,18 +179,6 @@ public class TimeBoundaryQueryQueryToolChest
};
}
@Override
public QueryRunner<Result<TimeBoundaryResultValue>> preMergeQueryDecoration(QueryRunner<Result<TimeBoundaryResultValue>> runner)
{
return runner;
}
@Override
public QueryRunner<Result<TimeBoundaryResultValue>> postMergeQueryDecoration(QueryRunner<Result<TimeBoundaryResultValue>> runner)
{
return runner;
}
public Ordering<Result<TimeBoundaryResultValue>> getOrdering()
{
return Ordering.natural();

View File

@ -61,7 +61,7 @@ import java.util.Map;
/**
*/
public class TimeseriesQueryQueryToolChest implements QueryToolChest<Result<TimeseriesResultValue>, TimeseriesQuery>
public class TimeseriesQueryQueryToolChest extends QueryToolChest<Result<TimeseriesResultValue>, TimeseriesQuery>
{
private static final byte TIMESERIES_QUERY = 0x0;
@ -259,12 +259,6 @@ public class TimeseriesQueryQueryToolChest implements QueryToolChest<Result<Time
return new IntervalChunkingQueryRunner<Result<TimeseriesResultValue>>(runner, Period.months(1));
}
@Override
public QueryRunner<Result<TimeseriesResultValue>> postMergeQueryDecoration(QueryRunner<Result<TimeseriesResultValue>> runner)
{
return runner;
}
public Ordering<Result<TimeseriesResultValue>> getOrdering()
{
return Ordering.natural();

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.2.10-SNAPSHOT</version>
<version>0.3.4-SNAPSHOT</version>
</parent>
<dependencies>
@ -79,6 +79,10 @@
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.datatype</groupId>
<artifactId>jackson-datatype-guava</artifactId>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.datatype</groupId>
<artifactId>jackson-datatype-joda</artifactId>

View File

@ -0,0 +1,8 @@
package com.metamx.druid;
import org.joda.time.Interval;
public interface LogicalSegment
{
public Interval getInterval();
}

View File

@ -24,7 +24,7 @@ import org.joda.time.Interval;
/**
*/
public class TimelineObjectHolder<VersionType, ObjectType>
public class TimelineObjectHolder<VersionType, ObjectType> implements LogicalSegment
{
private final Interval interval;
private final VersionType version;
@ -41,6 +41,7 @@ public class TimelineObjectHolder<VersionType, ObjectType>
this.object = object;
}
@Override
public Interval getInterval()
{
return interval;

View File

@ -36,6 +36,7 @@ import com.fasterxml.jackson.databind.SerializationFeature;
import com.fasterxml.jackson.databind.SerializerProvider;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.fasterxml.jackson.databind.ser.std.ToStringSerializer;
import com.fasterxml.jackson.datatype.guava.GuavaModule;
import com.fasterxml.jackson.datatype.joda.JodaModule;
import com.google.common.base.Throwables;
import com.metamx.common.Granularity;
@ -171,10 +172,11 @@ public class DefaultObjectMapper extends ObjectMapper
}
);
registerModule(serializerModule);
registerModule(new GuavaModule());
configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
configure(MapperFeature.AUTO_DETECT_GETTERS, false);
configure(MapperFeature.AUTO_DETECT_CREATORS, false);
// configure(MapperFeature.AUTO_DETECT_CREATORS, false); https://github.com/FasterXML/jackson-databind/issues/170
configure(MapperFeature.AUTO_DETECT_FIELDS, false);
configure(MapperFeature.AUTO_DETECT_IS_GETTERS, false);
configure(MapperFeature.AUTO_DETECT_SETTERS, false);

View File

@ -0,0 +1,126 @@
/*
* 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.utils;
import com.google.common.io.ByteStreams;
import com.google.common.io.Closeables;
import com.google.common.io.Files;
import com.metamx.common.ISE;
import com.metamx.common.StreamUtils;
import com.metamx.common.logger.Logger;
import sun.misc.IOUtils;
import java.io.BufferedInputStream;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileNotFoundException;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.util.zip.GZIPInputStream;
import java.util.zip.ZipEntry;
import java.util.zip.ZipInputStream;
import java.util.zip.ZipOutputStream;
/**
*/
public class CompressionUtils
{
private static final Logger log = new Logger(CompressionUtils.class);
public static long zip(File directory, File outputZipFile) throws IOException
{
if (!directory.isDirectory()) {
throw new IOException(String.format("directory[%s] is not a directory", directory));
}
if (!outputZipFile.getName().endsWith(".zip")) {
log.warn("No .zip suffix[%s], putting files from [%s] into it anyway.", outputZipFile, directory);
}
long totalSize = 0;
ZipOutputStream zipOut = null;
try {
zipOut = new ZipOutputStream(new FileOutputStream(outputZipFile));
File[] files = directory.listFiles();
for (File file : files) {
log.info("Adding file[%s] with size[%,d]. Total size[%,d]", file, file.length(), totalSize);
if (file.length() >= Integer.MAX_VALUE) {
zipOut.close();
outputZipFile.delete();
throw new IOException(String.format("file[%s] too large [%,d]", file, file.length()));
}
zipOut.putNextEntry(new ZipEntry(file.getName()));
totalSize += ByteStreams.copy(Files.newInputStreamSupplier(file), zipOut);
}
}
finally {
Closeables.closeQuietly(zipOut);
}
return totalSize;
}
public static void unzip(File pulledFile, File outDir) throws IOException
{
if (!(outDir.exists() && outDir.isDirectory())) {
throw new ISE("outDir[%s] must exist and be a directory", outDir);
}
log.info("Unzipping file[%s] to [%s]", pulledFile, outDir);
InputStream in = null;
try {
in = new BufferedInputStream(new FileInputStream(pulledFile));
unzip(in, outDir);
}
finally {
Closeables.closeQuietly(in);
}
}
public static void unzip(InputStream in, File outDir) throws IOException
{
ZipInputStream zipIn = new ZipInputStream(in);
ZipEntry entry;
while ((entry = zipIn.getNextEntry()) != null) {
OutputStream out = null;
try {
out = new FileOutputStream(new File(outDir, entry.getName()));
ByteStreams.copy(zipIn, out);
zipIn.closeEntry();
}
finally {
Closeables.closeQuietly(out);
}
}
}
public static void gunzip(File pulledFile, File outDir) throws IOException
{
log.info("Gunzipping file[%s] to [%s]", pulledFile, outDir);
StreamUtils.copyToFileAndClose(new GZIPInputStream(new FileInputStream(pulledFile)), outDir);
if (!pulledFile.delete()) {
log.error("Could not delete tmpFile[%s].", pulledFile);
}
}
}

View File

@ -19,12 +19,15 @@
package com.metamx.druid.utils;
import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.metamx.common.guava.Comparators;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.TreeSet;
@ -63,6 +66,17 @@ public class JodaUtils
return retVal;
}
public static boolean overlaps(final Interval i, Iterable<Interval> intervals) {
return Iterables.any(intervals, new Predicate<Interval>()
{
@Override
public boolean apply(@Nullable Interval input)
{
return input.overlaps(i);
}
});
}
public static DateTime minDateTime(DateTime... times)
{
if (times == null) {

View File

@ -24,11 +24,11 @@
<artifactId>druid-services</artifactId>
<name>druid-services</name>
<description>druid-services</description>
<version>0.2.10-SNAPSHOT</version>
<version>0.3.4-SNAPSHOT</version>
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.2.10-SNAPSHOT</version>
<version>0.3.4-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.2.10-SNAPSHOT</version>
<version>0.3.4-SNAPSHOT</version>
</parent>
<modules>

View File

@ -9,7 +9,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid-examples</artifactId>
<version>0.2.10-SNAPSHOT</version>
<version>0.3.4-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -12,7 +12,7 @@ import com.metamx.druid.log.LogLevelAdjuster;
import com.metamx.druid.realtime.MetadataUpdater;
import com.metamx.druid.realtime.MetadataUpdaterConfig;
import com.metamx.druid.realtime.RealtimeNode;
import com.metamx.druid.loading.SegmentPusher;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.phonebook.PhoneBook;
@ -72,8 +72,8 @@ public class RealtimeStandaloneMain
// dummyMetadataUpdater will not send updates to db because standalone demo has no db
rn.setMetadataUpdater(dummyMetadataUpdater);
rn.setSegmentPusher(
new SegmentPusher()
rn.setDataSegmentPusher(
new DataSegmentPusher()
{
@Override
public DataSegment push(File file, DataSegment segment) throws IOException

View File

@ -9,7 +9,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid-examples</artifactId>
<version>0.2.10-SNAPSHOT</version>
<version>0.3.4-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -12,7 +12,7 @@ import com.metamx.druid.log.LogLevelAdjuster;
import com.metamx.druid.realtime.MetadataUpdater;
import com.metamx.druid.realtime.MetadataUpdaterConfig;
import com.metamx.druid.realtime.RealtimeNode;
import com.metamx.druid.loading.SegmentPusher;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.phonebook.PhoneBook;
import druid.examples.twitter.TwitterSpritzerFirehoseFactory;
@ -48,10 +48,12 @@ public class RealtimeStandaloneMain
rn.setPhoneBook(dummyPhoneBook);
MetadataUpdater dummyMetadataUpdater =
new MetadataUpdater(new DefaultObjectMapper(),
new MetadataUpdater(
new DefaultObjectMapper(),
Config.createFactory(Initialization.loadProperties()).build(MetadataUpdaterConfig.class),
dummyPhoneBook,
null) {
null
) {
@Override
public void publishSegment(DataSegment segment) throws IOException
{
@ -74,8 +76,8 @@ public class RealtimeStandaloneMain
// dummyMetadataUpdater will not send updates to db because standalone demo has no db
rn.setMetadataUpdater(dummyMetadataUpdater);
rn.setSegmentPusher(
new SegmentPusher()
rn.setDataSegmentPusher(
new DataSegmentPusher()
{
@Override
public DataSegment push(File file, DataSegment segment) throws IOException

View File

@ -62,7 +62,7 @@ import static java.lang.Thread.sleep;
* Notes on JSON parsing: as of twitter4j 2.2.x, the json parser has some bugs (ex: Status.toString()
* can have number format exceptions), so it might be necessary to extract raw json and process it
* separately. If so, set twitter4.jsonStoreEnabled=true and look at DataObjectFactory#getRawJSON();
* org.codehaus.jackson.map.ObjectMapper should be used to parse.
* com.fasterxml.jackson.databind.ObjectMapper should be used to parse.
* @author pbaclace
*/
@JsonTypeName("twitzer")

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.2.10-SNAPSHOT</version>
<version>0.3.4-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -72,30 +72,30 @@ class SimpleColumn implements Column
@Override
public DictionaryEncodedColumn getDictionaryEncoding()
{
return dictionaryEncodedColumn.get();
return dictionaryEncodedColumn == null ? null : dictionaryEncodedColumn.get();
}
@Override
public RunLengthColumn getRunLengthColumn()
{
return runLengthColumn.get();
return runLengthColumn == null ? null : runLengthColumn.get();
}
@Override
public GenericColumn getGenericColumn()
{
return genericColumn.get();
return genericColumn == null ? null : genericColumn.get();
}
@Override
public ComplexColumn getComplexColumn()
{
return complexColumn.get();
return complexColumn == null ? null : complexColumn.get();
}
@Override
public BitmapIndex getBitmapIndex()
{
return bitmapIndex.get();
return bitmapIndex == null ? null : bitmapIndex.get();
}
}

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.2.10-SNAPSHOT</version>
<version>0.3.4-SNAPSHOT</version>
</parent>
<dependencies>
@ -57,14 +57,6 @@
<groupId>javax.mail</groupId>
<artifactId>mail</artifactId>
</exclusion>
<exclusion>
<groupId>org.codehaus.jackson</groupId>
<artifactId>jackson-core-asl</artifactId>
</exclusion>
<exclusion>
<groupId>org.codehaus.jackson</groupId>
<artifactId>jackson-mapper-asl</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>

View File

@ -266,8 +266,7 @@ public class DeterminePartitionsJob implements Jobby
Context context
) throws IOException, InterruptedException
{
// Create group key
// TODO -- There are more efficient ways to do this
// Create group key, there are probably more efficient ways of doing this
final Map<String, Set<String>> dims = Maps.newTreeMap();
for(final String dim : inputRow.getDimensions()) {
final Set<String> dimValues = ImmutableSortedSet.copyOf(inputRow.getDimension(dim));
@ -394,6 +393,9 @@ public class DeterminePartitionsJob implements Jobby
final Interval interval = maybeInterval.get();
final byte[] groupKey = interval.getStart().toString().getBytes(Charsets.UTF_8);
// Emit row-counter value.
write(context, groupKey, new DimValueCount("", "", 1));
for(final Map.Entry<String, Iterable<String>> dimAndValues : dims.entrySet()) {
final String dim = dimAndValues.getKey();
@ -510,9 +512,23 @@ public class DeterminePartitionsJob implements Jobby
Context context, SortableBytes keyBytes, Iterable<DimValueCount> combinedIterable
) throws IOException, InterruptedException
{
PeekingIterator<DimValueCount> iterator = Iterators.peekingIterator(combinedIterable.iterator());
final DateTime bucket = new DateTime(new String(keyBytes.getGroupKey(), Charsets.UTF_8));
final PeekingIterator<DimValueCount> iterator = Iterators.peekingIterator(combinedIterable.iterator());
// "iterator" will take us over many candidate dimensions
log.info(
"Determining partitions for interval: %s",
config.getGranularitySpec().bucketInterval(bucket).orNull()
);
// First DVC should be the total row count indicator
final DimValueCount firstDvc = iterator.next();
final int totalRows = firstDvc.numRows;
if(!firstDvc.dim.equals("") || !firstDvc.value.equals("")) {
throw new IllegalStateException("WTF?! Expected total row indicator on first k/v pair!");
}
// "iterator" will now take us over many candidate dimensions
DimPartitions currentDimPartitions = null;
DimPartition currentDimPartition = null;
String currentDimPartitionStart = null;
@ -636,8 +652,6 @@ public class DeterminePartitionsJob implements Jobby
throw new ISE("No suitable partitioning dimension found!");
}
final int totalRows = dimPartitionss.values().iterator().next().getRows();
int maxCardinality = Integer.MIN_VALUE;
long minVariance = Long.MAX_VALUE;
DimPartitions minVariancePartitions = null;
@ -645,12 +659,14 @@ public class DeterminePartitionsJob implements Jobby
for(final DimPartitions dimPartitions : dimPartitionss.values()) {
if(dimPartitions.getRows() != totalRows) {
throw new ISE(
"WTF?! Dimension[%s] row count %,d != expected row count %,d",
log.info(
"Dimension[%s] is not present in all rows (row count %,d != expected row count %,d)",
dimPartitions.dim,
dimPartitions.getRows(),
totalRows
);
continue;
}
// Make sure none of these shards are oversized
@ -684,7 +700,6 @@ public class DeterminePartitionsJob implements Jobby
throw new ISE("No suitable partitioning dimension found!");
}
final DateTime bucket = new DateTime(new String(keyBytes.getGroupKey(), Charsets.UTF_8));
final OutputStream out = Utils.makePathAndOutputStream(
context, config.makeSegmentPartitionInfoPath(new Bucket(0, bucket, 0)), config.isOverwriteFiles()
);

View File

@ -662,8 +662,9 @@ public class HadoopDruidIndexerConfig
return new Path(
String.format(
"%s/%s_%s/%s/%s",
"%s/%s/%s_%s/%s/%s",
getSegmentOutputDir(),
dataSource,
bucketInterval.getStart().toString(),
bucketInterval.getEnd().toString(),
getVersion().toString(),

View File

@ -379,7 +379,8 @@ public class IndexGeneratorJob implements Jobby
);
} else if (outputFS instanceof LocalFileSystem) {
loadSpec = ImmutableMap.<String, Object>of(
"type", "test"
"type", "local",
"path", indexOutURI.getPath()
);
} else {
throw new ISE("Unknown file system[%s]", outputFS.getClass());

View File

@ -28,7 +28,7 @@
<parent>
<groupId>com.metamx</groupId>
<artifactId>druid</artifactId>
<version>0.2.10-SNAPSHOT</version>
<version>0.3.4-SNAPSHOT</version>
</parent>
<dependencies>

View File

@ -17,9 +17,7 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.merger.coordinator;
import com.metamx.druid.merger.common.TaskStatus;
package com.metamx.druid.merger.common;
public interface TaskCallback
{

View File

@ -1,57 +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;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.TaskContext;
/**
*/
public class TaskHolder
{
private final Task task;
private final TaskContext taskContext;
@JsonCreator
public TaskHolder(
@JsonProperty("task") Task task,
@JsonProperty("taskContext") TaskContext taskContext
)
{
this.task = task;
this.taskContext = taskContext;
}
@JsonProperty
public Task getTask()
{
return task;
}
@JsonProperty
public TaskContext getTaskContext()
{
return taskContext;
}
}

View File

@ -17,40 +17,30 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.merger.coordinator;
package com.metamx.druid.merger.common;
import com.google.common.base.Function;
import com.google.common.base.Objects;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Ordering;
import com.metamx.druid.merger.common.task.Task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.joda.time.Interval;
import java.util.Set;
import java.util.TreeSet;
/**
* Represents a transaction as well as the lock it holds. Not immutable: the task set can change.
* Represents a lock held by some task. Immutable.
*/
public class TaskGroup
public class TaskLock
{
private final String groupId;
private final String dataSource;
private final Interval interval;
private final String version;
private final Set<Task> taskSet = new TreeSet<Task>(
new Ordering<Task>()
{
@Override
public int compare(Task task, Task task1)
{
return task.getId().compareTo(task1.getId());
}
}.nullsFirst()
);
public TaskGroup(String groupId, String dataSource, Interval interval, String version)
@JsonCreator
public TaskLock(
@JsonProperty("groupId") String groupId,
@JsonProperty("dataSource") String dataSource,
@JsonProperty("interval") Interval interval,
@JsonProperty("version") String version
)
{
this.groupId = groupId;
this.dataSource = dataSource;
@ -58,29 +48,48 @@ public class TaskGroup
this.version = version;
}
@JsonProperty
public String getGroupId()
{
return groupId;
}
@JsonProperty
public String getDataSource()
{
return dataSource;
}
@JsonProperty
public Interval getInterval()
{
return interval;
}
@JsonProperty
public String getVersion()
{
return version;
}
public Set<Task> getTaskSet()
@Override
public boolean equals(Object o)
{
return taskSet;
if (!(o instanceof TaskLock)) {
return false;
} else {
final TaskLock x = (TaskLock) o;
return Objects.equal(this.groupId, x.groupId) &&
Objects.equal(this.dataSource, x.dataSource) &&
Objects.equal(this.interval, x.interval) &&
Objects.equal(this.version, x.version);
}
}
@Override
public int hashCode()
{
return Objects.hashCode(groupId, dataSource, interval, version);
}
@Override
@ -91,21 +100,6 @@ public class TaskGroup
.add("dataSource", dataSource)
.add("interval", interval)
.add("version", version)
.add(
"taskSet",
Lists.newArrayList(
Iterables.transform(
taskSet, new Function<Task, Object>()
{
@Override
public Object apply(Task task)
{
return task.getId();
}
}
)
)
)
.toString();
}
}

View File

@ -20,64 +20,42 @@
package com.metamx.druid.merger.common;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Objects;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merger.common.task.Task;
import java.util.Collections;
import java.util.List;
/**
* Represents the status of a task. The task may be ongoing ({@link #isComplete()} false) or it may be
* complete ({@link #isComplete()} true).
* <p/>
* TaskStatus objects are immutable.
*/
public class TaskStatus
{
public static enum Status
{
RUNNING,
SUCCESS,
FAILED,
CONTINUED
FAILED
}
public static TaskStatus running(String taskId)
{
return new TaskStatus(
taskId,
Status.RUNNING,
Collections.<DataSegment>emptyList(),
Collections.<Task>emptyList(),
-1
);
return new TaskStatus(taskId, Status.RUNNING, -1);
}
public static TaskStatus success(String taskId)
{
return new TaskStatus(taskId, Status.SUCCESS, -1);
}
public static TaskStatus failure(String taskId)
{
return new TaskStatus(taskId, Status.FAILED, Collections.<DataSegment>emptyList(), Collections.<Task>emptyList(), -1);
}
public static TaskStatus success(String taskId, List<DataSegment> segments)
{
return new TaskStatus(taskId, Status.SUCCESS, ImmutableList.copyOf(segments), Collections.<Task>emptyList(), -1);
}
public static TaskStatus continued(String taskId, List<Task> nextTasks)
{
Preconditions.checkArgument(nextTasks.size() > 0, "nextTasks.size() > 0");
return new TaskStatus(
taskId,
Status.CONTINUED,
Collections.<DataSegment>emptyList(),
ImmutableList.copyOf(nextTasks),
-1
);
return new TaskStatus(taskId, Status.FAILED, -1);
}
private final String id;
private final List<DataSegment> segments;
private final List<Task> nextTasks;
private final Status status;
private final long duration;
@ -85,16 +63,16 @@ public class TaskStatus
private TaskStatus(
@JsonProperty("id") String id,
@JsonProperty("status") Status status,
@JsonProperty("segments") List<DataSegment> segments,
@JsonProperty("nextTasks") List<Task> nextTasks,
@JsonProperty("duration") long duration
)
{
this.id = id;
this.segments = segments;
this.nextTasks = nextTasks;
this.status = status;
this.duration = duration;
// Check class invariants.
Preconditions.checkNotNull(id, "id");
Preconditions.checkNotNull(status, "status");
}
@JsonProperty("id")
@ -109,18 +87,6 @@ public class TaskStatus
return status;
}
@JsonProperty("segments")
public List<DataSegment> getSegments()
{
return segments;
}
@JsonProperty("nextTasks")
public List<Task> getNextTasks()
{
return nextTasks;
}
@JsonProperty("duration")
public long getDuration()
{
@ -129,43 +95,38 @@ public class TaskStatus
/**
* Signals that a task is not yet complete, and is still runnable on a worker. Exactly one of isRunnable,
* isContinued, isSuccess, or isFailure will be true at any one time.
* isSuccess, or isFailure will be true at any one time.
*/
@JsonIgnore
public boolean isRunnable()
{
return status == Status.RUNNING;
}
/**
* Returned by tasks when they complete successfully without spawning subtasks. Exactly one of isRunnable,
* isContinued, isSuccess, or isFailure will be true at any one time.
*/
public boolean isContinued()
{
return status == Status.CONTINUED;
}
/**
* Inverse of {@link #isRunnable}.
*/
@JsonIgnore
public boolean isComplete()
{
return !isRunnable();
}
/**
* Returned by tasks when they spawn subtasks. Exactly one of isRunnable, isContinued, isSuccess, or isFailure will
* Returned by tasks when they spawn subtasks. Exactly one of isRunnable, isSuccess, or isFailure will
* be true at any one time.
*/
@JsonIgnore
public boolean isSuccess()
{
return status == Status.SUCCESS;
}
/**
* Returned by tasks when they complete unsuccessfully. Exactly one of isRunnable, isContinued, isSuccess, or
* Returned by tasks when they complete unsuccessfully. Exactly one of isRunnable, isSuccess, or
* isFailure will be true at any one time.
*/
@JsonIgnore
public boolean isFailure()
{
return status == Status.FAILED;
@ -173,7 +134,7 @@ public class TaskStatus
public TaskStatus withDuration(long _duration)
{
return new TaskStatus(id, status, segments, nextTasks, _duration);
return new TaskStatus(id, status, _duration);
}
@Override
@ -181,8 +142,6 @@ public class TaskStatus
{
return Objects.toStringHelper(this)
.add("id", id)
.add("segments", segments)
.add("nextTasks", nextTasks)
.add("status", status)
.add("duration", duration)
.toString();

View File

@ -20,19 +20,23 @@
package com.metamx.druid.merger.common;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import com.metamx.druid.loading.S3SegmentPuller;
import com.metamx.druid.loading.S3SegmentGetterConfig;
import com.metamx.druid.loading.S3ZippedSegmentPuller;
import com.metamx.druid.loading.SegmentPuller;
import com.google.common.collect.Maps;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.loading.MMappedQueryableIndexFactory;
import com.metamx.druid.loading.S3DataSegmentPuller;
import com.metamx.druid.loading.SegmentKiller;
import com.metamx.druid.loading.SegmentLoaderConfig;
import com.metamx.druid.loading.SegmentLoadingException;
import com.metamx.druid.loading.SingleSegmentLoader;
import com.metamx.druid.merger.common.actions.TaskActionClient;
import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
import com.metamx.druid.loading.SegmentPusher;
import com.metamx.emitter.service.ServiceEmitter;
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
import java.io.File;
import java.util.List;
import java.util.Map;
/**
@ -40,67 +44,84 @@ import java.util.Map;
*/
public class TaskToolbox
{
private final IndexerCoordinatorConfig config;
private final TaskConfig config;
private final TaskActionClient taskActionClient;
private final ServiceEmitter emitter;
private final RestS3Service s3Client;
private final SegmentPusher segmentPusher;
private final DataSegmentPusher segmentPusher;
private final SegmentKiller segmentKiller;
private final ObjectMapper objectMapper;
public TaskToolbox(
IndexerCoordinatorConfig config,
TaskConfig config,
TaskActionClient taskActionClient,
ServiceEmitter emitter,
RestS3Service s3Client,
SegmentPusher segmentPusher,
DataSegmentPusher segmentPusher,
SegmentKiller segmentKiller,
ObjectMapper objectMapper
)
{
this.config = config;
this.taskActionClient = taskActionClient;
this.emitter = emitter;
this.s3Client = s3Client;
this.segmentPusher = segmentPusher;
this.segmentKiller = segmentKiller;
this.objectMapper = objectMapper;
}
public IndexerCoordinatorConfig getConfig()
public TaskConfig getConfig()
{
return config;
}
public TaskActionClient getTaskActionClient()
{
return taskActionClient;
}
public ServiceEmitter getEmitter()
{
return emitter;
}
public RestS3Service getS3Client()
{
return s3Client;
}
public SegmentPusher getSegmentPusher()
public DataSegmentPusher getSegmentPusher()
{
return segmentPusher;
}
public SegmentKiller getSegmentKiller()
{
return segmentKiller;
}
public ObjectMapper getObjectMapper()
{
return objectMapper;
}
public Map<String, SegmentPuller> getSegmentGetters(final Task task)
public Map<DataSegment, File> getSegments(final Task task, List<DataSegment> segments)
throws SegmentLoadingException
{
final S3SegmentGetterConfig getterConfig = new S3SegmentGetterConfig()
final SingleSegmentLoader loader = new SingleSegmentLoader(
new S3DataSegmentPuller(s3Client),
new MMappedQueryableIndexFactory(),
new SegmentLoaderConfig()
{
@Override
public File getCacheDirectory()
{
return new File(config.getTaskDir(task), "fetched_segments");
}
};
}
);
return ImmutableMap.<String, SegmentPuller>builder()
.put("s3", new S3SegmentPuller(s3Client, getterConfig))
.put("s3_union", new S3SegmentPuller(s3Client, getterConfig))
.put("s3_zip", new S3ZippedSegmentPuller(s3Client, getterConfig))
.build();
Map<DataSegment, File> retVal = Maps.newLinkedHashMap();
for (DataSegment segment : segments) {
retVal.put(segment, loader.getSegmentFiles(segment));
}
return retVal;
}
}

View File

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

View File

@ -0,0 +1,53 @@
package com.metamx.druid.merger.common.actions;
import com.google.common.base.Optional;
import com.google.common.base.Throwables;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.task.Task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import org.joda.time.Interval;
public class LockAcquireAction implements TaskAction<Optional<TaskLock>>
{
private final Task task;
private final Interval interval;
@JsonCreator
public LockAcquireAction(
@JsonProperty("task") Task task,
@JsonProperty("interval") Interval interval
)
{
this.task = task;
this.interval = interval;
}
@JsonProperty
public Task getTask()
{
return task;
}
@JsonProperty
public Interval getInterval()
{
return interval;
}
public TypeReference<Optional<TaskLock>> getReturnTypeReference()
{
return new TypeReference<Optional<TaskLock>>() {};
}
@Override
public Optional<TaskLock> perform(TaskActionToolbox toolbox)
{
try {
return toolbox.getTaskLockbox().tryLock(task, interval);
} catch (Exception e) {
throw Throwables.propagate(e);
}
}
}

View File

@ -0,0 +1,45 @@
package com.metamx.druid.merger.common.actions;
import com.google.common.base.Optional;
import com.google.common.base.Throwables;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.task.Task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import java.util.List;
public class LockListAction implements TaskAction<List<TaskLock>>
{
private final Task task;
@JsonCreator
public LockListAction(
@JsonProperty("task") Task task
)
{
this.task = task;
}
@JsonProperty
public Task getTask()
{
return task;
}
public TypeReference<List<TaskLock>> getReturnTypeReference()
{
return new TypeReference<List<TaskLock>>() {};
}
@Override
public List<TaskLock> perform(TaskActionToolbox toolbox)
{
try {
return toolbox.getTaskLockbox().findLocksForTask(task);
} catch (Exception e) {
throw Throwables.propagate(e);
}
}
}

View File

@ -0,0 +1,55 @@
package com.metamx.druid.merger.common.actions;
import com.google.common.base.Throwables;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.task.Task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import org.joda.time.Interval;
import java.util.List;
public class LockReleaseAction implements TaskAction<Void>
{
private final Task task;
private final Interval interval;
@JsonCreator
public LockReleaseAction(
@JsonProperty("task") Task task,
@JsonProperty("interval") Interval interval
)
{
this.task = task;
this.interval = interval;
}
@JsonProperty
public Task getTask()
{
return task;
}
@JsonProperty
public Interval getInterval()
{
return interval;
}
public TypeReference<Void> getReturnTypeReference()
{
return new TypeReference<Void>() {};
}
@Override
public Void perform(TaskActionToolbox toolbox)
{
try {
toolbox.getTaskLockbox().unlock(task, interval);
return null;
} catch (Exception e) {
throw Throwables.propagate(e);
}
}
}

View File

@ -0,0 +1,75 @@
package com.metamx.druid.merger.common.actions;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.base.Charsets;
import com.google.common.base.Throwables;
import com.metamx.common.logger.Logger;
import com.metamx.http.client.HttpClient;
import com.metamx.http.client.response.ToStringResponseHandler;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.netflix.curator.x.discovery.ServiceInstance;
import com.netflix.curator.x.discovery.ServiceProvider;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.Map;
public class RemoteTaskActionClient implements TaskActionClient
{
private final HttpClient httpClient;
private final ServiceProvider serviceProvider;
private final ObjectMapper jsonMapper;
private static final Logger log = new Logger(RemoteTaskActionClient.class);
public RemoteTaskActionClient(HttpClient httpClient, ServiceProvider serviceProvider, ObjectMapper jsonMapper)
{
this.httpClient = httpClient;
this.serviceProvider = serviceProvider;
this.jsonMapper = jsonMapper;
}
@Override
public <RetType> RetType submit(TaskAction<RetType> taskAction)
{
try {
byte[] dataToSend = jsonMapper.writeValueAsBytes(taskAction);
final String response = httpClient.post(getServiceUri().toURL())
.setContent("application/json", dataToSend)
.go(new ToStringResponseHandler(Charsets.UTF_8))
.get();
final Map<String, Object> responseDict = jsonMapper.readValue(
response,
new TypeReference<Map<String, Object>>() {}
);
return jsonMapper.convertValue(responseDict.get("result"), taskAction.getReturnTypeReference());
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
private URI getServiceUri() throws Exception
{
final ServiceInstance instance = serviceProvider.getInstance();
final String scheme;
final String host;
final int port;
final String path = "/mmx/merger/v1/action";
host = instance.getAddress();
if (instance.getSslPort() != null && instance.getSslPort() > 0) {
scheme = "https";
port = instance.getSslPort();
} else {
scheme = "http";
port = instance.getPort();
}
return new URI(scheme, null, host, port, path, null, null);
}
}

View File

@ -0,0 +1,76 @@
package com.metamx.druid.merger.common.actions;
import com.google.common.base.Predicate;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.metamx.common.ISE;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.emitter.service.ServiceMetricEvent;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import java.util.List;
import java.util.Set;
public class SegmentInsertAction implements TaskAction<Void>
{
private final Task task;
private final Set<DataSegment> segments;
@JsonCreator
public SegmentInsertAction(
@JsonProperty("task") Task task,
@JsonProperty("segments") Set<DataSegment> segments
)
{
this.task = task;
this.segments = ImmutableSet.copyOf(segments);
}
@JsonProperty
public Task getTask()
{
return task;
}
@JsonProperty
public Set<DataSegment> getSegments()
{
return segments;
}
public TypeReference<Void> getReturnTypeReference()
{
return new TypeReference<Void>() {};
}
@Override
public Void perform(TaskActionToolbox toolbox)
{
if(!toolbox.taskLockCoversSegments(task, segments, false)) {
throw new ISE("Segments not covered by locks for task: %s", task.getId());
}
try {
toolbox.getMergerDBCoordinator().announceHistoricalSegments(segments);
// Emit metrics
final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder()
.setUser2(task.getDataSource())
.setUser4(task.getType());
for (DataSegment segment : segments) {
metricBuilder.setUser5(segment.getInterval().toString());
toolbox.getEmitter().emit(metricBuilder.build("indexer/segment/bytes", segment.getSize()));
}
return null;
} catch (Exception e) {
throw Throwables.propagate(e);
}
}
}

View File

@ -0,0 +1,63 @@
package com.metamx.druid.merger.common.actions;
import com.google.common.base.Throwables;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merger.common.task.Task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import org.joda.time.Interval;
import java.util.List;
public class SegmentListUnusedAction implements TaskAction<List<DataSegment>>
{
private final Task task;
private final String dataSource;
private final Interval interval;
@JsonCreator
public SegmentListUnusedAction(
@JsonProperty("task") Task task,
@JsonProperty("dataSource") String dataSource,
@JsonProperty("interval") Interval interval
)
{
this.task = task;
this.dataSource = dataSource;
this.interval = interval;
}
@JsonProperty
public Task getTask()
{
return task;
}
@JsonProperty
public String getDataSource()
{
return dataSource;
}
@JsonProperty
public Interval getInterval()
{
return interval;
}
public TypeReference<List<DataSegment>> getReturnTypeReference()
{
return new TypeReference<List<DataSegment>>() {};
}
@Override
public List<DataSegment> perform(TaskActionToolbox toolbox)
{
try {
return toolbox.getMergerDBCoordinator().getUnusedSegmentsForInterval(dataSource, interval);
} catch (Exception e) {
throw Throwables.propagate(e);
}
}
}

View File

@ -0,0 +1,63 @@
package com.metamx.druid.merger.common.actions;
import com.google.common.base.Throwables;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merger.common.task.Task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import org.joda.time.Interval;
import java.util.List;
public class SegmentListUsedAction implements TaskAction<List<DataSegment>>
{
private final Task task;
private final String dataSource;
private final Interval interval;
@JsonCreator
public SegmentListUsedAction(
@JsonProperty("task") Task task,
@JsonProperty("dataSource") String dataSource,
@JsonProperty("interval") Interval interval
)
{
this.task = task;
this.dataSource = dataSource;
this.interval = interval;
}
@JsonProperty
public Task getTask()
{
return task;
}
@JsonProperty
public String getDataSource()
{
return dataSource;
}
@JsonProperty
public Interval getInterval()
{
return interval;
}
public TypeReference<List<DataSegment>> getReturnTypeReference()
{
return new TypeReference<List<DataSegment>>() {};
}
@Override
public List<DataSegment> perform(TaskActionToolbox toolbox)
{
try {
return toolbox.getMergerDBCoordinator().getUsedSegmentsForInterval(dataSource, interval);
} catch (Exception e) {
throw Throwables.propagate(e);
}
}
}

View File

@ -0,0 +1,76 @@
package com.metamx.druid.merger.common.actions;
import com.google.common.base.Predicate;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.metamx.common.ISE;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.emitter.service.ServiceMetricEvent;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import java.util.List;
import java.util.Set;
public class SegmentNukeAction implements TaskAction<Void>
{
private final Task task;
private final Set<DataSegment> segments;
@JsonCreator
public SegmentNukeAction(
@JsonProperty("task") Task task,
@JsonProperty("segments") Set<DataSegment> segments
)
{
this.task = task;
this.segments = ImmutableSet.copyOf(segments);
}
@JsonProperty
public Task getTask()
{
return task;
}
@JsonProperty
public Set<DataSegment> getSegments()
{
return segments;
}
public TypeReference<Void> getReturnTypeReference()
{
return new TypeReference<Void>() {};
}
@Override
public Void perform(TaskActionToolbox toolbox)
{
if(!toolbox.taskLockCoversSegments(task, segments, true)) {
throw new ISE("Segments not covered by locks for task: %s", task.getId());
}
try {
toolbox.getMergerDBCoordinator().deleteSegments(segments);
// Emit metrics
final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder()
.setUser2(task.getDataSource())
.setUser4(task.getType());
for (DataSegment segment : segments) {
metricBuilder.setUser5(segment.getInterval().toString());
toolbox.getEmitter().emit(metricBuilder.build("indexer/segmentNuked/bytes", segment.getSize()));
}
return null;
} catch (Exception e) {
throw Throwables.propagate(e);
}
}
}

View File

@ -0,0 +1,57 @@
package com.metamx.druid.merger.common.actions;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.metamx.druid.merger.common.task.Task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import java.util.List;
public class SpawnTasksAction implements TaskAction<Void>
{
private final Task task;
private final List<Task> newTasks;
@JsonCreator
public SpawnTasksAction(
@JsonProperty("task") Task task,
@JsonProperty("newTasks") List<Task> newTasks
)
{
this.task = task;
this.newTasks = ImmutableList.copyOf(newTasks);
}
@JsonProperty
public Task getTask()
{
return task;
}
@JsonProperty
public List<Task> getNewTasks()
{
return newTasks;
}
public TypeReference<Void> getReturnTypeReference()
{
return new TypeReference<Void>() {};
}
@Override
public Void perform(TaskActionToolbox toolbox)
{
try {
for(final Task newTask : newTasks) {
toolbox.getTaskQueue().add(newTask);
}
return null;
} catch (Exception e) {
throw Throwables.propagate(e);
}
}
}

View File

@ -0,0 +1,24 @@
package com.metamx.druid.merger.common.actions;
import com.metamx.druid.merger.common.task.Task;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.fasterxml.jackson.core.type.TypeReference;
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "lockAcquire", value = LockAcquireAction.class),
@JsonSubTypes.Type(name = "lockList", value = LockListAction.class),
@JsonSubTypes.Type(name = "lockRelease", value = LockReleaseAction.class),
@JsonSubTypes.Type(name = "segmentInsertion", value = SegmentInsertAction.class),
@JsonSubTypes.Type(name = "segmentListUsed", value = SegmentListUsedAction.class),
@JsonSubTypes.Type(name = "segmentListUnused", value = SegmentListUnusedAction.class),
@JsonSubTypes.Type(name = "segmentNuke", value = SegmentNukeAction.class),
@JsonSubTypes.Type(name = "spawnTasks", value = SpawnTasksAction.class)
})
public interface TaskAction<RetType>
{
public Task getTask(); // TODO Look into replacing this with task ID so stuff serializes smaller
public TypeReference<RetType> getReturnTypeReference(); // T_T
public RetType perform(TaskActionToolbox toolbox);
}

View File

@ -0,0 +1,6 @@
package com.metamx.druid.merger.common.actions;
public interface TaskActionClient
{
public <RetType> RetType submit(TaskAction<RetType> taskAction);
}

View File

@ -0,0 +1,95 @@
package com.metamx.druid.merger.common.actions;
import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
import com.metamx.common.ISE;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.MergerDBCoordinator;
import com.metamx.druid.merger.coordinator.TaskLockbox;
import com.metamx.druid.merger.coordinator.TaskQueue;
import com.metamx.emitter.service.ServiceEmitter;
import java.util.List;
import java.util.Set;
public class TaskActionToolbox
{
private final TaskQueue taskQueue;
private final TaskLockbox taskLockbox;
private final MergerDBCoordinator mergerDBCoordinator;
private final ServiceEmitter emitter;
public TaskActionToolbox(
TaskQueue taskQueue,
TaskLockbox taskLockbox,
MergerDBCoordinator mergerDBCoordinator,
ServiceEmitter emitter
)
{
this.taskQueue = taskQueue;
this.taskLockbox = taskLockbox;
this.mergerDBCoordinator = mergerDBCoordinator;
this.emitter = emitter;
}
public TaskQueue getTaskQueue()
{
return taskQueue;
}
public TaskLockbox getTaskLockbox()
{
return taskLockbox;
}
public MergerDBCoordinator getMergerDBCoordinator()
{
return mergerDBCoordinator;
}
public ServiceEmitter getEmitter()
{
return emitter;
}
public boolean taskLockCoversSegments(
final Task task,
final Set<DataSegment> segments,
final boolean allowOlderVersions
)
{
// Verify that each of these segments falls under some lock
// NOTE: It is possible for our lock to be revoked (if the task has failed and given up its locks) after we check
// NOTE: it and before we perform the segment insert, but, that should be OK since the worst that happens is we
// NOTE: insert some segments from the task but not others.
final List<TaskLock> taskLocks = getTaskLockbox().findLocksForTask(task);
for(final DataSegment segment : segments) {
final boolean ok = Iterables.any(
taskLocks, new Predicate<TaskLock>()
{
@Override
public boolean apply(TaskLock taskLock)
{
final boolean versionOk = allowOlderVersions
? taskLock.getVersion().compareTo(segment.getVersion()) >= 0
: taskLock.getVersion().equals(segment.getVersion());
return versionOk
&& taskLock.getDataSource().equals(segment.getDataSource())
&& taskLock.getInterval().contains(segment.getInterval());
}
}
);
if (!ok) {
return false;
}
}
return true;
}
}

View File

@ -0,0 +1,21 @@
package com.metamx.druid.merger.common.config;
import com.metamx.druid.merger.common.task.Task;
import org.skife.config.Config;
import org.skife.config.Default;
import java.io.File;
public abstract class TaskConfig
{
@Config("druid.merger.taskDir")
public abstract File getBaseTaskDir();
@Config("druid.merger.rowFlushBoundary")
@Default("500000")
public abstract int getDefaultRowFlushBoundary();
public File getTaskDir(final Task task) {
return new File(getBaseTaskDir(), task.getId());
}
}

View File

@ -35,7 +35,7 @@ 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 com.metamx.druid.loading.SegmentPusher;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.query.QueryRunner;
import com.metamx.druid.realtime.FireDepartmentMetrics;
import com.metamx.druid.realtime.FireHydrant;
@ -45,11 +45,11 @@ import com.metamx.druid.realtime.Schema;
import com.metamx.druid.realtime.Sink;
import org.apache.commons.io.FileUtils;
import org.joda.time.Interval;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Set;
@ -61,7 +61,7 @@ public class YeOldePlumberSchool implements PlumberSchool
{
private final Interval interval;
private final String version;
private final SegmentPusher segmentPusher;
private final DataSegmentPusher dataSegmentPusher;
private final File tmpSegmentDir;
private static final Logger log = new Logger(YeOldePlumberSchool.class);
@ -70,13 +70,13 @@ public class YeOldePlumberSchool implements PlumberSchool
public YeOldePlumberSchool(
@JsonProperty("interval") Interval interval,
@JsonProperty("version") String version,
@JacksonInject("segmentPusher") SegmentPusher segmentPusher,
@JacksonInject("segmentPusher") DataSegmentPusher dataSegmentPusher,
@JacksonInject("tmpSegmentDir") File tmpSegmentDir
)
{
this.interval = interval;
this.version = version;
this.segmentPusher = segmentPusher;
this.dataSegmentPusher = dataSegmentPusher;
this.tmpSegmentDir = tmpSegmentDir;
}
@ -120,13 +120,13 @@ public class YeOldePlumberSchool implements PlumberSchool
@Override
public void finishJob()
{
// The segment we will upload
File fileToUpload = null;
try {
// User should have persisted everything by now.
Preconditions.checkState(!theSink.swappable(), "All data must be persisted before fininshing the job!");
// The segment we will upload
final File fileToUpload;
if(spilled.size() == 0) {
throw new IllegalStateException("Nothing indexed?");
} else if(spilled.size() == 1) {
@ -149,7 +149,7 @@ public class YeOldePlumberSchool implements PlumberSchool
.withVersion(version)
.withBinaryVersion(IndexIO.getVersionFromDir(fileToUpload));
segmentPusher.push(fileToUpload, segmentToUpload);
dataSegmentPusher.push(fileToUpload, segmentToUpload);
log.info(
"Uploaded segment[%s]",
@ -160,6 +160,17 @@ public class YeOldePlumberSchool implements PlumberSchool
log.warn(e, "Failed to merge and upload");
throw Throwables.propagate(e);
}
finally {
try {
if (fileToUpload != null) {
log.info("Deleting Index File[%s]", fileToUpload);
FileUtils.deleteDirectory(fileToUpload);
}
}
catch (IOException e) {
log.warn(e, "Error deleting directory[%s]", fileToUpload);
}
}
}
private void spillIfSwappable()

View File

@ -22,11 +22,10 @@ package com.metamx.druid.merger.common.task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Objects;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.coordinator.TaskContext;
import com.metamx.druid.merger.common.TaskToolbox;
import org.joda.time.Interval;
public abstract class AbstractTask implements Task
@ -34,30 +33,19 @@ public abstract class AbstractTask implements Task
private final String id;
private final String groupId;
private final String dataSource;
private final Interval interval;
private final Optional<Interval> interval;
public AbstractTask(String id, String dataSource, Interval interval)
protected AbstractTask(String id, String dataSource, Interval interval)
{
this(id, id, dataSource, interval);
}
@JsonCreator
public AbstractTask(
@JsonProperty("id") String id,
@JsonProperty("groupId") String groupId,
@JsonProperty("dataSource") String dataSource,
@JsonProperty("interval") Interval interval
)
protected AbstractTask(String id, String groupId, String dataSource, Interval interval)
{
Preconditions.checkNotNull(id, "id");
Preconditions.checkNotNull(groupId, "groupId");
Preconditions.checkNotNull(dataSource, "dataSource");
Preconditions.checkNotNull(interval, "interval");
this.id = id;
this.groupId = groupId;
this.dataSource = dataSource;
this.interval = interval;
this.id = Preconditions.checkNotNull(id, "id");
this.groupId = Preconditions.checkNotNull(groupId, "groupId");
this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource");
this.interval = Optional.fromNullable(interval);
}
@JsonProperty
@ -81,15 +69,15 @@ public abstract class AbstractTask implements Task
return dataSource;
}
@JsonProperty
@JsonProperty("interval")
@Override
public Interval getInterval()
public Optional<Interval> getFixedInterval()
{
return interval;
}
@Override
public TaskStatus preflight(TaskContext context) throws Exception
public TaskStatus preflight(TaskToolbox toolbox) throws Exception
{
return TaskStatus.running(id);
}
@ -101,7 +89,7 @@ public abstract class AbstractTask implements Task
.add("id", id)
.add("type", getType())
.add("dataSource", dataSource)
.add("interval", getInterval())
.add("interval", getFixedInterval())
.toString();
}
}

View File

@ -62,7 +62,7 @@ public class AppendTask extends MergeTask
throws Exception
{
VersionedIntervalTimeline<String, DataSegment> timeline = new VersionedIntervalTimeline<String, DataSegment>(
Ordering.natural().nullsFirst()
Ordering.<String>natural().nullsFirst()
);
for (DataSegment segment : segments.keySet()) {
@ -111,9 +111,9 @@ public class AppendTask extends MergeTask
}
@Override
public Type getType()
public String getType()
{
return Task.Type.APPEND;
return "append";
}
private class SegmentToMergeHolder

View File

@ -82,8 +82,8 @@ public class DefaultMergeTask extends MergeTask
}
@Override
public Type getType()
public String getType()
{
return Task.Type.MERGE;
return "merge";
}
}

View File

@ -21,6 +21,9 @@ package com.metamx.druid.merger.common.task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.metamx.common.logger.Logger;
import com.metamx.druid.QueryGranularity;
@ -30,19 +33,16 @@ import com.metamx.druid.index.v1.IncrementalIndex;
import com.metamx.druid.index.v1.IncrementalIndexAdapter;
import com.metamx.druid.index.v1.IndexMerger;
import com.metamx.druid.index.v1.IndexableAdapter;
import com.metamx.druid.jackson.DefaultObjectMapper;
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.coordinator.TaskContext;
import com.metamx.druid.merger.common.actions.LockListAction;
import com.metamx.druid.merger.common.actions.SegmentInsertAction;
import com.metamx.druid.shard.NoneShardSpec;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import java.io.File;
import java.util.ArrayList;
public class DeleteTask extends AbstractTask
{
@ -63,29 +63,31 @@ public class DeleteTask extends AbstractTask
new DateTime().toString()
),
dataSource,
interval
Preconditions.checkNotNull(interval, "interval")
);
}
@Override
public Type getType()
public String getType()
{
return Task.Type.DELETE;
return "delete";
}
@Override
public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
// Strategy: Create an empty segment covering the interval to be deleted
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this)));
final Interval interval = this.getFixedInterval().get();
final IncrementalIndex empty = new IncrementalIndex(0, QueryGranularity.NONE, new AggregatorFactory[0]);
final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(this.getInterval(), empty);
final IndexableAdapter emptyAdapter = new IncrementalIndexAdapter(interval, empty);
// Create DataSegment
final DataSegment segment =
DataSegment.builder()
.dataSource(this.getDataSource())
.interval(this.getInterval())
.version(context.getVersion())
.interval(interval)
.version(myLock.getVersion())
.shardSpec(new NoneShardSpec())
.build();
@ -102,6 +104,8 @@ public class DeleteTask extends AbstractTask
segment.getVersion()
);
return TaskStatus.success(getId(), Lists.newArrayList(uploadedSegment));
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(uploadedSegment)));
return TaskStatus.success(getId());
}
}

View File

@ -22,7 +22,7 @@ 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.collect.Iterables;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Ordering;
@ -33,18 +33,15 @@ import com.metamx.common.logger.Logger;
import com.metamx.druid.input.InputRow;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.coordinator.TaskContext;
import com.metamx.druid.merger.common.actions.SpawnTasksAction;
import com.metamx.druid.realtime.Firehose;
import com.metamx.druid.realtime.FirehoseFactory;
import com.metamx.druid.realtime.Schema;
import com.metamx.druid.shard.NoneShardSpec;
import com.metamx.druid.shard.ShardSpec;
import com.metamx.druid.shard.SingleDimensionShardSpec;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -53,8 +50,15 @@ public class IndexDeterminePartitionsTask extends AbstractTask
{
@JsonProperty
private final FirehoseFactory firehoseFactory;
@JsonProperty private final Schema schema;
@JsonProperty private final long targetPartitionSize;
@JsonProperty
private final Schema schema;
@JsonProperty
private final long targetPartitionSize;
@JsonProperty
private final int rowFlushBoundary;
private static final Logger log = new Logger(IndexTask.class);
@ -64,7 +68,8 @@ public class IndexDeterminePartitionsTask extends AbstractTask
@JsonProperty("interval") Interval interval,
@JsonProperty("firehose") FirehoseFactory firehoseFactory,
@JsonProperty("schema") Schema schema,
@JsonProperty("targetPartitionSize") long targetPartitionSize
@JsonProperty("targetPartitionSize") long targetPartitionSize,
@JsonProperty("rowFlushBoundary") int rowFlushBoundary
)
{
super(
@ -76,30 +81,30 @@ public class IndexDeterminePartitionsTask extends AbstractTask
),
groupId,
schema.getDataSource(),
interval
Preconditions.checkNotNull(interval, "interval")
);
this.firehoseFactory = firehoseFactory;
this.schema = schema;
this.targetPartitionSize = targetPartitionSize;
this.rowFlushBoundary = rowFlushBoundary;
}
@Override
public Type getType()
public String getType()
{
return Type.INDEX;
return "index_partitions";
}
@Override
public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
log.info("Running with targetPartitionSize[%d]", targetPartitionSize);
// This is similar to what DeterminePartitionsJob does in the hadoop indexer, but we don't require
// a preconfigured partition dimension (we'll just pick the one with highest cardinality).
// TODO: Replace/merge/whatever with hadoop determine-partitions code
// XXX - Space-efficiency (stores all unique dimension values, although at least not all combinations)
// XXX - Time-efficiency (runs all this on one single node instead of through map/reduce)
// We know this exists
final Interval interval = getFixedInterval().get();
// Blacklist dimensions that have multiple values per row
final Set<String> unusableDimensions = Sets.newHashSet();
@ -111,24 +116,24 @@ public class IndexDeterminePartitionsTask extends AbstractTask
final Firehose firehose = firehoseFactory.connect();
try {
while(firehose.hasMore()) {
while (firehose.hasMore()) {
final InputRow inputRow = firehose.nextRow();
if(getInterval().contains(inputRow.getTimestampFromEpoch())) {
if (interval.contains(inputRow.getTimestampFromEpoch())) {
// Extract dimensions from event
for (final String dim : inputRow.getDimensions()) {
final List<String> dimValues = inputRow.getDimension(dim);
if(!unusableDimensions.contains(dim)) {
if (!unusableDimensions.contains(dim)) {
if(dimValues.size() == 1) {
if (dimValues.size() == 1) {
// Track this value
TreeMultiset<String> dimensionValueMultiset = dimensionValueMultisets.get(dim);
if(dimensionValueMultiset == null) {
if (dimensionValueMultiset == null) {
dimensionValueMultiset = TreeMultiset.create();
dimensionValueMultisets.put(dim, dimensionValueMultiset);
}
@ -149,7 +154,8 @@ public class IndexDeterminePartitionsTask extends AbstractTask
}
}
} finally {
}
finally {
firehose.close();
}
@ -169,7 +175,7 @@ public class IndexDeterminePartitionsTask extends AbstractTask
}
};
if(dimensionValueMultisets.isEmpty()) {
if (dimensionValueMultisets.isEmpty()) {
// No suitable partition dimension. We'll make one big segment and hope for the best.
log.info("No suitable partition dimension found");
shardSpecs.add(new NoneShardSpec());
@ -191,9 +197,9 @@ public class IndexDeterminePartitionsTask extends AbstractTask
// Iterate over unique partition dimension values in sorted order
String currentPartitionStart = null;
int currentPartitionSize = 0;
for(final String partitionDimValue : partitionDimValues.elementSet()) {
for (final String partitionDimValue : partitionDimValues.elementSet()) {
currentPartitionSize += partitionDimValues.count(partitionDimValue);
if(currentPartitionSize >= targetPartitionSize) {
if (currentPartitionSize >= targetPartitionSize) {
final ShardSpec shardSpec = new SingleDimensionShardSpec(
partitionDim,
currentPartitionStart,
@ -229,28 +235,31 @@ public class IndexDeterminePartitionsTask extends AbstractTask
}
}
return TaskStatus.continued(
getId(),
Lists.transform(
shardSpecs, new Function<ShardSpec, Task>()
List<Task> nextTasks = Lists.transform(
shardSpecs,
new Function<ShardSpec, Task>()
{
@Override
public Task apply(ShardSpec shardSpec)
{
return new IndexGeneratorTask(
getGroupId(),
getInterval(),
getFixedInterval().get(),
firehoseFactory,
new Schema(
schema.getDataSource(),
schema.getAggregators(),
schema.getIndexGranularity(),
shardSpec
)
),
rowFlushBoundary
);
}
}
)
);
toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, nextTasks));
return TaskStatus.success(getId());
}
}

View File

@ -21,25 +21,26 @@ package com.metamx.druid.merger.common.task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableList;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Maps;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.input.InputRow;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.actions.LockListAction;
import com.metamx.druid.merger.common.actions.SegmentInsertAction;
import com.metamx.druid.merger.common.index.YeOldePlumberSchool;
import com.metamx.druid.merger.coordinator.TaskContext;
import com.metamx.druid.realtime.FireDepartmentMetrics;
import com.metamx.druid.realtime.Firehose;
import com.metamx.druid.realtime.FirehoseFactory;
import com.metamx.druid.realtime.Plumber;
import com.metamx.druid.realtime.Schema;
import com.metamx.druid.loading.SegmentPusher;
import com.metamx.druid.realtime.Sink;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@ -57,6 +58,9 @@ public class IndexGeneratorTask extends AbstractTask
@JsonProperty
private final Schema schema;
@JsonProperty
private final int rowFlushBoundary;
private static final Logger log = new Logger(IndexTask.class);
@JsonCreator
@ -64,7 +68,8 @@ public class IndexGeneratorTask extends AbstractTask
@JsonProperty("groupId") String groupId,
@JsonProperty("interval") Interval interval,
@JsonProperty("firehose") FirehoseFactory firehoseFactory,
@JsonProperty("schema") Schema schema
@JsonProperty("schema") Schema schema,
@JsonProperty("rowFlushBoundary") int rowFlushBoundary
)
{
super(
@ -77,22 +82,29 @@ public class IndexGeneratorTask extends AbstractTask
),
groupId,
schema.getDataSource(),
interval
Preconditions.checkNotNull(interval, "interval")
);
this.firehoseFactory = firehoseFactory;
this.schema = schema;
this.rowFlushBoundary = rowFlushBoundary;
}
@Override
public Type getType()
public String getType()
{
return Type.INDEX;
return "index_generator";
}
@Override
public TaskStatus run(final TaskContext context, final TaskToolbox toolbox) throws Exception
public TaskStatus run(final TaskToolbox toolbox) throws Exception
{
// We should have a lock from before we started running
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this)));
// We know this exists
final Interval interval = getFixedInterval().get();
// Set up temporary directory for indexing
final File tmpDir = new File(
String.format(
@ -101,9 +113,9 @@ public class IndexGeneratorTask extends AbstractTask
String.format(
"%s_%s_%s_%s_%s",
this.getDataSource(),
this.getInterval().getStart(),
this.getInterval().getEnd(),
context.getVersion(),
interval.getStart(),
interval.getEnd(),
myLock.getVersion(),
schema.getShardSpec().getPartitionNum()
)
)
@ -111,7 +123,7 @@ public class IndexGeneratorTask extends AbstractTask
// We need to track published segments.
final List<DataSegment> pushedSegments = new CopyOnWriteArrayList<DataSegment>();
final SegmentPusher wrappedSegmentPusher = new SegmentPusher()
final DataSegmentPusher wrappedDataSegmentPusher = new DataSegmentPusher()
{
@Override
public DataSegment push(File file, DataSegment segment) throws IOException
@ -126,12 +138,17 @@ public class IndexGeneratorTask extends AbstractTask
final FireDepartmentMetrics metrics = new FireDepartmentMetrics();
final Firehose firehose = firehoseFactory.connect();
final Plumber plumber = new YeOldePlumberSchool(
getInterval(),
context.getVersion(),
wrappedSegmentPusher,
interval,
myLock.getVersion(),
wrappedDataSegmentPusher,
tmpDir
).findPlumber(schema, metrics);
// rowFlushBoundary for this job
final int myRowFlushBoundary = this.rowFlushBoundary > 0
? rowFlushBoundary
: toolbox.getConfig().getDefaultRowFlushBoundary();
try {
while(firehose.hasMore()) {
final InputRow inputRow = firehose.nextRow();
@ -150,7 +167,7 @@ public class IndexGeneratorTask extends AbstractTask
int numRows = sink.add(inputRow);
metrics.incrementProcessed();
if(numRows >= toolbox.getConfig().getRowFlushBoundary()) {
if(numRows >= myRowFlushBoundary) {
plumber.persist(firehose.commit());
}
} else {
@ -175,8 +192,11 @@ public class IndexGeneratorTask extends AbstractTask
metrics.rowOutput()
);
// Request segment pushes
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.copyOf(pushedSegments)));
// Done
return TaskStatus.success(getId(), ImmutableList.copyOf(pushedSegments));
return TaskStatus.success(getId());
}
/**
@ -185,7 +205,7 @@ public class IndexGeneratorTask extends AbstractTask
* @return true or false
*/
private boolean shouldIndex(InputRow inputRow) {
if(!getInterval().contains(inputRow.getTimestampFromEpoch())) {
if(!getFixedInterval().get().contains(inputRow.getTimestampFromEpoch())) {
return false;
}

View File

@ -29,7 +29,7 @@ import com.metamx.druid.aggregation.AggregatorFactory;
import com.metamx.druid.indexer.granularity.GranularitySpec;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.coordinator.TaskContext;
import com.metamx.druid.merger.common.actions.SpawnTasksAction;
import com.metamx.druid.realtime.FirehoseFactory;
import com.metamx.druid.realtime.Schema;
import com.metamx.druid.shard.NoneShardSpec;
@ -42,11 +42,23 @@ import java.util.List;
public class IndexTask extends AbstractTask
{
@JsonProperty private final GranularitySpec granularitySpec;
@JsonProperty private final AggregatorFactory[] aggregators;
@JsonProperty private final QueryGranularity indexGranularity;
@JsonProperty private final long targetPartitionSize;
@JsonProperty private final FirehoseFactory firehoseFactory;
@JsonProperty
private final GranularitySpec granularitySpec;
@JsonProperty
private final AggregatorFactory[] aggregators;
@JsonProperty
private final QueryGranularity indexGranularity;
@JsonProperty
private final long targetPartitionSize;
@JsonProperty
private final FirehoseFactory firehoseFactory;
@JsonProperty
private final int rowFlushBoundary;
private static final Logger log = new Logger(IndexTask.class);
@ -57,7 +69,8 @@ public class IndexTask extends AbstractTask
@JsonProperty("aggregators") AggregatorFactory[] aggregators,
@JsonProperty("indexGranularity") QueryGranularity indexGranularity,
@JsonProperty("targetPartitionSize") long targetPartitionSize,
@JsonProperty("firehose") FirehoseFactory firehoseFactory
@JsonProperty("firehose") FirehoseFactory firehoseFactory,
@JsonProperty("rowFlushBoundary") int rowFlushBoundary
)
{
super(
@ -75,6 +88,7 @@ public class IndexTask extends AbstractTask
this.indexGranularity = indexGranularity;
this.targetPartitionSize = targetPartitionSize;
this.firehoseFactory = firehoseFactory;
this.rowFlushBoundary = rowFlushBoundary;
}
public List<Task> toSubtasks()
@ -95,7 +109,8 @@ public class IndexTask extends AbstractTask
indexGranularity,
new NoneShardSpec()
),
targetPartitionSize
targetPartitionSize,
rowFlushBoundary
)
);
} else {
@ -110,7 +125,8 @@ public class IndexTask extends AbstractTask
aggregators,
indexGranularity,
new NoneShardSpec()
)
),
rowFlushBoundary
)
);
}
@ -120,19 +136,20 @@ public class IndexTask extends AbstractTask
}
@Override
public Type getType()
public String getType()
{
return Type.INDEX;
return "index";
}
@Override
public TaskStatus preflight(TaskContext context) throws Exception
public TaskStatus preflight(TaskToolbox toolbox) throws Exception
{
return TaskStatus.continued(getId(), toSubtasks());
toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, toSubtasks()));
return TaskStatus.success(getId());
}
@Override
public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
throw new IllegalStateException("IndexTasks should not be run!");
}

View File

@ -0,0 +1,98 @@
package com.metamx.druid.merger.common.task;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.metamx.common.ISE;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.actions.LockListAction;
import com.metamx.druid.merger.common.actions.SegmentListUnusedAction;
import com.metamx.druid.merger.common.actions.SegmentNukeAction;
import com.metamx.druid.merger.common.TaskLock;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import java.util.List;
/**
*/
public class KillTask extends AbstractTask
{
private static final Logger log = new Logger(KillTask.class);
@JsonCreator
public KillTask(
@JsonProperty("dataSource") String dataSource,
@JsonProperty("interval") Interval interval
)
{
super(
String.format(
"kill_%s_%s_%s_%s",
dataSource,
interval.getStart(),
interval.getEnd(),
new DateTime().toString()
),
dataSource,
interval
);
}
@Override
public String getType()
{
return "kill";
}
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
// Confirm we have a lock (will throw if there isn't exactly one element)
final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction(this)));
if(!myLock.getDataSource().equals(getDataSource())) {
throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource());
}
if(!myLock.getInterval().equals(getFixedInterval().get())) {
throw new ISE("WTF?! Lock interval[%s] != task interval[%s]", myLock.getInterval(), getFixedInterval().get());
}
// List unused segments
final List<DataSegment> unusedSegments = toolbox.getTaskActionClient()
.submit(
new SegmentListUnusedAction(
this,
myLock.getDataSource(),
myLock.getInterval()
)
);
// Verify none of these segments have versions > lock version
for(final DataSegment unusedSegment : unusedSegments) {
if(unusedSegment.getVersion().compareTo(myLock.getVersion()) > 0) {
throw new ISE(
"WTF?! Unused segment[%s] has version[%s] > task version[%s]",
unusedSegment.getIdentifier(),
unusedSegment.getVersion(),
myLock.getVersion()
);
}
log.info("OK to kill segment: %s", unusedSegment.getIdentifier());
}
// Kill segments
toolbox.getSegmentKiller().kill(unusedSegments);
// Remove metadata for these segments
toolbox.getTaskActionClient().submit(new SegmentNukeAction(this, ImmutableSet.copyOf(unusedSegments)));
return TaskStatus.success(getId());
}
}

View File

@ -22,6 +22,7 @@ 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;
@ -31,24 +32,22 @@ 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.Maps;
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.loading.SegmentPuller;
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.coordinator.TaskContext;
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.apache.commons.codec.digest.DigestUtils;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@ -118,11 +117,12 @@ public abstract class MergeTask extends AbstractTask
}
@Override
public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception
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(), context.getVersion(), segments);
final DataSegment mergedSegment = computeMergedSegment(getDataSource(), myLock.getVersion(), segments);
final File taskDir = toolbox.getConfig().getTaskDir(this);
try {
@ -147,13 +147,7 @@ public abstract class MergeTask extends AbstractTask
// download segments to merge
final Map<String, SegmentPuller> segmentGetters = toolbox.getSegmentGetters(this);
final Map<DataSegment, File> gettedSegments = Maps.newHashMap();
for (final DataSegment segment : segments) {
Map<String, Object> loadSpec = segment.getLoadSpec();
SegmentPuller segmentPuller = segmentGetters.get(loadSpec.get("type"));
gettedSegments.put(segment, segmentPuller.getSegmentFiles(segment));
}
final Map<DataSegment, File> gettedSegments = toolbox.getSegments(this, segments);
// merge files together
final File fileToUpload = merge(gettedSegments, new File(taskDir, "merged"));
@ -176,7 +170,9 @@ public abstract class MergeTask extends AbstractTask
emitter.emit(builder.build("merger/uploadTime", System.currentTimeMillis() - uploadStart));
emitter.emit(builder.build("merger/mergeSize", uploadedSegment.getSize()));
return TaskStatus.success(getId(), Lists.newArrayList(uploadedSegment));
toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(uploadedSegment)));
return TaskStatus.success(getId());
}
catch (Exception e) {
log.error(
@ -205,7 +201,7 @@ public abstract class MergeTask extends AbstractTask
* we are operating on every segment that overlaps the chosen interval.
*/
@Override
public TaskStatus preflight(TaskContext context)
public TaskStatus preflight(TaskToolbox toolbox)
{
final Function<DataSegment, String> toIdentifier = new Function<DataSegment, String>()
{
@ -216,7 +212,13 @@ public abstract class MergeTask extends AbstractTask
}
};
final Set<String> current = ImmutableSet.copyOf(Iterables.transform(context.getCurrentSegments(), toIdentifier));
final Set<String> current = ImmutableSet.copyOf(
Iterables.transform(
toolbox.getTaskActionClient()
.submit(new SegmentListUsedAction(this, getDataSource(), getFixedInterval().get())),
toIdentifier
)
);
final Set<String> requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier));
final Set<String> missingFromRequested = Sets.difference(current, requested);
@ -254,7 +256,7 @@ public abstract class MergeTask extends AbstractTask
return Objects.toStringHelper(this)
.add("id", getId())
.add("dataSource", getDataSource())
.add("interval", getInterval())
.add("interval", getFixedInterval())
.add("segments", segments)
.toString();
}
@ -280,7 +282,11 @@ public abstract class MergeTask extends AbstractTask
)
);
return String.format("%s_%s", dataSource, DigestUtils.shaHex(segmentIDs).toLowerCase());
return String.format(
"%s_%s",
dataSource,
Hashing.sha1().hashString(segmentIDs, Charsets.UTF_8).toString().toLowerCase()
);
}
private static Interval computeMergedInterval(final List<DataSegment> segments)

View File

@ -21,15 +21,9 @@ package com.metamx.druid.merger.common.task;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.google.common.base.Optional;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.coordinator.TaskContext;
import com.metamx.druid.merger.common.task.IndexDeterminePartitionsTask;
import com.metamx.druid.merger.common.task.IndexGeneratorTask;
import com.metamx.druid.merger.common.task.IndexTask;
import org.joda.time.DateTime;
import org.joda.time.Interval;
/**
@ -38,53 +32,66 @@ import org.joda.time.Interval;
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultMergeTask.class)
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "append", value = AppendTask.class),
@JsonSubTypes.Type(name = "merge", value = DefaultMergeTask.class),
@JsonSubTypes.Type(name = "delete", value = DeleteTask.class),
@JsonSubTypes.Type(name = "kill", value = KillTask.class),
@JsonSubTypes.Type(name = "index", value = IndexTask.class),
@JsonSubTypes.Type(name = "index_partitions", value = IndexDeterminePartitionsTask.class),
@JsonSubTypes.Type(name = "index_generator", value = IndexGeneratorTask.class)
})
public interface Task
{
enum Type
{
INDEX,
MERGE,
APPEND,
DELETE,
TEST
}
/**
* Returns ID of this task. Must be unique across all tasks ever created.
*/
public String getId();
/**
* Returns group ID of this task. Tasks with the same group ID can share locks. If tasks do not need to share locks,
* a common convention is to set group ID equal to task ID.
*/
public String getGroupId();
public Type getType();
/**
* Returns a descriptive label for this task type. Used for metrics emission and logging.
*/
public String getType();
/**
* Returns the datasource this task operates on. Each task can operate on only one datasource.
*/
public String getDataSource();
public Interval getInterval();
/**
* Returns fixed interval for this task, if any. Tasks without fixed intervals are not granted locks when started
* and must explicitly request them.
*/
public Optional<Interval> getFixedInterval();
/**
* Execute preflight checks for a task. This typically runs on the coordinator, and will be run while
* holding a lock on our dataSource and interval. If this method throws an exception, the task should be
* considered a failure.
*
* @param context Context for this task, gathered under indexer lock
* @param toolbox Toolbox for this task
*
* @return Some kind of status (runnable means continue on to a worker, non-runnable means we completed without
* using a worker).
*
* @throws Exception
*/
public TaskStatus preflight(TaskContext context) throws Exception;
public TaskStatus preflight(TaskToolbox toolbox) throws Exception;
/**
* Execute a task. This typically runs on a worker as determined by a TaskRunner, and will be run while
* holding a lock on our dataSource and interval. If this method throws an exception, the task should be
* considered a failure.
*
* @param context Context for this task, gathered under indexer lock
* @param toolbox Toolbox for this task
*
* @return Some kind of finished status (isRunnable must be false).
*
* @throws Exception
*/
public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception;
public TaskStatus run(TaskToolbox toolbox) throws Exception;
}

View File

@ -3,8 +3,6 @@ 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 com.metamx.druid.merger.coordinator.TaskContext;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@ -25,15 +23,13 @@ public class V8toV9UpgradeTask extends AbstractTask
}
@Override
public Type getType()
public String getType()
{
throw new UnsupportedOperationException("Do we really need to return a Type?");
return "8to9";
}
@Override
public TaskStatus run(
TaskContext context, TaskToolbox toolbox
) throws Exception
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
throw new UnsupportedOperationException();
}

View File

@ -24,18 +24,25 @@ import com.google.common.base.Function;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.metamx.common.Pair;
import com.metamx.common.logger.Logger;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.actions.TaskAction;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig;
import org.joda.time.DateTime;
import org.skife.jdbi.v2.DBI;
import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.exceptions.StatementException;
import org.skife.jdbi.v2.tweak.HandleCallback;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Map;
@ -68,6 +75,7 @@ public class DbTaskStorage implements TaskStorage
log.info("Inserting task %s with status: %s", task.getId(), status);
try {
dbi.withHandle(
new HandleCallback<Void>()
{
@ -91,21 +99,22 @@ public class DbTaskStorage implements TaskStorage
}
}
);
} catch (StatementException e) {
// Might be a duplicate task ID.
if(getTask(task.getId()).isPresent()) {
throw new TaskExistsException(task.getId(), e);
} else {
throw e;
}
}
}
@Override
public void setStatus(final String taskid, final TaskStatus status)
public void setStatus(final TaskStatus status)
{
Preconditions.checkNotNull(taskid, "task");
Preconditions.checkNotNull(status, "status");
Preconditions.checkArgument(
taskid.equals(status.getId()),
"Task/Status ID mismatch[%s/%s]",
taskid,
status.getId()
);
log.info("Updating task %s to status: %s", taskid, status);
log.info("Updating task %s to status: %s", status.getId(), status);
int updated = dbi.withHandle(
new HandleCallback<Integer>()
@ -115,12 +124,13 @@ public class DbTaskStorage implements TaskStorage
{
return handle.createStatement(
String.format(
"UPDATE %s SET status_code = :status_code, status_payload = :status_payload WHERE id = :id",
"UPDATE %s SET status_code = :status_code, status_payload = :status_payload WHERE id = :id AND status_code = :old_status_code",
dbConnectorConfig.getTaskTable()
)
)
.bind("id", taskid)
.bind("id", status.getId())
.bind("status_code", status.getStatusCode().toString())
.bind("old_status_code", TaskStatus.Status.RUNNING.toString())
.bind("status_payload", jsonMapper.writeValueAsString(status))
.execute();
}
@ -128,35 +138,35 @@ public class DbTaskStorage implements TaskStorage
);
if(updated != 1) {
throw new IllegalStateException(String.format("Task not found: %s", taskid));
throw new IllegalStateException(String.format("Running task not found: %s", status.getId()));
}
}
@Override
public void setVersion(final String taskid, final String version)
public Optional<Task> getTask(final String taskid)
{
Preconditions.checkNotNull(taskid, "task");
Preconditions.checkNotNull(version, "version");
log.info("Updating task %s to version: %s", taskid, version);
dbi.withHandle(
new HandleCallback<Void>()
return dbi.withHandle(
new HandleCallback<Optional<Task>>()
{
@Override
public Void withHandle(Handle handle) throws Exception
public Optional<Task> withHandle(Handle handle) throws Exception
{
handle.createStatement(
final List<Map<String, Object>> dbTasks =
handle.createQuery(
String.format(
"UPDATE %s SET version = :version WHERE id = :id",
"SELECT payload FROM %s WHERE id = :id",
dbConnectorConfig.getTaskTable()
)
)
.bind("id", taskid)
.bind("version", version)
.execute();
.list();
return null;
if(dbTasks.size() == 0) {
return Optional.absent();
} else {
final Map<String, Object> dbStatus = Iterables.getOnlyElement(dbTasks);
return Optional.of(jsonMapper.readValue(dbStatus.get("payload").toString(), Task.class));
}
}
}
);
@ -192,36 +202,6 @@ public class DbTaskStorage implements TaskStorage
);
}
@Override
public Optional<String> getVersion(final String taskid)
{
return dbi.withHandle(
new HandleCallback<Optional<String>>()
{
@Override
public Optional<String> withHandle(Handle handle) throws Exception
{
final List<Map<String, Object>> dbStatuses =
handle.createQuery(
String.format(
"SELECT version FROM %s WHERE id = :id",
dbConnectorConfig.getTaskTable()
)
)
.bind("id", taskid)
.list();
if(dbStatuses.size() == 0) {
return Optional.absent();
} else {
final Map<String, Object> dbStatus = Iterables.getOnlyElement(dbStatuses);
return Optional.fromNullable((String) dbStatus.get("version"));
}
}
}
);
}
@Override
public List<Task> getRunningTasks()
{
@ -259,4 +239,183 @@ public class DbTaskStorage implements TaskStorage
}
);
}
@Override
public void addLock(final String taskid, final TaskLock taskLock)
{
Preconditions.checkNotNull(taskid, "taskid");
Preconditions.checkNotNull(taskLock, "taskLock");
log.info(
"Adding lock on interval[%s] version[%s] for task: %s",
taskLock.getInterval(),
taskLock.getVersion(),
taskid
);
dbi.withHandle(
new HandleCallback<Integer>()
{
@Override
public Integer withHandle(Handle handle) throws Exception
{
return handle.createStatement(
String.format(
"INSERT INTO %s (task_id, lock_payload) VALUES (:task_id, :lock_payload)",
dbConnectorConfig.getTaskLockTable()
)
)
.bind("task_id", taskid)
.bind("lock_payload", jsonMapper.writeValueAsString(taskLock))
.execute();
}
}
);
}
@Override
public void removeLock(String taskid, TaskLock taskLockToRemove)
{
Preconditions.checkNotNull(taskid, "taskid");
Preconditions.checkNotNull(taskLockToRemove, "taskLockToRemove");
final Map<Long, TaskLock> taskLocks = getLocksWithIds(taskid);
for(final Map.Entry<Long, TaskLock> taskLockWithId : taskLocks.entrySet()) {
final long id = taskLockWithId.getKey();
final TaskLock taskLock = taskLockWithId.getValue();
if(taskLock.equals(taskLockToRemove)) {
log.info("Deleting TaskLock with id[%d]: %s", id, taskLock);
dbi.withHandle(
new HandleCallback<Integer>()
{
@Override
public Integer withHandle(Handle handle) throws Exception
{
return handle.createStatement(
String.format(
"DELETE FROM %s WHERE id = :id",
dbConnectorConfig.getTaskLockTable()
)
)
.bind("id", id)
.execute();
}
}
);
}
}
}
@Override
public List<TaskLock> getLocks(String taskid)
{
return ImmutableList.copyOf(
Iterables.transform(
getLocksWithIds(taskid).entrySet(), new Function<Map.Entry<Long, TaskLock>, TaskLock>()
{
@Override
public TaskLock apply(Map.Entry<Long, TaskLock> e)
{
return e.getValue();
}
}
)
);
}
@Override
public <T> void addAuditLog(final TaskAction<T> taskAction)
{
Preconditions.checkNotNull(taskAction, "taskAction");
log.info("Logging action for task[%s]: %s", taskAction.getTask().getId(), taskAction);
dbi.withHandle(
new HandleCallback<Integer>()
{
@Override
public Integer withHandle(Handle handle) throws Exception
{
return handle.createStatement(
String.format(
"INSERT INTO %s (task_id, log_payload) VALUES (:task_id, :log_payload)",
dbConnectorConfig.getTaskLogTable()
)
)
.bind("task_id", taskAction.getTask().getId())
.bind("log_payload", jsonMapper.writeValueAsString(taskAction))
.execute();
}
}
);
}
@Override
public List<TaskAction> getAuditLogs(final String taskid)
{
return dbi.withHandle(
new HandleCallback<List<TaskAction>>()
{
@Override
public List<TaskAction> withHandle(Handle handle) throws Exception
{
final List<Map<String, Object>> dbTaskLogs =
handle.createQuery(
String.format(
"SELECT log_payload FROM %s WHERE task_id = :task_id",
dbConnectorConfig.getTaskLogTable()
)
)
.bind("task_id", taskid)
.list();
return Lists.transform(
dbTaskLogs, new Function<Map<String, Object>, TaskAction>()
{
@Override
public TaskAction apply(Map<String, Object> row)
{
try {
return jsonMapper.readValue(row.get("payload").toString(), TaskAction.class);
} catch(Exception e) {
throw Throwables.propagate(e);
}
}
}
);
}
}
);
}
private Map<Long, TaskLock> getLocksWithIds(final String taskid)
{
return dbi.withHandle(
new HandleCallback<Map<Long, TaskLock>>()
{
@Override
public Map<Long, TaskLock> withHandle(Handle handle) throws Exception
{
final List<Map<String, Object>> dbTaskLocks =
handle.createQuery(
String.format(
"SELECT id, lock_payload FROM %s WHERE task_id = :task_id",
dbConnectorConfig.getTaskLockTable()
)
)
.bind("task_id", taskid)
.list();
final Map<Long, TaskLock> retMap = Maps.newHashMap();
for(final Map<String, Object> row : dbTaskLocks) {
retMap.put((Long)row.get("id"), jsonMapper.readValue(row.get("lock_payload").toString(), TaskLock.class));
}
return retMap;
}
}
);
}
}

View File

@ -20,14 +20,17 @@
package com.metamx.druid.merger.coordinator;
import com.google.common.base.Throwables;
import com.google.common.collect.Lists;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import com.metamx.druid.merger.common.TaskCallback;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.task.Task;
import org.apache.commons.io.FileUtils;
import java.io.File;
import java.util.Collection;
import java.util.concurrent.ExecutorService;
/**
@ -38,7 +41,7 @@ public class LocalTaskRunner implements TaskRunner
private final TaskToolbox toolbox;
private final ExecutorService exec;
private static final Logger log = new Logger(TaskQueue.class);
private static final Logger log = new Logger(LocalTaskRunner.class);
public LocalTaskRunner(
TaskToolbox toolbox,
@ -56,7 +59,7 @@ public class LocalTaskRunner implements TaskRunner
}
@Override
public void run(final Task task, final TaskContext context, final TaskCallback callback)
public void run(final Task task, final TaskCallback callback)
{
exec.submit(
new Runnable()
@ -65,12 +68,12 @@ public class LocalTaskRunner implements TaskRunner
public void run()
{
final long startTime = System.currentTimeMillis();
final File taskDir = toolbox.getConfig().getTaskDir(task);
TaskStatus status;
try {
status = task.run(context, toolbox);
log.info("Running task: %s", task.getId());
status = task.run(toolbox);
}
catch (InterruptedException e) {
log.error(e, "Interrupted while running task[%s]", task);
@ -86,23 +89,43 @@ public class LocalTaskRunner implements TaskRunner
}
try {
final File taskDir = toolbox.getConfig().getTaskDir(task);
if (taskDir.exists()) {
log.info("Removing task directory: %s", taskDir);
FileUtils.deleteDirectory(taskDir);
}
}
catch (Exception e) {
log.error(e, "Failed to delete task directory[%s]", taskDir.toString());
log.error(e, "Failed to delete task directory: %s", task.getId());
}
try {
callback.notify(status.withDuration(System.currentTimeMillis() - startTime));
} catch(Throwable t) {
log.error(t, "Uncaught Throwable during callback for task[%s]", task);
throw Throwables.propagate(t);
} catch(Exception e) {
log.error(e, "Uncaught Exception during callback for task[%s]", task);
throw Throwables.propagate(e);
}
}
}
);
}
@Override
public Collection<TaskRunnerWorkItem> getRunningTasks()
{
return Lists.newArrayList();
}
@Override
public Collection<TaskRunnerWorkItem> getPendingTasks()
{
return Lists.newArrayList();
}
@Override
public Collection<ZkWorker> getWorkers()
{
return Lists.newArrayList();
}
}

View File

@ -21,17 +21,20 @@ package com.metamx.druid.merger.coordinator;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.HashMultimap;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Maps;
import com.google.common.collect.Multimap;
import com.metamx.common.logger.Logger;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.actions.TaskAction;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.task.Task;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Map;
import java.util.concurrent.locks.ReentrantLock;
/**
* Implements an in-heap TaskStorage facility, with no persistence across restarts. This class is not
@ -39,13 +42,19 @@ import java.util.Map;
*/
public class LocalTaskStorage implements TaskStorage
{
private final ReentrantLock giant = new ReentrantLock();
private final Map<String, TaskStuff> tasks = Maps.newHashMap();
private final Multimap<String, TaskLock> taskLocks = HashMultimap.create();
private final Multimap<String, TaskAction> taskActions = ArrayListMultimap.create();
private static final Logger log = new Logger(LocalTaskStorage.class);
@Override
public void insert(Task task, TaskStatus status)
{
giant.lock();
try {
Preconditions.checkNotNull(task, "task");
Preconditions.checkNotNull(status, "status");
Preconditions.checkArgument(
@ -54,78 +63,157 @@ public class LocalTaskStorage implements TaskStorage
task.getId(),
status.getId()
);
Preconditions.checkState(!tasks.containsKey(task.getId()), "Task ID must not already be present: %s", task.getId());
if(tasks.containsKey(task.getId())) {
throw new TaskExistsException(task.getId());
}
log.info("Inserting task %s with status: %s", task.getId(), status);
tasks.put(task.getId(), new TaskStuff(task, status));
} finally {
giant.unlock();
}
}
@Override
public void setStatus(String taskid, TaskStatus status)
public Optional<Task> getTask(String taskid)
{
giant.lock();
try {
Preconditions.checkNotNull(taskid, "taskid");
if(tasks.containsKey(taskid)) {
return Optional.of(tasks.get(taskid).getTask());
} else {
return Optional.absent();
}
} finally {
giant.unlock();
}
}
@Override
public void setStatus(TaskStatus status)
{
giant.lock();
try {
Preconditions.checkNotNull(status, "status");
final String taskid = status.getId();
Preconditions.checkState(tasks.containsKey(taskid), "Task ID must already be present: %s", taskid);
Preconditions.checkState(tasks.get(taskid).getStatus().isRunnable(), "Task status must be runnable: %s", taskid);
log.info("Updating task %s to status: %s", taskid, status);
tasks.put(taskid, tasks.get(taskid).withStatus(status));
} finally {
giant.unlock();
}
}
@Override
public Optional<TaskStatus> getStatus(String taskid)
{
giant.lock();
try {
Preconditions.checkNotNull(taskid, "taskid");
if(tasks.containsKey(taskid)) {
return Optional.of(tasks.get(taskid).status);
return Optional.of(tasks.get(taskid).getStatus());
} else {
return Optional.absent();
}
}
@Override
public void setVersion(String taskid, String version)
{
Preconditions.checkNotNull(taskid, "taskid");
Preconditions.checkNotNull(version, "status");
Preconditions.checkState(tasks.containsKey(taskid), "Task ID must already be present: %s", taskid);
log.info("Updating task %s to version: %s", taskid, version);
tasks.put(taskid, tasks.get(taskid).withVersion(version));
}
@Override
public Optional<String> getVersion(String taskid)
{
Preconditions.checkNotNull(taskid, "taskid");
if(tasks.containsKey(taskid)) {
return tasks.get(taskid).version;
} else {
return Optional.absent();
} finally {
giant.unlock();
}
}
@Override
public List<Task> getRunningTasks()
{
giant.lock();
try {
final ImmutableList.Builder<Task> listBuilder = ImmutableList.builder();
for(final TaskStuff taskStuff : tasks.values()) {
if(taskStuff.status.isRunnable()) {
listBuilder.add(taskStuff.task);
if(taskStuff.getStatus().isRunnable()) {
listBuilder.add(taskStuff.getTask());
}
}
return listBuilder.build();
} finally {
giant.unlock();
}
}
@Override
public void addLock(final String taskid, final TaskLock taskLock)
{
giant.lock();
try {
Preconditions.checkNotNull(taskLock, "taskLock");
taskLocks.put(taskid, taskLock);
} finally {
giant.unlock();
}
}
@Override
public void removeLock(final String taskid, final TaskLock taskLock)
{
giant.lock();
try {
Preconditions.checkNotNull(taskLock, "taskLock");
taskLocks.remove(taskid, taskLock);
} finally {
giant.unlock();
}
}
@Override
public List<TaskLock> getLocks(final String taskid)
{
giant.lock();
try {
return ImmutableList.copyOf(taskLocks.get(taskid));
} finally {
giant.unlock();
}
}
@Override
public <T> void addAuditLog(TaskAction<T> taskAction)
{
giant.lock();
try {
taskActions.put(taskAction.getTask().getId(), taskAction);
} finally {
giant.unlock();
}
}
@Override
public List<TaskAction> getAuditLogs(String taskid)
{
giant.lock();
try {
return ImmutableList.copyOf(taskActions.get(taskid));
} finally {
giant.unlock();
}
}
private static class TaskStuff
{
final Task task;
final TaskStatus status;
final Optional<String> version;
private TaskStuff(Task task, TaskStatus status)
{
this(task, status, Optional.<String>absent());
}
private TaskStuff(Task task, TaskStatus status, Optional<String> version)
{
Preconditions.checkNotNull(task);
Preconditions.checkNotNull(status);
@ -133,17 +221,21 @@ public class LocalTaskStorage implements TaskStorage
this.task = task;
this.status = status;
this.version = version;
}
public Task getTask()
{
return task;
}
public TaskStatus getStatus()
{
return status;
}
private TaskStuff withStatus(TaskStatus _status)
{
return new TaskStuff(task, _status, version);
}
private TaskStuff withVersion(String _version)
{
return new TaskStuff(task, status, Optional.of(_version));
return new TaskStuff(task, _status);
}
}
}

View File

@ -21,6 +21,7 @@ package com.metamx.druid.merger.coordinator;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Throwables;
import com.google.common.collect.Lists;
import com.google.common.collect.Ordering;
import com.metamx.common.logger.Logger;
@ -28,18 +29,23 @@ import com.metamx.druid.TimelineObjectHolder;
import com.metamx.druid.VersionedIntervalTimeline;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.db.DbConnectorConfig;
import org.joda.time.DateTime;
import org.joda.time.Interval;
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.ResultIterator;
import org.skife.jdbi.v2.StatementContext;
import org.skife.jdbi.v2.TransactionCallback;
import org.skife.jdbi.v2.TransactionStatus;
import org.skife.jdbi.v2.tweak.HandleCallback;
import javax.annotation.Nullable;
import java.io.IOException;
import java.sql.SQLException;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
*/
@ -47,8 +53,6 @@ public class MergerDBCoordinator
{
private static final Logger log = new Logger(MergerDBCoordinator.class);
private final Object lock = new Object();
private final ObjectMapper jsonMapper;
private final DbConnectorConfig dbConnectorConfig;
private final DBI dbi;
@ -64,10 +68,9 @@ public class MergerDBCoordinator
this.dbi = dbi;
}
public List<DataSegment> getSegmentsForInterval(final String dataSource, final Interval interval) throws IOException
public List<DataSegment> getUsedSegmentsForInterval(final String dataSource, final Interval interval)
throws IOException
{
synchronized (lock) {
// XXX Could be reading from a cache if we can assume we're the only one editing the DB
final VersionedIntervalTimeline<String, DataSegment> timeline = dbi.withHandle(
@ -116,7 +119,7 @@ public class MergerDBCoordinator
new Function<TimelineObjectHolder<String, DataSegment>, DataSegment>()
{
@Override
public DataSegment apply(@Nullable TimelineObjectHolder<String, DataSegment> input)
public DataSegment apply(TimelineObjectHolder<String, DataSegment> input)
{
return input.getObject().getChunk(0).getObject();
}
@ -124,21 +127,31 @@ public class MergerDBCoordinator
);
return segments;
}
}
public void announceHistoricalSegment(final DataSegment segment) throws Exception
public void announceHistoricalSegments(final Set<DataSegment> segments) throws Exception
{
synchronized (lock) {
try {
List<Map<String, Object>> exists = dbi.withHandle(
new HandleCallback<List<Map<String, Object>>>()
dbi.inTransaction(
new TransactionCallback<Void>()
{
@Override
public List<Map<String, Object>> withHandle(Handle handle) throws Exception
public Void inTransaction(Handle handle, TransactionStatus transactionStatus) throws Exception
{
return handle.createQuery(
for(final DataSegment segment : segments) {
announceHistoricalSegment(handle, segment);
}
return null;
}
}
);
}
private void announceHistoricalSegment(final Handle handle, final DataSegment segment) throws Exception
{
try {
final List<Map<String, Object>> exists = handle.createQuery(
String.format(
"SELECT id FROM %s WHERE id = ':identifier'",
dbConnectorConfig.getSegmentTable()
@ -147,21 +160,12 @@ public class MergerDBCoordinator
"identifier",
segment.getIdentifier()
).list();
}
}
);
if (!exists.isEmpty()) {
log.info("Found [%s] in DB, not updating DB", segment.getIdentifier());
return;
}
dbi.withHandle(
new HandleCallback<Void>()
{
@Override
public Void withHandle(Handle handle) throws Exception
{
handle.createStatement(
String.format(
"INSERT INTO %s (id, dataSource, created_date, start, end, partitioned, version, used, payload) VALUES (:id, :dataSource, :created_date, :start, :end, :partitioned, :version, :used, :payload)",
@ -179,17 +183,90 @@ public class MergerDBCoordinator
.bind("payload", jsonMapper.writeValueAsString(segment))
.execute();
return null;
}
}
);
log.info("Published segment [%s] to DB", segment.getIdentifier());
}
catch (Exception e) {
log.error(e, "Exception inserting into DB");
throw new RuntimeException(e);
throw e;
}
}
public void deleteSegments(final Set<DataSegment> segments) throws Exception
{
dbi.inTransaction(
new TransactionCallback<Void>()
{
@Override
public Void inTransaction(Handle handle, TransactionStatus transactionStatus) throws Exception
{
for(final DataSegment segment : segments) {
deleteSegment(handle, segment);
}
return null;
}
}
);
}
private void deleteSegment(final Handle handle, final DataSegment segment)
{
handle.createStatement(
String.format("DELETE from %s WHERE id = :id", dbConnectorConfig.getSegmentTable())
).bind("id", segment.getIdentifier())
.execute();
}
public List<DataSegment> getUnusedSegmentsForInterval(final String dataSource, final Interval interval)
{
List<DataSegment> matchingSegments = dbi.withHandle(
new HandleCallback<List<DataSegment>>()
{
@Override
public List<DataSegment> withHandle(Handle handle) throws Exception
{
return handle.createQuery(
String.format(
"SELECT payload FROM %s WHERE dataSource = :dataSource and start >= :start and end <= :end and used = 0",
dbConnectorConfig.getSegmentTable()
)
)
.bind("dataSource", dataSource)
.bind("start", interval.getStart().toString())
.bind("end", interval.getEnd().toString())
.fold(
Lists.<DataSegment>newArrayList(),
new Folder3<List<DataSegment>, Map<String, Object>>()
{
@Override
public List<DataSegment> fold(
List<DataSegment> accumulator,
Map<String, Object> stringObjectMap,
FoldController foldController,
StatementContext statementContext
) throws SQLException
{
try {
DataSegment segment = jsonMapper.readValue(
(String) stringObjectMap.get("payload"),
DataSegment.class
);
accumulator.add(segment);
return accumulator;
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
}
);
}
}
);
log.info("Found %,d segments for %s for interval %s.", matchingSegments.size(), dataSource, interval);
return matchingSegments;
}
}

View File

@ -20,7 +20,6 @@
package com.metamx.druid.merger.coordinator;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Joiner;
import com.google.common.base.Predicate;
import com.google.common.base.Throwables;
@ -29,17 +28,13 @@ import com.google.common.collect.MinMaxPriorityQueue;
import com.google.common.collect.Sets;
import com.google.common.primitives.Ints;
import com.metamx.common.ISE;
import com.metamx.common.concurrent.ScheduledExecutors;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.PeriodGranularity;
import com.metamx.druid.merger.common.TaskHolder;
import com.metamx.druid.merger.common.TaskCallback;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig;
import com.metamx.druid.merger.coordinator.scaling.AutoScalingData;
import com.metamx.druid.merger.coordinator.scaling.ScalingStrategy;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
import com.metamx.druid.merger.worker.Worker;
import com.metamx.emitter.EmittingLogger;
@ -49,31 +44,31 @@ import com.netflix.curator.framework.recipes.cache.PathChildrenCacheEvent;
import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener;
import com.netflix.curator.utils.ZKPaths;
import org.apache.zookeeper.CreateMode;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.joda.time.Period;
import java.util.Arrays;
import java.util.Collection;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentSkipListSet;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
/**
* The RemoteTaskRunner encapsulates all interactions with Zookeeper and keeps track of which workers
* are running which tasks. The RemoteTaskRunner is event driven and updates state according to ephemeral node
* changes in ZK.
* The RemoteTaskRunner's primary responsibility is to assign tasks to worker nodes and manage retries in failure
* scenarios. The RemoteTaskRunner keeps track of which workers are running which tasks and manages coordinator and
* worker interactions over Zookeeper. The RemoteTaskRunner is event driven and updates state according to ephemeral
* node changes in ZK.
* <p/>
* The RemoteTaskRunner will assign tasks to a node until the node hits capacity. RemoteTaskRunners have scaling
* strategies to help them decide when to create or delete new resources. When tasks are assigned to the remote
* task runner and no workers have capacity to handle the task, provisioning will be done according to the strategy.
* The remote task runner periodically runs a check to see if any worker nodes have not had any work for a
* specified period of time. If so, the worker node will be terminated.
* The RemoteTaskRunner will assign tasks to a node until the node hits capacity. At that point, task assignment will
* fail. The RemoteTaskRunner depends on another component to create additional worker resources.
* For example, {@link com.metamx.druid.merger.coordinator.scaling.ResourceManagementScheduler} can take care of these duties.
* <p/>
* If a worker node becomes inexplicably disconnected from Zk, the RemoteTaskRunner will automatically retry any tasks
* that were associated with the node.
@ -89,20 +84,19 @@ public class RemoteTaskRunner implements TaskRunner
private final PathChildrenCache workerPathCache;
private final ScheduledExecutorService scheduledExec;
private final RetryPolicyFactory retryPolicyFactory;
private final ScalingStrategy strategy;
private final WorkerSetupManager workerSetupManager;
// all workers that exist in ZK
private final Map<String, WorkerWrapper> zkWorkers = new ConcurrentHashMap<String, WorkerWrapper>();
// all tasks that are assigned or need to be assigned
private final Map<String, TaskWrapper> tasks = new ConcurrentHashMap<String, TaskWrapper>();
private final Map<String, ZkWorker> zkWorkers = new ConcurrentHashMap<String, ZkWorker>();
// all tasks that have been assigned to a worker
private final TaskRunnerWorkQueue runningTasks = new TaskRunnerWorkQueue();
// tasks that have not yet run
private final TaskRunnerWorkQueue pendingTasks = new TaskRunnerWorkQueue();
private final ExecutorService runPendingTasksExec = Executors.newSingleThreadExecutor();
private final ConcurrentSkipListSet<String> currentlyProvisioning = new ConcurrentSkipListSet<String>();
private final ConcurrentSkipListSet<String> currentlyTerminating = new ConcurrentSkipListSet<String>();
private final Object statusLock = new Object();
private volatile DateTime lastProvisionTime = new DateTime();
private volatile DateTime lastTerminateTime = new DateTime();
private volatile boolean started = false;
public RemoteTaskRunner(
@ -112,7 +106,6 @@ public class RemoteTaskRunner implements TaskRunner
PathChildrenCache workerPathCache,
ScheduledExecutorService scheduledExec,
RetryPolicyFactory retryPolicyFactory,
ScalingStrategy strategy,
WorkerSetupManager workerSetupManager
)
{
@ -122,7 +115,6 @@ public class RemoteTaskRunner implements TaskRunner
this.workerPathCache = workerPathCache;
this.scheduledExec = scheduledExec;
this.retryPolicyFactory = retryPolicyFactory;
this.strategy = strategy;
this.workerSetupManager = workerSetupManager;
}
@ -130,6 +122,11 @@ public class RemoteTaskRunner implements TaskRunner
public void start()
{
try {
if (started) {
return;
}
// Add listener for creation/deletion of workers
workerPathCache.getListenable().addListener(
new PathChildrenCacheListener()
{
@ -156,77 +153,6 @@ public class RemoteTaskRunner implements TaskRunner
);
workerPathCache.start();
// Schedule termination of worker nodes periodically
Period period = new Period(config.getTerminateResourcesDuration());
PeriodGranularity granularity = new PeriodGranularity(period, config.getTerminateResourcesOriginDateTime(), null);
final long startTime = granularity.next(granularity.truncate(new DateTime().getMillis()));
ScheduledExecutors.scheduleAtFixedRate(
scheduledExec,
new Duration(
System.currentTimeMillis(),
startTime
),
config.getTerminateResourcesDuration(),
new Runnable()
{
@Override
public void run()
{
if (currentlyTerminating.isEmpty()) {
if (zkWorkers.size() <= workerSetupManager.getWorkerSetupData().getMinNumWorkers()) {
return;
}
int workerCount = 0;
List<WorkerWrapper> thoseLazyWorkers = Lists.newArrayList();
for (WorkerWrapper workerWrapper : zkWorkers.values()) {
workerCount++;
if (workerCount > workerSetupManager.getWorkerSetupData().getMinNumWorkers() &&
workerWrapper.getRunningTasks().isEmpty() &&
System.currentTimeMillis() - workerWrapper.getLastCompletedTaskTime().getMillis()
> config.getMaxWorkerIdleTimeMillisBeforeDeletion()
) {
thoseLazyWorkers.add(workerWrapper);
}
}
AutoScalingData terminated = strategy.terminate(
Lists.transform(
thoseLazyWorkers,
new Function<WorkerWrapper, String>()
{
@Override
public String apply(WorkerWrapper input)
{
return input.getWorker().getIp();
}
}
)
);
if (terminated != null) {
currentlyTerminating.addAll(terminated.getNodeIds());
lastTerminateTime = new DateTime();
}
} else {
Duration durSinceLastTerminate = new Duration(new DateTime(), lastTerminateTime);
if (durSinceLastTerminate.isLongerThan(config.getMaxScalingDuration())) {
log.makeAlert("Worker node termination taking too long")
.addData("millisSinceLastTerminate", durSinceLastTerminate.getMillis())
.addData("terminatingCount", currentlyTerminating.size())
.emit();
}
log.info(
"%s still terminating. Wait for all nodes to terminate before trying again.",
currentlyTerminating
);
}
}
}
);
started = true;
}
catch (Exception e) {
@ -238,8 +164,12 @@ public class RemoteTaskRunner implements TaskRunner
public void stop()
{
try {
for (WorkerWrapper workerWrapper : zkWorkers.values()) {
workerWrapper.close();
if (!started) {
return;
}
for (ZkWorker zkWorker : zkWorkers.values()) {
zkWorker.close();
}
}
catch (Exception e) {
@ -250,119 +180,218 @@ public class RemoteTaskRunner implements TaskRunner
}
}
public boolean hasStarted()
@Override
public Collection<ZkWorker> getWorkers()
{
return started;
}
public int getNumWorkers()
{
return zkWorkers.size();
return zkWorkers.values();
}
@Override
public void run(Task task, TaskContext context, TaskCallback callback)
public Collection<TaskRunnerWorkItem> getRunningTasks()
{
if (tasks.containsKey(task.getId())) {
throw new ISE("Assigned a task[%s] that already exists, WTF is happening?!", task.getId());
return runningTasks.values();
}
TaskWrapper taskWrapper = new TaskWrapper(
task, context, callback, retryPolicyFactory.makeRetryPolicy()
@Override
public Collection<TaskRunnerWorkItem> getPendingTasks()
{
return pendingTasks.values();
}
public ZkWorker findWorkerRunningTask(String taskId)
{
for (ZkWorker zkWorker : zkWorkers.values()) {
if (zkWorker.getRunningTasks().contains(taskId)) {
return zkWorker;
}
}
return null;
}
/**
* A task will be run only if there is no current knowledge in the RemoteTaskRunner of the task.
*
* @param task task to run
* @param callback callback to be called exactly once
*/
@Override
public void run(Task task, TaskCallback callback)
{
if (runningTasks.containsKey(task.getId()) || pendingTasks.containsKey(task.getId())) {
throw new ISE("Assigned a task[%s] that is already running or pending, WTF is happening?!", task.getId());
}
TaskRunnerWorkItem taskRunnerWorkItem = new TaskRunnerWorkItem(
task, callback, retryPolicyFactory.makeRetryPolicy(), new DateTime()
);
tasks.put(taskWrapper.getTask().getId(), taskWrapper);
addPendingTask(taskRunnerWorkItem);
}
private void addPendingTask(final TaskRunnerWorkItem taskRunnerWorkItem)
{
log.info("Added pending task %s", taskRunnerWorkItem.getTask().getId());
pendingTasks.put(taskRunnerWorkItem.getTask().getId(), taskRunnerWorkItem);
runPendingTasks();
}
/**
* This method uses a single threaded executor to extract all pending tasks and attempt to run them. Any tasks that
* are successfully assigned to a worker will be moved from pendingTasks to runningTasks. This method is thread-safe.
* This method should be run each time there is new worker capacity or if new tasks are assigned.
*/
private void runPendingTasks()
{
Future future = runPendingTasksExec.submit(
new Callable<Void>()
{
@Override
public Void call() throws Exception
{
// make a copy of the pending tasks because assignTask may delete tasks from pending and move them
// into running status
List<TaskRunnerWorkItem> copy = Lists.newArrayList(pendingTasks.values());
for (TaskRunnerWorkItem taskWrapper : copy) {
assignTask(taskWrapper);
}
private void assignTask(TaskWrapper taskWrapper)
{
WorkerWrapper workerWrapper = findWorkerRunningTask(taskWrapper);
// If the task already exists, we don't need to announce it
if (workerWrapper != null) {
final Worker worker = workerWrapper.getWorker();
try {
log.info("Worker[%s] is already running task[%s].", worker.getHost(), taskWrapper.getTask().getId());
TaskStatus taskStatus = jsonMapper.readValue(
workerWrapper.getStatusCache()
.getCurrentData(
JOINER.join(config.getStatusPath(), worker.getHost(), taskWrapper.getTask().getId())
)
.getData(),
TaskStatus.class
return null;
}
}
);
if (taskStatus.isComplete()) {
TaskCallback callback = taskWrapper.getCallback();
if (callback != null) {
callback.notify(taskStatus);
try {
future.get();
}
new CleanupPaths(worker.getHost(), taskWrapper.getTask().getId()).run();
}
}
catch (Exception e) {
log.error(e, "Task exists, but hit exception!");
retryTask(new CleanupPaths(worker.getHost(), taskWrapper.getTask().getId()), taskWrapper);
}
} else {
// Announce the task or retry if there is not enough capacity
workerWrapper = findWorkerForTask();
if (workerWrapper != null) {
announceTask(workerWrapper.getWorker(), taskWrapper);
} else {
retryTask(null, taskWrapper);
catch (InterruptedException e) {
log.error(e, "InterruptedException in runPendingTasks()");
throw Throwables.propagate(e);
}
catch (ExecutionException e) {
log.error(e, "ExecutionException in runPendingTasks()");
throw Throwables.propagate(e.getCause());
}
}
/**
* Retries a task that has failed.
* Retries a task by inserting it back into the pending queue after a given delay.
* This method will also clean up any status paths that were associated with the task.
*
* @param pre - A runnable that is executed before the retry occurs
* @param taskWrapper - a container for task properties
* @param taskRunnerWorkItem - the task to retry
* @param workerId - the worker that was previously running this task
*/
private void retryTask(
final Runnable pre,
final TaskWrapper taskWrapper
)
private void retryTask(final TaskRunnerWorkItem taskRunnerWorkItem, final String workerId)
{
final Task task = taskWrapper.getTask();
final RetryPolicy retryPolicy = taskWrapper.getRetryPolicy();
log.info("Registering retry for failed task[%s]", task.getId());
if (retryPolicy.hasExceededRetryThreshold()) {
log.makeAlert("Task exceeded maximum retry count")
.addData("task", task.getId())
.addData("retryCount", retryPolicy.getNumRetries())
.emit();
return;
}
final String taskId = taskRunnerWorkItem.getTask().getId();
log.info("Retry scheduled in %s for %s", taskRunnerWorkItem.getRetryPolicy().getRetryDelay(), taskId);
scheduledExec.schedule(
new Runnable()
{
@Override
public void run()
{
if (pre != null) {
pre.run();
}
if (tasks.containsKey(task.getId())) {
log.info("Retry[%d] for task[%s]", retryPolicy.getNumRetries(), task.getId());
assignTask(taskWrapper);
}
cleanup(workerId, taskId);
addPendingTask(taskRunnerWorkItem);
}
},
retryPolicy.getAndIncrementRetryDelay().getMillis(),
taskRunnerWorkItem.getRetryPolicy().getAndIncrementRetryDelay().getMillis(),
TimeUnit.MILLISECONDS
);
}
/**
* When a new worker appears, listeners are registered for status changes.
* Status changes indicate the creation or completion of task.
* Removes a task from the running queue and clears out the ZK status path of the task.
*
* @param workerId - the worker that was previously running the task
* @param taskId - the task to cleanup
*/
private void cleanup(final String workerId, final String taskId)
{
runningTasks.remove(taskId);
final String statusPath = JOINER.join(config.getStatusPath(), workerId, taskId);
try {
cf.delete().guaranteed().forPath(statusPath);
}
catch (Exception e) {
log.info("Tried to delete status path[%s] that didn't exist! Must've gone away already?", statusPath);
}
}
/**
* Ensures no workers are already running a task before assigning the task to a worker.
* It is possible that a worker is running a task that the RTR has no knowledge of. This occurs when the RTR
* needs to bootstrap after a restart.
*
* @param taskRunnerWorkItem - the task to assign
*/
private void assignTask(TaskRunnerWorkItem taskRunnerWorkItem)
{
try {
final String taskId = taskRunnerWorkItem.getTask().getId();
ZkWorker zkWorker = findWorkerRunningTask(taskId);
// If a worker is already running this task, we don't need to announce it
if (zkWorker != null) {
final Worker worker = zkWorker.getWorker();
log.info("Worker[%s] is already running task[%s].", worker.getHost(), taskId);
runningTasks.put(taskId, pendingTasks.remove(taskId));
log.info("Task %s switched from pending to running", taskId);
} else {
// Nothing running this task, announce it in ZK for a worker to run it
zkWorker = findWorkerForTask();
if (zkWorker != null) {
announceTask(zkWorker.getWorker(), taskRunnerWorkItem);
}
}
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
/**
* Creates a ZK entry under a specific path associated with a worker. The worker is responsible for
* removing the task ZK entry and creating a task status ZK entry.
*
* @param theWorker The worker the task is assigned to
* @param taskRunnerWorkItem The task to be assigned
*/
private void announceTask(Worker theWorker, TaskRunnerWorkItem taskRunnerWorkItem) throws Exception
{
final Task task = taskRunnerWorkItem.getTask();
log.info("Coordinator asking Worker[%s] to add task[%s]", theWorker.getHost(), task.getId());
byte[] rawBytes = jsonMapper.writeValueAsBytes(task);
if (rawBytes.length > config.getMaxNumBytes()) {
throw new ISE("Length of raw bytes for task too large[%,d > %,d]", rawBytes.length, config.getMaxNumBytes());
}
cf.create()
.withMode(CreateMode.EPHEMERAL)
.forPath(
JOINER.join(
config.getTaskPath(),
theWorker.getHost(),
task.getId()
),
rawBytes
);
runningTasks.put(task.getId(), pendingTasks.remove(task.getId()));
log.info("Task %s switched from pending to running", task.getId());
// Syncing state with Zookeeper - don't assign new tasks until the task we just assigned is actually running
// on a worker - this avoids overflowing a worker with tasks
synchronized (statusLock) {
while (findWorkerRunningTask(task.getId()) == null) {
statusLock.wait(config.getTaskAssignmentTimeoutDuration().getMillis());
}
}
}
/**
* When a new worker appears, listeners are registered for status changes associated with tasks assigned to
* the worker. Status changes indicate the creation or completion of a task.
* The RemoteTaskRunner updates state according to these changes.
*
* @param worker - contains metadata for a worker that has appeared in ZK
@ -370,11 +399,9 @@ public class RemoteTaskRunner implements TaskRunner
private void addWorker(final Worker worker)
{
try {
currentlyProvisioning.removeAll(strategy.ipLookup(Arrays.<String>asList(worker.getIp())));
final String workerStatusPath = JOINER.join(config.getStatusPath(), worker.getHost());
final PathChildrenCache statusCache = new PathChildrenCache(cf, workerStatusPath, true);
final WorkerWrapper workerWrapper = new WorkerWrapper(
final ZkWorker zkWorker = new ZkWorker(
worker,
statusCache,
jsonMapper
@ -387,7 +414,7 @@ public class RemoteTaskRunner implements TaskRunner
@Override
public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
{
synchronized (statusLock) {
try {
if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED) ||
event.getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED)) {
@ -412,7 +439,7 @@ public class RemoteTaskRunner implements TaskRunner
}
catch (Exception e) {
log.warn(e, "Worker[%s] wrote bogus status for task: %s", worker.getHost(), taskId);
retryTask(new CleanupPaths(worker.getHost(), taskId), tasks.get(taskId));
retryTask(runningTasks.get(taskId), worker.getHost());
throw Throwables.propagate(e);
}
@ -423,29 +450,38 @@ public class RemoteTaskRunner implements TaskRunner
taskId
);
// Synchronizing state with ZK
synchronized (statusLock) {
statusLock.notify();
}
if (taskStatus.isComplete()) {
// Worker is done with this task
workerWrapper.setLastCompletedTaskTime(new DateTime());
final TaskWrapper taskWrapper = tasks.get(taskId);
if (taskWrapper == null) {
final TaskRunnerWorkItem taskRunnerWorkItem = runningTasks.get(taskId);
if (taskRunnerWorkItem == null) {
log.warn(
"WTF?! Worker[%s] completed a task I didn't know about: %s",
"WTF?! Worker[%s] announcing a status for a task I didn't know about: %s",
worker.getHost(),
taskId
);
} else {
final TaskCallback callback = taskWrapper.getCallback();
}
// Cleanup
if (taskStatus.isComplete()) {
if (taskRunnerWorkItem != null) {
final TaskCallback callback = taskRunnerWorkItem.getCallback();
if (callback != null) {
callback.notify(taskStatus);
}
tasks.remove(taskId);
cf.delete().guaranteed().inBackground().forPath(event.getData().getPath());
}
// Worker is done with this task
zkWorker.setLastCompletedTaskTime(new DateTime());
cleanup(worker.getHost(), taskId);
runPendingTasks();
}
} else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED)) {
final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath());
if (runningTasks.containsKey(taskId)) {
log.info("Task %s just disappeared!", taskId);
retryTask(runningTasks.get(taskId), worker.getHost());
}
}
}
@ -457,76 +493,74 @@ public class RemoteTaskRunner implements TaskRunner
}
}
}
}
);
zkWorkers.put(worker.getHost(), workerWrapper);
zkWorkers.put(worker.getHost(), zkWorker);
statusCache.start();
runPendingTasks();
}
catch (Exception e) {
throw Throwables.propagate(e);
}
}
private WorkerWrapper findWorkerRunningTask(TaskWrapper taskWrapper)
{
for (WorkerWrapper workerWrapper : zkWorkers.values()) {
if (workerWrapper.getRunningTasks().contains(taskWrapper.getTask().getId())) {
return workerWrapper;
}
}
return null;
}
/**
* When a ephemeral worker node disappears from ZK, we have to make sure there are no tasks still assigned
* to the worker. If tasks remain, they are retried.
* When a ephemeral worker node disappears from ZK, incomplete running tasks will be retried by
* the logic in the status listener. We still have to make sure there are no tasks assigned
* to the worker but not yet running.
*
* @param worker - the removed worker
*/
private void removeWorker(final Worker worker)
{
currentlyTerminating.remove(worker.getHost());
WorkerWrapper workerWrapper = zkWorkers.get(worker.getHost());
if (workerWrapper != null) {
ZkWorker zkWorker = zkWorkers.get(worker.getHost());
if (zkWorker != null) {
try {
Set<String> tasksToRetry = Sets.newHashSet(workerWrapper.getRunningTasks());
tasksToRetry.addAll(cf.getChildren().forPath(JOINER.join(config.getTaskPath(), worker.getHost())));
Set<String> tasksPending = Sets.newHashSet(
cf.getChildren()
.forPath(JOINER.join(config.getTaskPath(), worker.getHost()))
);
log.info("%s had %d tasks pending", worker.getHost(), tasksPending.size());
for (String taskId : tasksToRetry) {
TaskWrapper taskWrapper = tasks.get(taskId);
if (taskWrapper != null) {
retryTask(new CleanupPaths(worker.getHost(), taskId), tasks.get(taskId));
for (String taskId : tasksPending) {
TaskRunnerWorkItem taskRunnerWorkItem = pendingTasks.get(taskId);
if (taskRunnerWorkItem != null) {
cf.delete().guaranteed().forPath(JOINER.join(config.getTaskPath(), worker.getHost(), taskId));
retryTask(taskRunnerWorkItem, worker.getHost());
} else {
log.warn("RemoteTaskRunner has no knowledge of pending task %s", taskId);
}
}
workerWrapper.getStatusCache().close();
zkWorker.getStatusCache().close();
}
catch (Exception e) {
log.error(e, "Failed to cleanly remove worker[%s]");
}
throw Throwables.propagate(e);
}
finally {
zkWorkers.remove(worker.getHost());
}
}
}
private WorkerWrapper findWorkerForTask()
private ZkWorker findWorkerForTask()
{
try {
final MinMaxPriorityQueue<WorkerWrapper> workerQueue = MinMaxPriorityQueue.<WorkerWrapper>orderedBy(
new Comparator<WorkerWrapper>()
final MinMaxPriorityQueue<ZkWorker> workerQueue = MinMaxPriorityQueue.<ZkWorker>orderedBy(
new Comparator<ZkWorker>()
{
@Override
public int compare(WorkerWrapper w1, WorkerWrapper w2)
public int compare(ZkWorker w1, ZkWorker w2)
{
return -Ints.compare(w1.getRunningTasks().size(), w2.getRunningTasks().size());
}
}
).create(
FunctionalIterable.create(zkWorkers.values()).filter(
new Predicate<WorkerWrapper>()
new Predicate<ZkWorker>()
{
@Override
public boolean apply(WorkerWrapper input)
public boolean apply(ZkWorker input)
{
return (!input.isAtCapacity() &&
input.getWorker()
@ -538,28 +572,7 @@ public class RemoteTaskRunner implements TaskRunner
);
if (workerQueue.isEmpty()) {
log.info("Worker nodes do not have capacity to run any more tasks!");
if (currentlyProvisioning.isEmpty()) {
AutoScalingData provisioned = strategy.provision();
if (provisioned != null) {
currentlyProvisioning.addAll(provisioned.getNodeIds());
lastProvisionTime = new DateTime();
}
} else {
Duration durSinceLastProvision = new Duration(new DateTime(), lastProvisionTime);
if (durSinceLastProvision.isLongerThan(config.getMaxScalingDuration())) {
log.makeAlert("Worker node provisioning taking too long")
.addData("millisSinceLastProvision", durSinceLastProvision.getMillis())
.addData("provisioningCount", currentlyProvisioning.size())
.emit();
}
log.info(
"%s still provisioning. Wait for all provisioned nodes to complete before requesting new worker.",
currentlyProvisioning
);
}
log.info("Worker nodes %s do not have capacity to run any more tasks!", zkWorkers.values());
return null;
}
@ -569,82 +582,4 @@ public class RemoteTaskRunner implements TaskRunner
throw Throwables.propagate(e);
}
}
/**
* Creates a ZK entry under a specific path associated with a worker. The worker is responsible for
* removing the task ZK entry and creating a task status ZK entry.
*
* @param theWorker The worker the task is assigned to
* @param taskWrapper The task to be assigned
*/
private void announceTask(Worker theWorker, TaskWrapper taskWrapper)
{
synchronized (statusLock) {
final Task task = taskWrapper.getTask();
final TaskContext taskContext = taskWrapper.getTaskContext();
try {
log.info("Coordinator asking Worker[%s] to add task[%s]", theWorker.getHost(), task.getId());
tasks.put(task.getId(), taskWrapper);
byte[] rawBytes = jsonMapper.writeValueAsBytes(new TaskHolder(task, taskContext));
if (rawBytes.length > config.getMaxNumBytes()) {
throw new ISE("Length of raw bytes for task too large[%,d > %,d]", rawBytes.length, config.getMaxNumBytes());
}
cf.create()
.withMode(CreateMode.EPHEMERAL)
.forPath(
JOINER.join(
config.getTaskPath(),
theWorker.getHost(),
task.getId()
),
jsonMapper.writeValueAsBytes(new TaskHolder(task, taskContext))
);
// Syncing state with Zookeeper
while (findWorkerRunningTask(taskWrapper) == null) {
statusLock.wait(config.getTaskAssignmentTimeoutDuration().getMillis());
}
}
catch (Exception e) {
log.error(e, "Exception creating task[%s] for worker node[%s]", task.getId(), theWorker.getHost());
throw Throwables.propagate(e);
}
}
}
private class CleanupPaths implements Runnable
{
private final String workerId;
private final String taskId;
private CleanupPaths(String workerId, String taskId)
{
this.workerId = workerId;
this.taskId = taskId;
}
@Override
public void run()
{
try {
final String statusPath = JOINER.join(config.getStatusPath(), workerId, taskId);
cf.delete().guaranteed().forPath(statusPath);
}
catch (Exception e) {
log.warn("Tried to delete a status path that didn't exist! Must've gone away already?");
}
try {
final String taskPath = JOINER.join(config.getTaskPath(), workerId, taskId);
cf.delete().guaranteed().forPath(taskPath);
}
catch (Exception e) {
log.warn("Tried to delete a task path that didn't exist! Must've gone away already?");
}
}
}
}

View File

@ -44,6 +44,11 @@ public class RetryPolicy
this.retryCount = 0;
}
public Duration getRetryDelay()
{
return currRetryDelay;
}
public Duration getAndIncrementRetryDelay()
{
Duration retVal = new Duration(currRetryDelay);

View File

@ -0,0 +1,22 @@
package com.metamx.druid.merger.coordinator;
public class TaskExistsException extends RuntimeException
{
private final String taskId;
public TaskExistsException(String taskId, Throwable t)
{
super(String.format("Task exists: %s", taskId), t);
this.taskId = taskId;
}
public TaskExistsException(String taskId)
{
this(taskId, null);
}
public String getTaskId()
{
return taskId;
}
}

View File

@ -0,0 +1,405 @@
package com.metamx.druid.merger.coordinator;
import com.google.common.base.Function;
import com.google.common.base.Optional;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.metamx.common.IAE;
import com.metamx.common.guava.Comparators;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.emitter.EmittingLogger;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.NavigableSet;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.locks.ReentrantLock;
/**
* Remembers which tasks have locked which intervals. Tasks are permitted to lock an interval if no other task
* outside their group has locked an overlapping interval for the same datasource. When a task locks an interval,
* it is assigned a version string that it can use to publish segments.
*/
public class TaskLockbox
{
// Datasource -> Interval -> Tasks + TaskLock
private final Map<String, NavigableMap<Interval, TaskLockPosse>> running = Maps.newHashMap();
private final TaskStorage taskStorage;
private final ReentrantLock giant = new ReentrantLock();
private static final EmittingLogger log = new EmittingLogger(TaskLockbox.class);
public TaskLockbox(TaskStorage taskStorage)
{
this.taskStorage = taskStorage;
}
/**
* Attempt to lock a task, without removing it from the queue. Equivalent to the long form of {@code tryLock}
* with no preferred version.
*
* @param task task to attempt to lock
*
* @return lock version if lock was acquired, absent otherwise
*/
public Optional<TaskLock> tryLock(final Task task, final Interval interval)
{
return tryLock(task, interval, Optional.<String>absent());
}
/**
* Attempt to lock a task, without removing it from the queue. Can safely be called multiple times on the same task.
* This method will attempt to assign version strings that obey the invariant that every version string is
* lexicographically greater than any other version string previously assigned to the same interval. This invariant
* is only mostly guaranteed, however; we assume clock monotonicity and we assume that callers specifying
* {@code preferredVersion} are doing the right thing.
*
* @param task task to attempt to lock
* @param preferredVersion use this version string if one has not yet been assigned
*
* @return lock version if lock was acquired, absent otherwise
*/
public Optional<TaskLock> tryLock(final Task task, final Interval interval, final Optional<String> preferredVersion)
{
giant.lock();
try {
if(task.getFixedInterval().isPresent() && !task.getFixedInterval().get().equals(interval)) {
// Task may only lock its fixed interval, if present
throw new IAE("Task must lock its fixed interval: %s", task.getId());
}
final String dataSource = task.getDataSource();
final List<TaskLockPosse> foundPosses = findLockPossesForInterval(dataSource, interval);
final TaskLockPosse posseToUse;
if (foundPosses.size() > 1) {
// Too many existing locks.
return Optional.absent();
} else if (foundPosses.size() == 1) {
// One existing lock -- check if we can add to it.
final TaskLockPosse foundPosse = Iterables.getOnlyElement(foundPosses);
if (foundPosse.getTaskLock().getInterval().contains(interval) && foundPosse.getTaskLock().getGroupId().equals(task.getGroupId())) {
posseToUse = foundPosse;
} else {
return Optional.absent();
}
} else {
// No existing locks. We can make a new one.
if (!running.containsKey(dataSource)) {
running.put(dataSource, new TreeMap<Interval, TaskLockPosse>(Comparators.intervalsByStartThenEnd()));
}
// Create new TaskLock and assign it a version.
// Assumption: We'll choose a version that is greater than any previously-chosen version for our interval. (This
// may not always be true, unfortunately. See below.)
final String version;
if (preferredVersion.isPresent()) {
// We have a preferred version. We'll trust our caller to not break our ordering assumptions and just use it.
version = preferredVersion.get();
} else {
// We are running under an interval lock right now, so just using the current time works as long as we can trust
// our clock to be monotonic and have enough resolution since the last time we created a TaskLock for the same
// interval. This may not always be true; to assure it we would need to use some method of timekeeping other
// than the wall clock.
version = new DateTime().toString();
}
posseToUse = new TaskLockPosse(new TaskLock(task.getGroupId(), dataSource, interval, version));
running.get(dataSource)
.put(interval, posseToUse);
log.info("Created new TaskLockPosse: %s", posseToUse);
}
// Add to existing TaskLockPosse, if necessary
if (posseToUse.getTaskIds().add(task.getId())) {
log.info("Added task[%s] to TaskLock[%s]", task.getId(), posseToUse.getTaskLock().getGroupId());
// Best effort to update task storage facility
try {
taskStorage.addLock(task.getId(), posseToUse.getTaskLock());
} catch(Exception e) {
log.makeAlert("Failed to persist lock in storage")
.addData("task", task.getId())
.addData("dataSource", posseToUse.getTaskLock().getDataSource())
.addData("interval", posseToUse.getTaskLock().getInterval())
.addData("version", posseToUse.getTaskLock().getVersion())
.emit();
}
} else {
log.info("Task[%s] already present in TaskLock[%s]", task.getId(), posseToUse.getTaskLock().getGroupId());
}
return Optional.of(posseToUse.getTaskLock());
}
finally {
giant.unlock();
}
}
/**
* Return the currently-active locks for some task.
*
* @param task task for which to locate locks
*/
public List<TaskLock> findLocksForTask(final Task task)
{
giant.lock();
try {
return Lists.transform(
findLockPossesForTask(task), new Function<TaskLockPosse, TaskLock>()
{
@Override
public TaskLock apply(TaskLockPosse taskLockPosse)
{
return taskLockPosse.getTaskLock();
}
}
);
} finally {
giant.unlock();
}
}
/**
* Release lock held for a task on a particular interval. Does nothing if the task does not currently
* hold the mentioned lock.
*
* @param task task to unlock
* @param interval interval to unlock
*/
public void unlock(final Task task, final Interval interval)
{
giant.lock();
try {
final String dataSource = task.getDataSource();
final NavigableMap<Interval, TaskLockPosse> dsRunning = running.get(dataSource);
// So we can alert if tasks try to release stuff they don't have
boolean removed = false;
if(dsRunning != null) {
final TaskLockPosse taskLockPosse = dsRunning.get(interval);
if(taskLockPosse != null) {
final TaskLock taskLock = taskLockPosse.getTaskLock();
// Remove task from live list
log.info("Removing task[%s] from TaskLock[%s]", task.getId(), taskLock.getGroupId());
removed = taskLockPosse.getTaskIds().remove(task.getId());
if (taskLockPosse.getTaskIds().isEmpty()) {
log.info("TaskLock is now empty: %s", taskLock);
running.get(dataSource).remove(taskLock.getInterval());
}
if (running.get(dataSource).size() == 0) {
running.remove(dataSource);
}
// Best effort to remove lock from storage
try {
taskStorage.removeLock(task.getId(), taskLock);
} catch(Exception e) {
log.makeAlert(e, "Failed to clean up lock from storage")
.addData("task", task.getId())
.addData("dataSource", taskLock.getDataSource())
.addData("interval", taskLock.getInterval())
.addData("version", taskLock.getVersion())
.emit();
}
}
}
if(!removed) {
log.makeAlert("Lock release without acquire")
.addData("task", task.getId())
.addData("interval", interval)
.emit();
}
} finally {
giant.unlock();
}
}
/**
* Release all locks for a task. Does nothing if the task is not currently locked.
*
* @param task task to unlock
*/
public void unlock(final Task task)
{
giant.lock();
try {
for(final TaskLockPosse taskLockPosse : findLockPossesForTask(task)) {
unlock(task, taskLockPosse.getTaskLock().getInterval());
}
}
finally {
giant.unlock();
}
}
/**
* Removes all locks from this lockbox.
*/
public void clear()
{
giant.lock();
try {
running.clear();
} finally {
giant.unlock();
}
}
/**
* Return the currently-active lock posses for some task.
*
* @param task task for which to locate locks
*/
private List<TaskLockPosse> findLockPossesForTask(final Task task)
{
giant.lock();
try {
final Iterable<TaskLockPosse> searchSpace;
if (task.getFixedInterval().isPresent()) {
// Narrow down search using findLockPossesForInterval
searchSpace = findLockPossesForInterval(task.getDataSource(), task.getFixedInterval().get());
} else {
// Scan through all locks for this datasource
final NavigableMap<Interval, TaskLockPosse> dsRunning = running.get(task.getDataSource());
if(dsRunning == null) {
searchSpace = ImmutableList.of();
} else {
searchSpace = dsRunning.values();
}
}
return ImmutableList.copyOf(
Iterables.filter(
searchSpace, new Predicate<TaskLockPosse>()
{
@Override
public boolean apply(TaskLockPosse taskLock)
{
return taskLock.getTaskIds().contains(task.getId());
}
}
)
);
}
finally {
giant.unlock();
}
}
/**
* Return all locks that overlap some search interval.
*/
private List<TaskLockPosse> findLockPossesForInterval(final String dataSource, final Interval interval)
{
giant.lock();
try {
final NavigableMap<Interval, TaskLockPosse> dsRunning = running.get(dataSource);
if (dsRunning == null) {
// No locks at all
return Collections.emptyList();
} else {
// Tasks are indexed by locked interval, which are sorted by interval start. Intervals are non-overlapping, so:
final NavigableSet<Interval> dsLockbox = dsRunning.navigableKeySet();
final Iterable<Interval> searchIntervals = Iterables.concat(
// Single interval that starts at or before ours
Collections.singletonList(dsLockbox.floor(new Interval(interval.getStart(), new DateTime(Long.MAX_VALUE)))),
// All intervals that start somewhere between our start instant (exclusive) and end instant (exclusive)
dsLockbox.subSet(
new Interval(interval.getStart(), new DateTime(Long.MAX_VALUE)),
false,
new Interval(interval.getEnd(), interval.getEnd()),
false
)
);
return Lists.newArrayList(
FunctionalIterable
.create(searchIntervals)
.filter(
new Predicate<Interval>()
{
@Override
public boolean apply(@Nullable Interval searchInterval)
{
return searchInterval != null && searchInterval.overlaps(interval);
}
}
)
.transform(
new Function<Interval, TaskLockPosse>()
{
@Override
public TaskLockPosse apply(Interval interval)
{
return dsRunning.get(interval);
}
}
)
);
}
}
finally {
giant.unlock();
}
}
private static class TaskLockPosse
{
final private TaskLock taskLock;
final private Set<String> taskIds;
public TaskLockPosse(TaskLock taskLock)
{
this.taskLock = taskLock;
taskIds = Sets.newHashSet();
}
public TaskLock getTaskLock()
{
return taskLock;
}
public Set<String> getTaskIds()
{
return taskIds;
}
}
}

View File

@ -25,8 +25,11 @@ import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.initialization.Initialization;
import com.metamx.druid.initialization.ServiceDiscoveryConfig;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
import com.metamx.druid.merger.coordinator.exec.TaskConsumer;
import com.metamx.druid.merger.coordinator.scaling.ResourceManagementScheduler;
import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerFactory;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.ServiceEmitter;
import com.netflix.curator.framework.CuratorFramework;
@ -40,26 +43,33 @@ import java.util.concurrent.locks.ReentrantLock;
/**
* Encapsulates the indexer leadership lifecycle.
*/
public class TaskMaster
public class TaskMasterLifecycle
{
private final LeaderSelector leaderSelector;
private final ReentrantLock giant = new ReentrantLock();
private final Condition mayBeStopped = giant.newCondition();
private final TaskQueue taskQueue;
private final TaskToolbox taskToolbox;
private volatile boolean leading = false;
private volatile TaskRunner theRunner;
private static final EmittingLogger log = new EmittingLogger(TaskMaster.class);
private static final EmittingLogger log = new EmittingLogger(TaskMasterLifecycle.class);
public TaskMaster(
final TaskQueue queue,
public TaskMasterLifecycle(
final TaskQueue taskQueue,
final TaskToolbox taskToolbox,
final IndexerCoordinatorConfig indexerCoordinatorConfig,
final ServiceDiscoveryConfig serviceDiscoveryConfig,
final MergerDBCoordinator mergerDBCoordinator,
final TaskRunnerFactory runnerFactory,
final ResourceManagementSchedulerFactory managementSchedulerFactory,
final CuratorFramework curator,
final ServiceEmitter emitter
)
{
this.taskQueue = taskQueue;
this.taskToolbox = taskToolbox;
this.leaderSelector = new LeaderSelector(
curator, indexerCoordinatorConfig.getLeaderLatchPath(), new LeaderSelectorListener()
{
@ -71,31 +81,45 @@ public class TaskMaster
try {
log.info("By the power of Grayskull, I have the power!");
final TaskRunner runner = runnerFactory.build();
final TaskConsumer consumer = new TaskConsumer(queue, runner, mergerDBCoordinator, emitter);
final TaskRunner taskRunner = runnerFactory.build();
theRunner = taskRunner;
final ResourceManagementScheduler scheduler = managementSchedulerFactory.build(theRunner);
final TaskConsumer taskConsumer = new TaskConsumer(
taskQueue,
taskRunner,
taskToolbox,
emitter
);
// Bootstrap task queue and task lockbox (load state stuff from the database)
taskQueue.bootstrap();
// Sensible order to start stuff:
final Lifecycle leaderLifecycle = new Lifecycle();
leaderLifecycle.addManagedInstance(queue);
leaderLifecycle.addManagedInstance(runner);
leaderLifecycle.addManagedInstance(taskQueue);
leaderLifecycle.addManagedInstance(taskRunner);
Initialization.makeServiceDiscoveryClient(curator, serviceDiscoveryConfig, leaderLifecycle);
leaderLifecycle.addManagedInstance(consumer);
leaderLifecycle.start();
leaderLifecycle.addManagedInstance(taskConsumer);
leading = true;
try {
leaderLifecycle.start();
while (leading) {
mayBeStopped.await();
}
} finally {
}
finally {
log.info("Bowing out!");
leaderLifecycle.stop();
}
} catch(Exception e) {
}
catch (Exception e) {
log.makeAlert(e, "Failed to lead").emit();
throw Throwables.propagate(e);
} finally {
}
finally {
giant.unlock();
}
}
@ -181,4 +205,19 @@ public class TaskMaster
throw Throwables.propagate(e);
}
}
public TaskRunner getTaskRunner()
{
return theRunner;
}
public TaskQueue getTaskQueue()
{
return taskQueue;
}
public TaskToolbox getTaskToolbox()
{
return taskToolbox;
}
}

View File

@ -19,75 +19,146 @@
package com.metamx.druid.merger.coordinator;
import com.google.common.base.Function;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
import com.google.common.base.Throwables;
import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Multimap;
import com.google.common.collect.Ordering;
import com.google.common.primitives.Booleans;
import com.metamx.common.Pair;
import com.metamx.common.guava.Comparators;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.task.Task;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import com.metamx.emitter.EmittingLogger;
import javax.annotation.Nullable;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.NavigableMap;
import java.util.NavigableSet;
import java.util.TreeMap;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
/**
* Interface between task producers and task consumers.
*
* <p/>
* The queue accepts tasks from producers using {@link #add} and delivers tasks to consumers using either
* {@link #take} or {@link #poll}. Ordering is mostly-FIFO, with deviations when the natural next task would conflict
* with a currently-running task. In that case, tasks are skipped until a runnable one is found.
*
* To manage locking, the queue keeps track of currently-running tasks as {@link TaskGroup} objects. The idea is that
* only one TaskGroup can be running on a particular dataSource + interval, and that TaskGroup has a single version
* string that all tasks in the group must use to publish segments. Tasks in the same TaskGroup may run concurrently.
*
* For persistence, the queue saves new tasks from {@link #add} and task status updates from {@link #done} using a
* {@link TaskStorage} object.
*
* <p/>
* To manage locking, the queue keeps track of currently-running tasks as {@link com.metamx.druid.merger.common.TaskLock} objects. The idea is that
* only one TaskLock can be running on a particular dataSource + interval, and that TaskLock has a single version
* string that all tasks in the group must use to publish segments. Tasks in the same TaskLock may run concurrently.
* <p/>
* For persistence, the queue saves new tasks from {@link #add} and task status updates from {@link #notify} using a
* {@link TaskStorage} obj
* <p/>
* To support leader election of our containing system, the queue can be stopped (in which case it will not accept
* any new tasks, or hand out any more tasks, until started again).
*/
public class TaskQueue
{
private final List<Task> queue = Lists.newLinkedList();
private final Map<String, NavigableMap<Interval, TaskGroup>> running = Maps.newHashMap();
private final TaskStorage taskStorage;
private final TaskLockbox taskLockbox;
private final ReentrantLock giant = new ReentrantLock();
private final Condition workMayBeAvailable = giant.newCondition();
private volatile boolean active = false;
private static final Logger log = new Logger(TaskQueue.class);
private static final EmittingLogger log = new EmittingLogger(TaskQueue.class);
public TaskQueue(TaskStorage taskStorage)
public TaskQueue(TaskStorage taskStorage, TaskLockbox taskLockbox)
{
this.taskStorage = Preconditions.checkNotNull(taskStorage, "taskStorage");
this.taskLockbox = Preconditions.checkNotNull(taskLockbox, "taskLockbox");
}
/**
* Starts this task queue. Loads tasks from our task storage facility and allows {@link #add(Task)} to accept
* new tasks. This should not be called on an already-started queue.
* Bootstraps this task queue and associated task lockbox. Clears the lockbox before running. Should be called
* while the queue is stopped. It is not a good idea to start the queue if this method fails.
*/
public void bootstrap()
{
giant.lock();
try {
Preconditions.checkState(!active, "queue must be stopped");
log.info("Bootstrapping queue (and associated lockbox)");
queue.clear();
taskLockbox.clear();
// Add running tasks to the queue
final List<Task> runningTasks = taskStorage.getRunningTasks();
for(final Task task : runningTasks) {
queue.add(task);
}
// Get all locks, along with which tasks they belong to
final Multimap<TaskLock, Task> tasksByLock = ArrayListMultimap.create();
for(final Task runningTask : runningTasks) {
for(final TaskLock taskLock : taskStorage.getLocks(runningTask.getId())) {
tasksByLock.put(taskLock, runningTask);
}
}
// Sort locks by version
final Ordering<TaskLock> byVersionOrdering = new Ordering<TaskLock>()
{
@Override
public int compare(TaskLock left, TaskLock right)
{
return left.getVersion().compareTo(right.getVersion());
}
};
// Acquire as many locks as possible, in version order
for(final Map.Entry<TaskLock, Task> taskAndLock : tasksByLock.entries()) {
final Task task = taskAndLock.getValue();
final TaskLock savedTaskLock = taskAndLock.getKey();
final Optional<TaskLock> acquiredTaskLock = taskLockbox.tryLock(
task,
savedTaskLock.getInterval(),
Optional.of(savedTaskLock.getVersion())
);
if(acquiredTaskLock.isPresent() && savedTaskLock.getVersion().equals(acquiredTaskLock.get().getVersion())) {
log.info(
"Reacquired lock on interval[%s] version[%s] for task: %s",
savedTaskLock.getInterval(),
savedTaskLock.getVersion(),
task.getId()
);
} else if(acquiredTaskLock.isPresent()) {
log.info(
"Could not reacquire lock on interval[%s] version[%s] (got version[%s] instead) for task: %s",
savedTaskLock.getInterval(),
savedTaskLock.getVersion(),
acquiredTaskLock.get().getVersion(),
task.getId()
);
} else {
log.info(
"Could not reacquire lock on interval[%s] version[%s] for task: %s",
savedTaskLock.getInterval(),
savedTaskLock.getVersion(),
task.getId()
);
}
}
log.info("Bootstrapped %,d tasks. Ready to go!", runningTasks.size());
} finally {
giant.unlock();
}
}
/**
* Starts this task queue. Allows {@link #add(Task)} to accept new tasks. This should not be called on
* an already-started queue.
*/
@LifecycleStart
public void start()
@ -95,62 +166,9 @@ public class TaskQueue
giant.lock();
try {
Preconditions.checkState(!active, "queue was already started!");
Preconditions.checkState(queue.isEmpty(), "queue must be empty!");
Preconditions.checkState(running.isEmpty(), "running list must be empty!");
// XXX - We might want a TaskStorage API that does this, but including the Pair type in the interface seems clumsy.
final List<Pair<Task, String>> runningTasks = Lists.transform(
taskStorage.getRunningTasks(),
new Function<Task, Pair<Task, String>>()
{
@Override
public Pair<Task, String> apply(Task task)
{
return Pair.of(task, taskStorage.getVersion(task.getId()).orNull());
}
}
);
// Sort by version, with nulls last
final Ordering<Pair<Task, String>> byVersionOrdering = new Ordering<Pair<Task, String>>()
{
final private Ordering<String> baseOrdering = Ordering.natural().nullsLast();
@Override
public int compare(
Pair<Task, String> left, Pair<Task, String> right
)
{
return baseOrdering.compare(left.rhs, right.rhs);
}
};
for(final Pair<Task, String> taskAndVersion : byVersionOrdering.sortedCopy(runningTasks)) {
final Task task = taskAndVersion.lhs;
final String preferredVersion = taskAndVersion.rhs;
queue.add(task);
if(preferredVersion != null) {
final Optional<String> version = tryLock(task, Optional.of(preferredVersion));
log.info(
"Bootstrapped task[%s] with preferred version[%s]: %s",
task.getId(),
preferredVersion,
version.isPresent() ? String.format("locked with version[%s]", version.get()) : "not lockable"
);
} else {
log.info("Bootstrapped task[%s] with no preferred version", task.getId());
}
}
log.info("Bootstrapped %,d tasks. Ready to go!", runningTasks.size());
Preconditions.checkState(!active, "queue must be stopped");
active = true;
workMayBeAvailable.signalAll();
}
finally {
@ -168,12 +186,12 @@ public class TaskQueue
giant.lock();
try {
log.info("Naptime! Shutting down until we are started again.");
queue.clear();
running.clear();
taskLockbox.clear();
active = false;
} finally {
}
finally {
giant.unlock();
}
}
@ -182,20 +200,35 @@ public class TaskQueue
* Adds some work to the queue and the underlying task storage facility with a generic "running" status.
*
* @param task task to add
*
* @return true
*/
public boolean add(Task task)
public boolean add(final Task task)
{
giant.lock();
try {
Preconditions.checkState(active, "Queue is not active!");
// If this throws with any sort of exception, including TaskExistsException, we don't want to
// insert the task into our queue.
try {
taskStorage.insert(task, TaskStatus.running(task.getId()));
} catch(TaskExistsException e) {
log.warn("Attempt to add task twice: %s", task.getId());
throw Throwables.propagate(e);
}
queue.add(task);
workMayBeAvailable.signalAll();
// Attempt to add this task to a running task group. Silently continue if this is not possible.
// The main reason this is here is so when subtasks are added, they end up in the same task group
// as their parent whenever possible.
if(task.getFixedInterval().isPresent()) {
taskLockbox.tryLock(task, task.getFixedInterval().get());
}
return true;
}
finally {
@ -205,48 +238,58 @@ public class TaskQueue
/**
* Locks and returns next doable work from the queue. Blocks if there is no doable work.
*
* @return runnable task
*/
public VersionedTaskWrapper take() throws InterruptedException
public Task take() throws InterruptedException
{
giant.lock();
try {
VersionedTaskWrapper taskWrapper;
Task task;
while((taskWrapper = poll()) == null) {
log.info("Waiting for work...");
workMayBeAvailable.await();
while ((task = poll()) == null) {
// awaitNanos because work may become available without this condition signalling,
// due to other folks messing with the taskLockbox
workMayBeAvailable.awaitNanos(1000000000L /* 1 second */);
}
return taskWrapper;
} finally {
return task;
}
finally {
giant.unlock();
}
}
/**
* Locks and removes next doable work from the queue. Returns null if there is no doable work.
*
* @return runnable task or null
*/
public VersionedTaskWrapper poll()
public Task poll()
{
giant.lock();
try {
log.info("Checking for doable work");
for(final Task task : queue) {
final Optional<String> maybeVersion = tryLock(task);
if(maybeVersion.isPresent()) {
Preconditions.checkState(active, "wtf? Found task when inactive");
taskStorage.setVersion(task.getId(), maybeVersion.get());
for (final Task task : queue) {
if(task.getFixedInterval().isPresent()) {
// If this task has a fixed interval, attempt to lock it right now.
final Optional<TaskLock> maybeLock = taskLockbox.tryLock(task, task.getFixedInterval().get());
if(maybeLock.isPresent()) {
log.info("Task claimed with fixed interval lock: %s", task.getId());
queue.remove(task);
log.info("Task claimed: %s", task);
return new VersionedTaskWrapper(task, maybeVersion.get());
return task;
}
} else {
// No fixed interval. Let's just run this and see what happens.
log.info("Task claimed with no fixed interval lock: %s", task.getId());
queue.remove(task);
return task;
}
}
log.info("No doable work found.");
return null;
}
finally {
@ -255,313 +298,62 @@ public class TaskQueue
}
/**
* Unlock some work. Does not update the task storage facility. Throws an exception if this work is not currently
* running.
* Notify this queue that some task has an updated status. If this update is valid, the status will be persisted in
* the task storage facility. If the status is a completed status, the task will be unlocked and no further
* updates will be accepted.
*
* @param task task to unlock
* @throws IllegalStateException if task is not currently locked
* @param task task to update
* @param taskStatus new task status
*
* @throws NullPointerException if task or status is null
* @throws IllegalArgumentException if the task ID does not match the status ID
* @throws IllegalStateException if this queue is currently shut down
*/
private void unlock(final Task task)
{
giant.lock();
try {
final String dataSource = task.getDataSource();
final Interval interval = task.getInterval();
final List<TaskGroup> maybeTaskGroup = Lists.newArrayList(
FunctionalIterable.create(findLocks(dataSource, interval))
.filter(
new Predicate<TaskGroup>()
{
@Override
public boolean apply(TaskGroup taskGroup)
{
return taskGroup.getTaskSet().contains(task);
}
}
)
);
final TaskGroup taskGroup;
if(maybeTaskGroup.size() == 1) {
taskGroup = maybeTaskGroup.get(0);
} else {
throw new IllegalStateException(String.format("Task must be running: %s", task.getId()));
}
// Remove task from live list
log.info("Removing task[%s] from TaskGroup[%s]", task.getId(), taskGroup.getGroupId());
taskGroup.getTaskSet().remove(task);
if(taskGroup.getTaskSet().size() == 0) {
log.info("TaskGroup complete: %s", taskGroup);
running.get(dataSource).remove(taskGroup.getInterval());
}
if(running.get(dataSource).size() == 0) {
running.remove(dataSource);
}
workMayBeAvailable.signalAll();
}
finally {
giant.unlock();
}
}
/**
* Unlock some task and update its status in the task storage facility. If "status" is a continuation status (i.e.
* it has nextTasks) this will add the next tasks to the queue with a generic running status.
*
* @param task task to unlock
* @param status task completion status; must not be runnable
* @throws IllegalStateException if task is not currently running, or if status is runnable
*/
public void done(final Task task, final TaskStatus status)
public void notify(final Task task, final TaskStatus taskStatus)
{
giant.lock();
try {
Preconditions.checkNotNull(task, "task");
Preconditions.checkNotNull(taskStatus, "status");
Preconditions.checkState(active, "Queue is not active!");
Preconditions.checkState(!status.isRunnable(), "status must no longer be runnable");
Preconditions.checkState(
task.getId().equals(status.getId()),
Preconditions.checkArgument(
task.getId().equals(taskStatus.getId()),
"Mismatching task ids[%s/%s]",
task.getId(),
status.getId()
taskStatus.getId()
);
// Might change on continuation failure
TaskStatus actualStatus = status;
// Add next tasks, if any
// Save status to DB
boolean didPersistStatus = false;
try {
for(final Task nextTask : status.getNextTasks()) {
add(nextTask);
tryLock(nextTask);
final Optional<TaskStatus> previousStatus = taskStorage.getStatus(task.getId());
if (!previousStatus.isPresent() || !previousStatus.get().isRunnable()) {
log.makeAlert("Ignoring notification for dead task").addData("task", task.getId()).emit();
return;
} else {
taskStorage.setStatus(taskStatus);
didPersistStatus = true;
}
} catch(Exception e) {
log.error(e, "Failed to continue task: %s", task.getId());
actualStatus = TaskStatus.failure(task.getId());
log.makeAlert(e, "Failed to persist status for task")
.addData("task", task.getId())
.addData("statusCode", taskStatus.getStatusCode())
.emit();
}
unlock(task);
// Update status in DB
taskStorage.setStatus(task.getId(), actualStatus);
if(taskStatus.isComplete()) {
if(didPersistStatus) {
log.info("Task done: %s", task);
}
finally {
giant.unlock();
}
}
/**
* Returns task status for a particular task ID. May collapse "continued" statuses down to "success" or "failure"
* if appropriate.
*/
public Optional<TaskStatus> getStatus(final String taskid)
{
giant.lock();
try {
final Optional<TaskStatus> statusOptional = taskStorage.getStatus(taskid);
if(statusOptional.isPresent()) {
// See if we can collapse this down
return Optional.of(collapseStatus(statusOptional.get()));
taskLockbox.unlock(task);
workMayBeAvailable.signalAll();
} else {
return statusOptional;
// TODO: This could be a task-status-submission retry queue instead of retrying the entire task,
// TODO: which is heavy and probably not necessary.
log.warn("Status could not be persisted! Reinserting task: %s", task.getId());
queue.add(task);
}
}
finally {
giant.unlock();
}
}
private TaskStatus collapseStatus(TaskStatus status)
{
if (status.isContinued()) {
int nSubtasks = 0;
int nSuccesses = 0;
List<DataSegment> segments = Lists.newArrayList();
for(final Task subtask : status.getNextTasks()) {
final TaskStatus subtaskStatus = collapseStatus(taskStorage.getStatus(subtask.getId()).get());
nSubtasks ++;
if (subtaskStatus.isFailure()) {
return TaskStatus.failure(status.getId());
} else if (subtaskStatus.isSuccess()) {
nSuccesses++;
segments.addAll(subtaskStatus.getSegments());
}
}
if (nSubtasks == nSuccesses) {
return TaskStatus.success(status.getId(), segments);
}
}
// unable to collapse it down
return status;
}
/**
* Attempt to lock a task, without removing it from the queue. Can safely be called multiple times on the same task.
*
* @param task task to attempt to lock
* @return lock version if lock was acquired, absent otherwise
*/
private Optional<String> tryLock(final Task task)
{
return tryLock(task, Optional.<String>absent());
}
/**
* Attempt to lock a task, without removing it from the queue. Can safely be called multiple times on the same task.
*
* @param task task to attempt to lock
* @param preferredVersion use this version if possible (no guarantees, though!)
* @return lock version if lock was acquired, absent otherwise
*/
private Optional<String> tryLock(final Task task, final Optional<String> preferredVersion)
{
giant.lock();
try {
final String dataSource = task.getDataSource();
final Interval interval = task.getInterval();
final List<TaskGroup> foundLocks = findLocks(dataSource, interval);
final TaskGroup taskGroupToUse;
if (foundLocks.size() > 1) {
// Too many existing locks.
return Optional.absent();
} else if (foundLocks.size() == 1) {
// One existing lock -- check if we can add to it.
final TaskGroup foundLock = Iterables.getOnlyElement(foundLocks);
if (foundLock.getInterval().contains(interval) && foundLock.getGroupId().equals(task.getGroupId())) {
taskGroupToUse = foundLock;
} else {
return Optional.absent();
}
} else {
// No existing locks. We can make a new one.
if (!running.containsKey(dataSource)) {
running.put(dataSource, new TreeMap<Interval, TaskGroup>(Comparators.intervalsByStartThenEnd()));
}
// Create new TaskGroup and assign it a version.
// Assumption: We'll choose a version that is greater than any previously-chosen version for our interval. (This
// may not always be true, unfortunately. See below.)
final String version;
if(preferredVersion.isPresent()) {
// We have a preferred version. Since this is a private method, we'll trust our caller to not break our
// ordering assumptions and just use it.
version = preferredVersion.get();
} else {
// We are running under an interval lock right now, so just using the current time works as long as we can trust
// our clock to be monotonic and have enough resolution since the last time we created a TaskGroup for the same
// interval. This may not always be true; to assure it we would need to use some method of timekeeping other
// than the wall clock.
version = new DateTime().toString();
}
taskGroupToUse = new TaskGroup(task.getGroupId(), dataSource, interval, version);
running.get(dataSource)
.put(interval, taskGroupToUse);
log.info("Created new TaskGroup[%s]", taskGroupToUse);
}
// Add to existing TaskGroup, if necessary
if (taskGroupToUse.getTaskSet().add(task)) {
log.info("Added task[%s] to TaskGroup[%s]", task.getId(), taskGroupToUse.getGroupId());
} else {
log.info("Task[%s] already present in TaskGroup[%s]", task.getId(), taskGroupToUse.getGroupId());
}
return Optional.of(taskGroupToUse.getVersion());
} finally {
giant.unlock();
}
}
/**
* Return all locks that overlap some search interval.
*/
private List<TaskGroup> findLocks(final String dataSource, final Interval interval)
{
giant.lock();
try {
final NavigableMap<Interval, TaskGroup> dsRunning = running.get(dataSource);
if(dsRunning == null) {
// No locks at all
return Collections.emptyList();
} else {
// Tasks are indexed by locked interval, which are sorted by interval start. Intervals are non-overlapping, so:
final NavigableSet<Interval> dsLockbox = dsRunning.navigableKeySet();
final Iterable<Interval> searchIntervals = Iterables.concat(
// Single interval that starts at or before ours
Collections.singletonList(dsLockbox.floor(new Interval(interval.getStart(), new DateTime(Long.MAX_VALUE)))),
// All intervals that start somewhere between our start instant (exclusive) and end instant (exclusive)
dsLockbox.subSet(
new Interval(interval.getStart(), new DateTime(Long.MAX_VALUE)),
false,
new Interval(interval.getEnd(), interval.getEnd()),
false
)
);
return Lists.newArrayList(
FunctionalIterable
.create(searchIntervals)
.filter(
new Predicate<Interval>()
{
@Override
public boolean apply(@Nullable Interval searchInterval)
{
return searchInterval != null && searchInterval.overlaps(interval);
}
}
)
.transform(
new Function<Interval, TaskGroup>()
{
@Override
public TaskGroup apply(Interval interval)
{
return dsRunning.get(interval);
}
}
)
);
}
}
finally {
giant.unlock();

View File

@ -19,20 +19,29 @@
package com.metamx.druid.merger.coordinator;
import com.metamx.druid.merger.common.TaskCallback;
import com.metamx.druid.merger.common.task.Task;
import java.util.Collection;
/**
* Interface for handing off tasks. Used by a {@link com.metamx.druid.merger.coordinator.exec.TaskConsumer} to run tasks that
* have been locked.
* Interface for handing off tasks. Used by a {@link com.metamx.druid.merger.coordinator.exec.TaskConsumer} to
* run tasks that have been locked.
*/
public interface TaskRunner
{
/**
* Run a task with a particular context and call a callback. The callback should be called exactly once.
* Run a task with a particular context and call a callback. The callback may be called multiple times with RUNNING
* status, but should be called exactly once with a non-RUNNING status (e.g. SUCCESS, FAILED, CONTINUED...).
*
* @param task task to run
* @param context task context to run under
* @param callback callback to be called exactly once
*/
public void run(Task task, TaskContext context, TaskCallback callback);
public void run(Task task, TaskCallback callback);
public Collection<TaskRunnerWorkItem> getRunningTasks();
public Collection<TaskRunnerWorkItem> getPendingTasks();
public Collection<ZkWorker> getWorkers();
}

View File

@ -0,0 +1,133 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.merger.coordinator;
import com.metamx.druid.merger.common.TaskCallback;
import com.metamx.druid.merger.common.task.Task;
import org.joda.time.DateTime;
import org.joda.time.DateTimeComparator;
/**
* A holder for a task and different components associated with the task
*/
public class TaskRunnerWorkItem implements Comparable<TaskRunnerWorkItem>
{
private final Task task;
private final TaskCallback callback;
private final RetryPolicy retryPolicy;
private final DateTime createdTime;
private volatile DateTime queueInsertionTime;
public TaskRunnerWorkItem(
Task task,
TaskCallback callback,
RetryPolicy retryPolicy,
DateTime createdTime
)
{
this.task = task;
this.callback = callback;
this.retryPolicy = retryPolicy;
this.createdTime = createdTime;
}
public Task getTask()
{
return task;
}
public TaskCallback getCallback()
{
return callback;
}
public RetryPolicy getRetryPolicy()
{
return retryPolicy;
}
public DateTime getCreatedTime()
{
return createdTime;
}
public DateTime getQueueInsertionTime()
{
return queueInsertionTime;
}
public TaskRunnerWorkItem withQueueInsertionTime(DateTime time)
{
this.queueInsertionTime = time;
return this;
}
@Override
public int compareTo(TaskRunnerWorkItem taskRunnerWorkItem)
{
return DateTimeComparator.getInstance().compare(createdTime, taskRunnerWorkItem.getCreatedTime());
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
TaskRunnerWorkItem that = (TaskRunnerWorkItem) o;
if (callback != null ? !callback.equals(that.callback) : that.callback != null) {
return false;
}
if (retryPolicy != null ? !retryPolicy.equals(that.retryPolicy) : that.retryPolicy != null) {
return false;
}
if (task != null ? !task.equals(that.task) : that.task != null) {
return false;
}
return true;
}
@Override
public int hashCode()
{
int result = task != null ? task.hashCode() : 0;
result = 31 * result + (callback != null ? callback.hashCode() : 0);
result = 31 * result + (retryPolicy != null ? retryPolicy.hashCode() : 0);
return result;
}
@Override
public String toString()
{
return "TaskRunnerWorkItem{" +
"task=" + task +
", callback=" + callback +
", retryPolicy=" + retryPolicy +
", createdTime=" + createdTime +
'}';
}
}

View File

@ -19,26 +19,17 @@
package com.metamx.druid.merger.coordinator;
import com.metamx.druid.merger.common.task.Task;
import org.joda.time.DateTime;
public class VersionedTaskWrapper
import java.util.concurrent.ConcurrentSkipListMap;
/**
*/
public class TaskRunnerWorkQueue extends ConcurrentSkipListMap<String, TaskRunnerWorkItem>
{
final Task task;
final String version;
public VersionedTaskWrapper(Task task, String version)
@Override
public TaskRunnerWorkItem put(String s, TaskRunnerWorkItem taskRunnerWorkItem)
{
this.task = task;
this.version = version;
}
public Task getTask()
{
return task;
}
public String getVersion()
{
return version;
return super.put(s, taskRunnerWorkItem.withQueueInsertionTime(new DateTime()));
}
}

View File

@ -21,6 +21,8 @@ package com.metamx.druid.merger.coordinator;
import com.google.common.base.Optional;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.actions.TaskAction;
import com.metamx.druid.merger.common.TaskLock;
import com.metamx.druid.merger.common.task.Task;
import java.util.List;
@ -29,20 +31,34 @@ public interface TaskStorage
{
/**
* Adds a task to the storage facility with a particular status. If the task ID already exists, this method
* will throw an exception.
* will throw a {@link TaskExistsException}.
*/
public void insert(Task task, TaskStatus status);
/**
* Updates task status in the storage facility.
* Persists task status in the storage facility. This method should throw an exception if the task status lifecycle
* is not respected (absent -> RUNNING -> SUCCESS/FAILURE).
*/
public void setStatus(String taskid, TaskStatus status);
public void setStatus(TaskStatus status);
/**
* Updates task version in the storage facility. If the task already has a version, this method will throw
* an exception.
* Persists lock state in the storage facility.
*/
public void setVersion(String taskid, String version);
public void addLock(String taskid, TaskLock taskLock);
/**
* Removes lock state from the storage facility. It is harmless to keep old locks in the storage facility, but
* this method can help reclaim wasted space.
*/
public void removeLock(String taskid, TaskLock taskLock);
/**
* Returns task as stored in the storage facility. If the task ID does not exist, this will return an
* absentee Optional.
*
* TODO -- This method probably wants to be combined with {@link #getStatus}.
*/
public Optional<Task> getTask(String taskid);
/**
* Returns task status as stored in the storage facility. If the task ID does not exist, this will return
@ -51,13 +67,22 @@ public interface TaskStorage
public Optional<TaskStatus> getStatus(String taskid);
/**
* Returns task version as stored in the storage facility. If the task ID does not exist, or if the task ID exists
* but was not yet assigned a version, this will return an absentee Optional.
* Add an action taken by a task to the audit log.
*/
public Optional<String> getVersion(String taskid);
public <T> void addAuditLog(TaskAction<T> taskAction);
/**
* Returns all actions taken by a task.
*/
public List<TaskAction> getAuditLogs(String taskid);
/**
* Returns a list of currently-running tasks as stored in the storage facility, in no particular order.
*/
public List<Task> getRunningTasks();
/**
* Returns a list of locks for a particular task.
*/
public List<TaskLock> getLocks(String taskid);
}

View File

@ -0,0 +1,177 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.merger.coordinator;
import com.google.common.base.Function;
import com.google.common.base.Optional;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.actions.SegmentInsertAction;
import com.metamx.druid.merger.common.actions.SpawnTasksAction;
import com.metamx.druid.merger.common.actions.TaskAction;
import com.metamx.druid.merger.common.task.Task;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
* Wraps a {@link TaskStorage}, providing a useful collection of read-only methods.
*/
public class TaskStorageQueryAdapter
{
private final TaskStorage storage;
public TaskStorageQueryAdapter(TaskStorage storage)
{
this.storage = storage;
}
public Optional<TaskStatus> getStatus(final String taskid)
{
return storage.getStatus(taskid);
}
/**
* Returns all recursive task statuses for a particular task, staying within the same task group. Includes that
* task, plus any tasks it spawned, and so on. Does not include spawned tasks that ended up in a different task
* group. Does not include this task's parents or siblings.
*/
public Map<String, Optional<TaskStatus>> getSameGroupChildStatuses(final String taskid)
{
final Optional<Task> taskOptional = storage.getTask(taskid);
final Optional<TaskStatus> statusOptional = storage.getStatus(taskid);
final ImmutableMap.Builder<String, Optional<TaskStatus>> resultBuilder = ImmutableMap.builder();
resultBuilder.put(taskid, statusOptional);
final Iterable<Task> nextTasks = FunctionalIterable
.create(storage.getAuditLogs(taskid)).filter(
new Predicate<TaskAction>()
{
@Override
public boolean apply(TaskAction taskAction)
{
return taskAction instanceof SpawnTasksAction;
}
}
).transformCat(
new Function<TaskAction, Iterable<Task>>()
{
@Override
public Iterable<Task> apply(TaskAction taskAction)
{
return ((SpawnTasksAction) taskAction).getNewTasks();
}
}
);
if(taskOptional.isPresent() && statusOptional.isPresent()) {
for(final Task nextTask : nextTasks) {
if(nextTask.getGroupId().equals(taskOptional.get().getGroupId())) {
resultBuilder.putAll(getSameGroupChildStatuses(nextTask.getId()));
}
}
}
return resultBuilder.build();
}
/**
* Like {@link #getSameGroupChildStatuses}, but flattens the recursive statuses into a single, merged status.
*/
public Optional<TaskStatus> getSameGroupMergedStatus(final String taskid)
{
final Map<String, Optional<TaskStatus>> statuses = getSameGroupChildStatuses(taskid);
int nSuccesses = 0;
int nFailures = 0;
int nTotal = 0;
for(final Optional<TaskStatus> statusOption : statuses.values()) {
nTotal ++;
if(statusOption.isPresent()) {
final TaskStatus status = statusOption.get();
if(status.isSuccess()) {
nSuccesses ++;
} else if(status.isFailure()) {
nFailures ++;
}
}
}
final Optional<TaskStatus> status;
if(nTotal == 0) {
status = Optional.absent();
} else if(nSuccesses == nTotal) {
status = Optional.of(TaskStatus.success(taskid));
} else if(nFailures > 0) {
status = Optional.of(TaskStatus.failure(taskid));
} else {
status = Optional.of(TaskStatus.running(taskid));
}
return status;
}
/**
* Returns all segments created by descendants for a particular task that stayed within the same task group. Includes
* that task, plus any tasks it spawned, and so on. Does not include spawned tasks that ended up in a different task
* group. Does not include this task's parents or siblings.
*/
public Set<DataSegment> getSameGroupNewSegments(final String taskid)
{
// TODO: This is useful for regular index tasks (so we know what was published), but
// TODO: for long-lived index tasks the list can get out of hand. We may want a limit.
final Optional<Task> taskOptional = storage.getTask(taskid);
final Set<DataSegment> segments = Sets.newHashSet();
final List<Task> nextTasks = Lists.newArrayList();
for(final TaskAction action : storage.getAuditLogs(taskid)) {
if(action instanceof SpawnTasksAction) {
nextTasks.addAll(((SpawnTasksAction) action).getNewTasks());
}
if(action instanceof SegmentInsertAction) {
segments.addAll(((SegmentInsertAction) action).getSegments());
}
}
if(taskOptional.isPresent()) {
for(final Task nextTask : nextTasks) {
if(nextTask.getGroupId().equals(taskOptional.get().getGroupId())) {
segments.addAll(getSameGroupNewSegments(nextTask.getId()));
}
}
}
return segments;
}
}

View File

@ -1,60 +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;
import com.metamx.druid.merger.common.task.Task;
/**
*/
public class TaskWrapper
{
private final Task task;
private final TaskContext taskContext;
private final TaskCallback callback;
private final RetryPolicy retryPolicy;
public TaskWrapper(Task task, TaskContext taskContext, TaskCallback callback, RetryPolicy retryPolicy)
{
this.task = task;
this.taskContext = taskContext;
this.callback = callback;
this.retryPolicy = retryPolicy;
}
public Task getTask()
{
return task;
}
public TaskContext getTaskContext()
{
return taskContext;
}
public TaskCallback getCallback()
{
return callback;
}
public RetryPolicy getRetryPolicy()
{
return retryPolicy;
}
}

View File

@ -37,8 +37,9 @@ import java.io.IOException;
import java.util.Set;
/**
* Holds information about a worker and a listener for task status changes associated with the worker.
*/
public class WorkerWrapper implements Closeable
public class ZkWorker implements Closeable
{
private final Worker worker;
private final PathChildrenCache statusCache;
@ -46,7 +47,7 @@ public class WorkerWrapper implements Closeable
private volatile DateTime lastCompletedTaskTime = new DateTime();
public WorkerWrapper(Worker worker, PathChildrenCache statusCache, final ObjectMapper jsonMapper)
public ZkWorker(Worker worker, PathChildrenCache statusCache, final ObjectMapper jsonMapper)
{
this.worker = worker;
this.statusCache = statusCache;
@ -105,4 +106,15 @@ public class WorkerWrapper implements Closeable
{
statusCache.close();
}
@Override
public String toString()
{
return "ZkWorker{" +
"worker=" + worker +
", statusCache=" + statusCache +
", cacheConverter=" + cacheConverter +
", lastCompletedTaskTime=" + lastCompletedTaskTime +
'}';
}
}

View File

@ -52,9 +52,6 @@ public abstract class IndexerCoordinatorConfig
@Default("local")
public abstract String getStorageImpl();
@Config("druid.merger.taskDir")
public abstract File getBaseTaskDir();
@Config("druid.merger.whitelist.enabled")
@Default("false")
public abstract boolean isWhitelistEnabled();
@ -63,10 +60,6 @@ public abstract class IndexerCoordinatorConfig
@Default("")
public abstract String getWhitelistDatasourcesString();
public File getTaskDir(final Task task) {
return new File(getBaseTaskDir(), task.getId());
}
public Set<String> getWhitelistDatasources()
{
if(whitelistDatasources == null) {
@ -80,10 +73,6 @@ public abstract class IndexerCoordinatorConfig
return whitelistDatasources;
}
@Config("druid.merger.rowFlushBoundary")
@Default("500000")
public abstract long getRowFlushBoundary();
@Config("druid.indexer.strategy")
@Default("noop")
public abstract String getStrategyImpl();

View File

@ -29,4 +29,12 @@ public abstract class IndexerDbConnectorConfig extends DbConnectorConfig
@JsonProperty("taskTable")
@Config("druid.database.taskTable")
public abstract String getTaskTable();
@JsonProperty("taskLockTable")
@Config("druid.database.taskLockTable")
public abstract String getTaskLockTable();
@JsonProperty("taskLogTable")
@Config("druid.database.taskLogTable")
public abstract String getTaskLogTable();
}

View File

@ -20,7 +20,6 @@
package com.metamx.druid.merger.coordinator.config;
import com.metamx.druid.merger.common.config.IndexerZkConfig;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.skife.config.Config;
import org.skife.config.Default;
@ -29,22 +28,6 @@ import org.skife.config.Default;
*/
public abstract class RemoteTaskRunnerConfig extends IndexerZkConfig
{
@Config("druid.indexer.terminateResources.duration")
@Default("PT1H")
public abstract Duration getTerminateResourcesDuration();
@Config("druid.indexer.terminateResources.originDateTime")
@Default("2012-01-01T00:55:00.000Z")
public abstract DateTime getTerminateResourcesOriginDateTime();
@Config("druid.indexer.maxWorkerIdleTimeMillisBeforeDeletion")
@Default("600000")
public abstract int getMaxWorkerIdleTimeMillisBeforeDeletion();
@Config("druid.indexer.maxScalingDuration")
@Default("PT1H")
public abstract Duration getMaxScalingDuration();
@Config("druid.indexer.taskAssignmentTimeoutDuration")
@Default("PT5M")
public abstract Duration getTaskAssignmentTimeoutDuration();

View File

@ -20,18 +20,14 @@
package com.metamx.druid.merger.coordinator.exec;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableSet;
import com.metamx.common.lifecycle.LifecycleStart;
import com.metamx.common.lifecycle.LifecycleStop;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merger.common.TaskCallback;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.MergerDBCoordinator;
import com.metamx.druid.merger.coordinator.TaskCallback;
import com.metamx.druid.merger.coordinator.TaskContext;
import com.metamx.druid.merger.coordinator.TaskQueue;
import com.metamx.druid.merger.coordinator.TaskRunner;
import com.metamx.druid.merger.coordinator.VersionedTaskWrapper;
import com.metamx.emitter.EmittingLogger;
import com.metamx.emitter.service.ServiceEmitter;
import com.metamx.emitter.service.ServiceMetricEvent;
@ -40,7 +36,7 @@ public class TaskConsumer implements Runnable
{
private final TaskQueue queue;
private final TaskRunner runner;
private final MergerDBCoordinator mergerDBCoordinator;
private final TaskToolbox toolbox;
private final ServiceEmitter emitter;
private final Thread thready;
@ -51,13 +47,13 @@ public class TaskConsumer implements Runnable
public TaskConsumer(
TaskQueue queue,
TaskRunner runner,
MergerDBCoordinator mergerDBCoordinator,
TaskToolbox toolbox,
ServiceEmitter emitter
)
{
this.queue = queue;
this.runner = runner;
this.mergerDBCoordinator = mergerDBCoordinator;
this.toolbox = toolbox;
this.emitter = emitter;
this.thready = new Thread(this);
}
@ -83,176 +79,111 @@ public class TaskConsumer implements Runnable
while (!Thread.currentThread().isInterrupted()) {
final Task task;
final String version;
try {
final VersionedTaskWrapper taskWrapper = queue.take();
task = taskWrapper.getTask();
version = taskWrapper.getVersion();
task = queue.take();
}
catch (InterruptedException e) {
log.info(e, "Interrupted while waiting for new work");
throw Throwables.propagate(e);
throw e;
}
try {
handoff(task, version);
handoff(task);
}
catch (Exception e) {
log.makeAlert(e, "Failed to hand off task")
.addData("task", task.getId())
.addData("type", task.getType().toString())
.addData("type", task.getType())
.addData("dataSource", task.getDataSource())
.addData("interval", task.getInterval())
.addData("interval", task.getFixedInterval())
.emit();
// Retry would be nice, but only after we have a way to throttle and limit them. Just fail for now.
if(!shutdown) {
queue.done(task, TaskStatus.failure(task.getId()));
if (!shutdown) {
queue.notify(task, TaskStatus.failure(task.getId()));
}
}
}
}
catch (Throwable t) {
catch (Exception e) {
// exit thread
log.error(t, "Uncaught Throwable while consuming tasks");
throw Throwables.propagate(t);
log.error(e, "Uncaught exception while consuming tasks");
throw Throwables.propagate(e);
}
}
private void handoff(final Task task, final String version) throws Exception
private void handoff(final Task task) throws Exception
{
final TaskContext context = new TaskContext(
version,
ImmutableSet.copyOf(
mergerDBCoordinator.getSegmentsForInterval(
task.getDataSource(),
task.getInterval()
)
)
);
final ServiceMetricEvent.Builder builder = new ServiceMetricEvent.Builder()
final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder()
.setUser2(task.getDataSource())
.setUser4(task.getType().toString())
.setUser5(task.getInterval().toString());
.setUser4(task.getType())
.setUser5(task.getFixedInterval().toString());
// Run preflight checks
TaskStatus preflightStatus;
try {
preflightStatus = task.preflight(context);
preflightStatus = task.preflight(toolbox);
log.info("Preflight done for task: %s", task.getId());
} catch(Exception e) {
}
catch (Exception e) {
preflightStatus = TaskStatus.failure(task.getId());
log.error(e, "Exception thrown during preflight for task: %s", task.getId());
}
if (!preflightStatus.isRunnable()) {
log.info("Task finished during preflight: %s", task.getId());
queue.done(task, preflightStatus);
queue.notify(task, preflightStatus);
return;
}
// Hand off work to TaskRunner
// Hand off work to TaskRunner, with a callback
runner.run(
task, context, new TaskCallback()
task, new TaskCallback()
{
@Override
public void notify(final TaskStatus statusFromRunner)
{
// task is done
log.info("TaskRunner finished task: %s", task);
// we might need to change this due to exceptions
TaskStatus status = statusFromRunner;
try {
log.info("Received %s status for task: %s", statusFromRunner.getStatusCode(), task);
// If we're not supposed to be running anymore, don't do anything. Somewhat racey if the flag gets set after
// we check and before we commit the database transaction, but better than nothing.
if(shutdown) {
if (shutdown) {
log.info("Abandoning task due to shutdown: %s", task.getId());
return;
}
// Publish returned segments
// FIXME: Publish in transaction
try {
for (DataSegment segment : status.getSegments()) {
if (!task.getDataSource().equals(segment.getDataSource())) {
throw new IllegalStateException(
String.format(
"Segment for task[%s] has invalid dataSource: %s",
task.getId(),
segment.getIdentifier()
)
);
}
queue.notify(task, statusFromRunner);
if (!task.getInterval().contains(segment.getInterval())) {
throw new IllegalStateException(
String.format(
"Segment for task[%s] has invalid interval: %s",
task.getId(),
segment.getIdentifier()
)
);
}
// Emit event and log, if the task is done
if (statusFromRunner.isComplete()) {
metricBuilder.setUser3(statusFromRunner.getStatusCode().toString());
emitter.emit(metricBuilder.build("indexer/time/run/millis", statusFromRunner.getDuration()));
if (!context.getVersion().equals(segment.getVersion())) {
throw new IllegalStateException(
String.format(
"Segment for task[%s] has invalid version: %s",
task.getId(),
segment.getIdentifier()
)
);
}
log.info("Publishing segment[%s] for task[%s]", segment.getIdentifier(), task.getId());
mergerDBCoordinator.announceHistoricalSegment(segment);
}
}
catch (Exception e) {
log.error(e, "Exception while publishing segments for task: %s", task);
status = TaskStatus.failure(task.getId()).withDuration(status.getDuration());
}
try {
queue.done(task, status);
}
catch (Exception e) {
log.error(e, "Exception while marking task done: %s", task);
throw Throwables.propagate(e);
}
// emit event and log
int bytes = 0;
for (DataSegment segment : status.getSegments()) {
bytes += segment.getSize();
}
builder.setUser3(status.getStatusCode().toString());
emitter.emit(builder.build("indexer/time/run/millis", status.getDuration()));
emitter.emit(builder.build("indexer/segment/count", status.getSegments().size()));
emitter.emit(builder.build("indexer/segment/bytes", bytes));
if (status.isFailure()) {
if (statusFromRunner.isFailure()) {
log.makeAlert("Failed to index")
.addData("task", task.getId())
.addData("type", task.getType().toString())
.addData("type", task.getType())
.addData("dataSource", task.getDataSource())
.addData("interval", task.getInterval())
.addData("interval", task.getFixedInterval())
.emit();
}
log.info(
"Task %s: %s (%d segments) (%d run duration)",
status.getStatusCode(),
"Task %s: %s (%d run duration)",
statusFromRunner.getStatusCode(),
task,
status.getSegments().size(),
status.getDuration()
statusFromRunner.getDuration()
);
}
}
catch (Exception e) {
log.makeAlert(e, "Failed to handle task callback")
.addData("task", task.getId())
.addData("statusCode", statusFromRunner.getStatusCode())
.emit();
}
}
}
);

View File

@ -0,0 +1,67 @@
/*
* 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.http;
import com.google.inject.Inject;
import com.metamx.common.logger.Logger;
import com.metamx.druid.merger.coordinator.TaskMasterLifecycle;
import javax.ws.rs.GET;
import javax.ws.rs.Path;
import javax.ws.rs.Produces;
import javax.ws.rs.core.Response;
/**
*/
@Path("/mmx/merger/v1/info")
public class IndexerCoordinatorInfoResource
{
private static final Logger log = new Logger(IndexerCoordinatorInfoResource.class);
private final TaskMasterLifecycle taskMasterLifecycle;
@Inject
public IndexerCoordinatorInfoResource(TaskMasterLifecycle taskMasterLifecycle)
{
this.taskMasterLifecycle = taskMasterLifecycle;
}
@GET
@Path("/pendingTasks}")
@Produces("application/json")
public Response getPendingTasks()
{
if (taskMasterLifecycle.getTaskRunner() == null) {
return Response.noContent().build();
}
return Response.ok(taskMasterLifecycle.getTaskRunner().getPendingTasks()).build();
}
@GET
@Path("/runningTasks}")
@Produces("application/json")
public Response getRunningTasks()
{
if (taskMasterLifecycle.getTaskRunner() == null) {
return Response.noContent().build();
}
return Response.ok(taskMasterLifecycle.getTaskRunner().getRunningTasks()).build();
}
}

View File

@ -49,11 +49,16 @@ import com.metamx.druid.initialization.Initialization;
import com.metamx.druid.initialization.ServerConfig;
import com.metamx.druid.initialization.ServiceDiscoveryConfig;
import com.metamx.druid.jackson.DefaultObjectMapper;
import com.metamx.druid.loading.S3SegmentPusher;
import com.metamx.druid.loading.S3SegmentPusherConfig;
import com.metamx.druid.loading.SegmentPusher;
import com.metamx.druid.loading.DataSegmentPusher;
import com.metamx.druid.loading.S3DataSegmentPusher;
import com.metamx.druid.loading.S3DataSegmentPusherConfig;
import com.metamx.druid.loading.S3SegmentKiller;
import com.metamx.druid.loading.SegmentKiller;
import com.metamx.druid.merger.common.TaskToolbox;
import com.metamx.druid.merger.common.actions.LocalTaskActionClient;
import com.metamx.druid.merger.common.actions.TaskActionToolbox;
import com.metamx.druid.merger.common.config.IndexerZkConfig;
import com.metamx.druid.merger.common.config.TaskConfig;
import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory;
import com.metamx.druid.merger.coordinator.DbTaskStorage;
import com.metamx.druid.merger.coordinator.LocalTaskRunner;
@ -61,20 +66,27 @@ import com.metamx.druid.merger.coordinator.LocalTaskStorage;
import com.metamx.druid.merger.coordinator.MergerDBCoordinator;
import com.metamx.druid.merger.coordinator.RemoteTaskRunner;
import com.metamx.druid.merger.coordinator.RetryPolicyFactory;
import com.metamx.druid.merger.coordinator.TaskMaster;
import com.metamx.druid.merger.coordinator.TaskLockbox;
import com.metamx.druid.merger.coordinator.TaskMasterLifecycle;
import com.metamx.druid.merger.coordinator.TaskQueue;
import com.metamx.druid.merger.coordinator.TaskRunner;
import com.metamx.druid.merger.coordinator.TaskRunnerFactory;
import com.metamx.druid.merger.coordinator.TaskStorage;
import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter;
import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig;
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
import com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig;
import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig;
import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig;
import com.metamx.druid.merger.coordinator.config.WorkerSetupManagerConfig;
import com.metamx.druid.merger.coordinator.scaling.AutoScalingStrategy;
import com.metamx.druid.merger.coordinator.scaling.EC2AutoScalingStrategy;
import com.metamx.druid.merger.coordinator.scaling.NoopScalingStrategy;
import com.metamx.druid.merger.coordinator.scaling.ScalingStrategy;
import com.metamx.druid.merger.coordinator.scaling.NoopAutoScalingStrategy;
import com.metamx.druid.merger.coordinator.scaling.ResourceManagementScheduler;
import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerConfig;
import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerFactory;
import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagementStrategy;
import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagmentConfig;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
import com.metamx.druid.utils.PropUtils;
import com.metamx.emitter.EmittingLogger;
@ -91,8 +103,6 @@ import com.metamx.metrics.MonitorSchedulerConfig;
import com.metamx.metrics.SysMonitor;
import com.netflix.curator.framework.CuratorFramework;
import com.netflix.curator.framework.recipes.cache.PathChildrenCache;
import org.jets3t.service.S3ServiceException;
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
import org.jets3t.service.security.AWSCredentials;
@ -130,17 +140,23 @@ public class IndexerCoordinatorNode extends RegisteringNode
private List<Monitor> monitors = null;
private ServiceEmitter emitter = null;
private DbConnectorConfig dbConnectorConfig = null;
private DBI dbi = null;
private RestS3Service s3Service = null;
private IndexerCoordinatorConfig config = null;
private TaskConfig taskConfig = null;
private TaskToolbox taskToolbox = null;
private MergerDBCoordinator mergerDBCoordinator = null;
private TaskStorage taskStorage = null;
private TaskQueue taskQueue = null;
private TaskLockbox taskLockbox = null;
private CuratorFramework curatorFramework = null;
private ScheduledExecutorFactory scheduledExecutorFactory = null;
private IndexerZkConfig indexerZkConfig;
private WorkerSetupManager workerSetupManager = null;
private TaskRunnerFactory taskRunnerFactory = null;
private TaskMaster taskMaster = null;
private ResourceManagementSchedulerFactory resourceManagementSchedulerFactory = null;
private TaskMasterLifecycle taskMasterLifecycle = null;
private Server server = null;
private boolean initialized = false;
@ -178,6 +194,18 @@ public class IndexerCoordinatorNode extends RegisteringNode
return this;
}
public IndexerCoordinatorNode setS3Service(RestS3Service s3Service)
{
this.s3Service = s3Service;
return this;
}
public IndexerCoordinatorNode setTaskLockbox(TaskLockbox taskLockbox)
{
this.taskLockbox = taskLockbox;
return this;
}
public IndexerCoordinatorNode setMergeDbCoordinator(MergerDBCoordinator mergeDbCoordinator)
{
this.mergerDBCoordinator = mergeDbCoordinator;
@ -202,24 +230,35 @@ public class IndexerCoordinatorNode extends RegisteringNode
return this;
}
public IndexerCoordinatorNode setResourceManagementSchedulerFactory(ResourceManagementSchedulerFactory resourceManagementSchedulerFactory)
{
this.resourceManagementSchedulerFactory = resourceManagementSchedulerFactory;
return this;
}
public void init() throws Exception
{
scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle);
initializeEmitter();
initializeMonitors();
initializeDB();
initializeIndexerCoordinatorConfig();
initializeTaskConfig();
initializeS3Service();
initializeMergeDBCoordinator();
initializeTaskToolbox();
initializeTaskStorage();
initializeTaskLockbox();
initializeTaskQueue();
initializeTaskToolbox();
initializeJacksonInjections();
initializeJacksonSubtypes();
initializeCurator();
initializeIndexerZkConfig();
initializeWorkerSetupManager();
initializeTaskRunnerFactory();
initializeTaskMaster();
initializeResourceManagement();
initializeTaskMasterLifecycle();
initializeServer();
final ScheduledExecutorService globalScheduledExec = scheduledExecutorFactory.create(1, "Global--%d");
@ -236,7 +275,8 @@ public class IndexerCoordinatorNode extends RegisteringNode
jsonMapper,
config,
emitter,
taskQueue,
taskMasterLifecycle,
new TaskStorageQueryAdapter(taskStorage),
workerSetupManager
)
);
@ -255,7 +295,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
@Override
public boolean doLocal()
{
return taskMaster.isLeading();
return taskMasterLifecycle.isLeading();
}
@Override
@ -265,7 +305,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
return new URL(
String.format(
"http://%s%s",
taskMaster.getLeader(),
taskMasterLifecycle.getLeader(),
requestURI
)
);
@ -283,20 +323,21 @@ public class IndexerCoordinatorNode extends RegisteringNode
initialized = true;
}
private void initializeTaskMaster()
private void initializeTaskMasterLifecycle()
{
if (taskMaster == null) {
if (taskMasterLifecycle == null) {
final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class);
taskMaster = new TaskMaster(
taskMasterLifecycle = new TaskMasterLifecycle(
taskQueue,
taskToolbox,
config,
serviceDiscoveryConfig,
mergerDBCoordinator,
taskRunnerFactory,
resourceManagementSchedulerFactory,
curatorFramework,
emitter
);
lifecycle.addManagedInstance(taskMaster);
lifecycle.addManagedInstance(taskMasterLifecycle);
}
}
@ -351,7 +392,7 @@ public class IndexerCoordinatorNode extends RegisteringNode
{
InjectableValues.Std injectables = new InjectableValues.Std();
injectables.addValue("s3Client", taskToolbox.getS3Client())
injectables.addValue("s3Client", s3Service)
.addValue("segmentPusher", taskToolbox.getSegmentPusher());
jsonMapper.setInjectableValues(injectables);
@ -387,6 +428,16 @@ public class IndexerCoordinatorNode extends RegisteringNode
}
}
private void initializeDB()
{
if (dbConnectorConfig == null) {
dbConnectorConfig = configFactory.build(DbConnectorConfig.class);
}
if (dbi == null) {
dbi = new DbConnector(dbConnectorConfig).getDBI();
}
}
private void initializeIndexerCoordinatorConfig()
{
if (config == null) {
@ -394,32 +445,56 @@ public class IndexerCoordinatorNode extends RegisteringNode
}
}
public void initializeTaskToolbox() throws S3ServiceException
private void initializeTaskConfig()
{
if (taskToolbox == null) {
final RestS3Service s3Client = new RestS3Service(
if (taskConfig == null) {
taskConfig = configFactory.build(TaskConfig.class);
}
}
public void initializeS3Service() throws S3ServiceException
{
this.s3Service = new RestS3Service(
new AWSCredentials(
PropUtils.getProperty(props, "com.metamx.aws.accessKey"),
PropUtils.getProperty(props, "com.metamx.aws.secretKey")
)
);
final SegmentPusher segmentPusher = new S3SegmentPusher(
s3Client,
configFactory.build(S3SegmentPusherConfig.class),
}
public void initializeTaskToolbox()
{
if (taskToolbox == null) {
final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher(
s3Service,
configFactory.build(S3DataSegmentPusherConfig.class),
jsonMapper
);
final SegmentKiller segmentKiller = new S3SegmentKiller(
s3Service
);
taskToolbox = new TaskToolbox(
taskConfig,
new LocalTaskActionClient(
taskStorage,
new TaskActionToolbox(taskQueue, taskLockbox, mergerDBCoordinator, emitter)
),
emitter,
s3Service,
dataSegmentPusher,
segmentKiller,
jsonMapper
);
taskToolbox = new TaskToolbox(config, emitter, s3Client, segmentPusher, jsonMapper);
}
}
public void initializeMergeDBCoordinator()
{
if (mergerDBCoordinator == null) {
final DbConnectorConfig dbConnectorConfig = configFactory.build(DbConnectorConfig.class);
mergerDBCoordinator = new MergerDBCoordinator(
jsonMapper,
dbConnectorConfig,
new DbConnector(dbConnectorConfig).getDBI()
dbi
);
}
}
@ -427,8 +502,15 @@ public class IndexerCoordinatorNode extends RegisteringNode
public void initializeTaskQueue()
{
if (taskQueue == null) {
// Don't start it here. The TaskMaster will handle that when it feels like it.
taskQueue = new TaskQueue(taskStorage);
// Don't start it here. The TaskMasterLifecycle will handle that when it feels like it.
taskQueue = new TaskQueue(taskStorage, taskLockbox);
}
}
public void initializeTaskLockbox()
{
if (taskLockbox == null) {
taskLockbox = new TaskLockbox(taskStorage);
}
}
@ -467,8 +549,6 @@ public class IndexerCoordinatorNode extends RegisteringNode
public void initializeWorkerSetupManager()
{
if (workerSetupManager == null) {
final DbConnectorConfig dbConnectorConfig = configFactory.build(DbConnectorConfig.class);
final DBI dbi = new DbConnector(dbConnectorConfig).getDBI();
final WorkerSetupManagerConfig workerSetupManagerConfig = configFactory.build(WorkerSetupManagerConfig.class);
DbConnector.createConfigTable(dbi, workerSetupManagerConfig.getConfigTable());
@ -504,35 +584,17 @@ public class IndexerCoordinatorNode extends RegisteringNode
.build()
);
ScalingStrategy strategy;
if (config.getStrategyImpl().equalsIgnoreCase("ec2")) {
strategy = new EC2AutoScalingStrategy(
jsonMapper,
new AmazonEC2Client(
new BasicAWSCredentials(
PropUtils.getProperty(props, "com.metamx.aws.accessKey"),
PropUtils.getProperty(props, "com.metamx.aws.secretKey")
)
),
configFactory.build(EC2AutoScalingStrategyConfig.class),
workerSetupManager
);
} else if (config.getStrategyImpl().equalsIgnoreCase("noop")) {
strategy = new NoopScalingStrategy();
} else {
throw new ISE("Invalid strategy implementation: %s", config.getStrategyImpl());
}
return new RemoteTaskRunner(
RemoteTaskRunner remoteTaskRunner = new RemoteTaskRunner(
jsonMapper,
configFactory.build(RemoteTaskRunnerConfig.class),
curatorFramework,
new PathChildrenCache(curatorFramework, indexerZkConfig.getAnnouncementPath(), true),
retryScheduledExec,
new RetryPolicyFactory(configFactory.build(RetryPolicyConfig.class)),
strategy,
workerSetupManager
);
return remoteTaskRunner;
}
};
@ -552,6 +614,56 @@ public class IndexerCoordinatorNode extends RegisteringNode
}
}
private void initializeResourceManagement()
{
if (resourceManagementSchedulerFactory == null) {
resourceManagementSchedulerFactory = new ResourceManagementSchedulerFactory()
{
@Override
public ResourceManagementScheduler build(TaskRunner runner)
{
final ScheduledExecutorService scalingScheduledExec = Executors.newScheduledThreadPool(
1,
new ThreadFactoryBuilder()
.setDaemon(true)
.setNameFormat("ScalingExec--%d")
.build()
);
AutoScalingStrategy strategy;
if (config.getStrategyImpl().equalsIgnoreCase("ec2")) {
strategy = new EC2AutoScalingStrategy(
jsonMapper,
new AmazonEC2Client(
new BasicAWSCredentials(
PropUtils.getProperty(props, "com.metamx.aws.accessKey"),
PropUtils.getProperty(props, "com.metamx.aws.secretKey")
)
),
configFactory.build(EC2AutoScalingStrategyConfig.class),
workerSetupManager
);
} else if (config.getStrategyImpl().equalsIgnoreCase("noop")) {
strategy = new NoopAutoScalingStrategy();
} else {
throw new ISE("Invalid strategy implementation: %s", config.getStrategyImpl());
}
return new ResourceManagementScheduler(
runner,
new SimpleResourceManagementStrategy(
strategy,
configFactory.build(SimpleResourceManagmentConfig.class),
workerSetupManager
),
configFactory.build(ResourceManagementSchedulerConfig.class),
scalingScheduledExec
);
}
};
}
}
public static class Builder
{
private ObjectMapper jsonMapper = null;

View File

@ -21,16 +21,21 @@ package com.metamx.druid.merger.coordinator.http;
import com.google.common.base.Optional;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import com.google.inject.Inject;
import com.metamx.common.logger.Logger;
import com.metamx.druid.client.DataSegment;
import com.metamx.druid.merger.common.TaskStatus;
import com.metamx.druid.merger.common.task.MergeTask;
import com.metamx.druid.merger.common.actions.TaskAction;
import com.metamx.druid.merger.common.task.Task;
import com.metamx.druid.merger.coordinator.TaskQueue;
import com.metamx.druid.merger.coordinator.TaskMasterLifecycle;
import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter;
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupData;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
import com.metamx.emitter.service.ServiceEmitter;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.core.type.TypeReference;
import javax.ws.rs.Consumes;
import javax.ws.rs.GET;
@ -39,6 +44,8 @@ import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.Produces;
import javax.ws.rs.core.Response;
import java.util.Map;
import java.util.Set;
/**
*/
@ -49,31 +56,34 @@ public class IndexerCoordinatorResource
private final IndexerCoordinatorConfig config;
private final ServiceEmitter emitter;
private final TaskQueue tasks;
private final TaskMasterLifecycle taskMasterLifecycle;
private final TaskStorageQueryAdapter taskStorageQueryAdapter;
private final WorkerSetupManager workerSetupManager;
private final ObjectMapper jsonMapper;
@Inject
public IndexerCoordinatorResource(
IndexerCoordinatorConfig config,
ServiceEmitter emitter,
TaskQueue tasks,
WorkerSetupManager workerSetupManager
TaskMasterLifecycle taskMasterLifecycle,
TaskStorageQueryAdapter taskStorageQueryAdapter,
WorkerSetupManager workerSetupManager,
ObjectMapper jsonMapper
) throws Exception
{
this.config = config;
this.emitter = emitter;
this.tasks = tasks;
this.taskMasterLifecycle = taskMasterLifecycle;
this.taskStorageQueryAdapter = taskStorageQueryAdapter;
this.workerSetupManager = workerSetupManager;
this.jsonMapper = jsonMapper;
}
@POST
@Path("/merge")
@Consumes("application/json")
@Produces("application/json")
public Response doMerge(
final MergeTask task
)
public Response doMerge(final Task task)
{
// legacy endpoint
return doIndex(task);
@ -83,9 +93,7 @@ public class IndexerCoordinatorResource
@Path("/index")
@Consumes("application/json")
@Produces("application/json")
public Response doIndex(
final Task task
)
public Response doIndex(final Task task)
{
// verify against whitelist
if (config.isWhitelistEnabled() && !config.getWhitelistDatasources().contains(task.getDataSource())) {
@ -99,16 +107,16 @@ public class IndexerCoordinatorResource
.build();
}
tasks.add(task);
return okResponse(task.getId());
taskMasterLifecycle.getTaskQueue().add(task);
return Response.ok(ImmutableMap.of("task", task.getId())).build();
}
@GET
@Path("/status/{taskid}")
@Path("/task/{taskid}/status")
@Produces("application/json")
public Response doStatus(@PathParam("taskid") String taskid)
public Response getTaskStatus(@PathParam("taskid") String taskid)
{
final Optional<TaskStatus> status = tasks.getStatus(taskid);
final Optional<TaskStatus> status = taskStorageQueryAdapter.getSameGroupMergedStatus(taskid);
if (!status.isPresent()) {
return Response.status(Response.Status.NOT_FOUND).build();
} else {
@ -116,9 +124,30 @@ public class IndexerCoordinatorResource
}
}
private Response okResponse(final String taskid)
@GET
@Path("/task/{taskid}/segments")
@Produces("application/json")
public Response getTaskSegments(@PathParam("taskid") String taskid)
{
return Response.ok(ImmutableMap.of("task", taskid)).build();
final Set<DataSegment> segments = taskStorageQueryAdapter.getSameGroupNewSegments(taskid);
return Response.ok().entity(segments).build();
}
// Legacy endpoint
// TODO Remove
@Deprecated
@GET
@Path("/status/{taskid}")
@Produces("application/json")
public Response getLegacyStatus(@PathParam("taskid") String taskid)
{
final Optional<TaskStatus> status = taskStorageQueryAdapter.getSameGroupMergedStatus(taskid);
final Set<DataSegment> segments = taskStorageQueryAdapter.getSameGroupNewSegments(taskid);
final Map<String, Object> ret = jsonMapper.convertValue(status, new TypeReference<Map<String, Object>>(){});
ret.put("segments", segments);
return Response.ok().entity(ret).build();
}
@GET
@ -141,4 +170,16 @@ public class IndexerCoordinatorResource
}
return Response.ok().build();
}
@POST
@Path("/action")
@Produces("application/json")
public <T> Response doAction(final TaskAction<T> action)
{
final T ret = taskMasterLifecycle.getTaskToolbox().getTaskActionClient().submit(action);
final Map<String, Object> retMap = Maps.newHashMap();
retMap.put("result", ret);
return Response.ok().entity(retMap).build();
}
}

View File

@ -22,15 +22,14 @@ package com.metamx.druid.merger.coordinator.http;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider;
import com.google.inject.Provides;
import com.metamx.druid.merger.coordinator.TaskQueue;
import com.metamx.druid.merger.coordinator.TaskMasterLifecycle;
import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter;
import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
import com.metamx.emitter.service.ServiceEmitter;
import com.sun.jersey.guice.JerseyServletModule;
import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
import javax.inject.Singleton;
/**
@ -40,21 +39,24 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
private final ObjectMapper jsonMapper;
private final IndexerCoordinatorConfig indexerCoordinatorConfig;
private final ServiceEmitter emitter;
private final TaskQueue tasks;
private final TaskMasterLifecycle taskMasterLifecycle;
private final TaskStorageQueryAdapter taskStorageQueryAdapter;
private final WorkerSetupManager workerSetupManager;
public IndexerCoordinatorServletModule(
ObjectMapper jsonMapper,
IndexerCoordinatorConfig indexerCoordinatorConfig,
ServiceEmitter emitter,
TaskQueue tasks,
TaskMasterLifecycle taskMasterLifecycle,
TaskStorageQueryAdapter taskStorageQueryAdapter,
WorkerSetupManager workerSetupManager
)
{
this.jsonMapper = jsonMapper;
this.indexerCoordinatorConfig = indexerCoordinatorConfig;
this.emitter = emitter;
this.tasks = tasks;
this.taskMasterLifecycle = taskMasterLifecycle;
this.taskStorageQueryAdapter = taskStorageQueryAdapter;
this.workerSetupManager = workerSetupManager;
}
@ -62,10 +64,12 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule
protected void configureServlets()
{
bind(IndexerCoordinatorResource.class);
bind(IndexerCoordinatorInfoResource.class);
bind(ObjectMapper.class).toInstance(jsonMapper);
bind(IndexerCoordinatorConfig.class).toInstance(indexerCoordinatorConfig);
bind(ServiceEmitter.class).toInstance(emitter);
bind(TaskQueue.class).toInstance(tasks);
bind(TaskMasterLifecycle.class).toInstance(taskMasterLifecycle);
bind(TaskStorageQueryAdapter.class).toInstance(taskStorageQueryAdapter);
bind(WorkerSetupManager.class).toInstance(workerSetupManager);
serve("/*").with(GuiceContainer.class);

View File

@ -43,4 +43,13 @@ public class AutoScalingData<T>
{
return nodes;
}
@Override
public String toString()
{
return "AutoScalingData{" +
"nodeIds=" + nodeIds +
", nodes=" + nodes +
'}';
}
}

View File

@ -22,8 +22,9 @@ package com.metamx.druid.merger.coordinator.scaling;
import java.util.List;
/**
* The AutoScalingStrategy has the actual methods to provision and terminate worker nodes.
*/
public interface ScalingStrategy<T>
public interface AutoScalingStrategy<T>
{
public AutoScalingData<T> provision();
@ -31,8 +32,8 @@ public interface ScalingStrategy<T>
/**
* Provides a lookup of ip addresses to node ids
* @param ips
* @return
* @param ips - nodes ips
* @return node ids
*/
public List<String> ipLookup(List<String> ips);
public List<String> ipToIdLookup(List<String> ips);
}

View File

@ -44,7 +44,7 @@ import java.util.List;
/**
*/
public class EC2AutoScalingStrategy implements ScalingStrategy<Instance>
public class EC2AutoScalingStrategy implements AutoScalingStrategy<Instance>
{
private static final EmittingLogger log = new EmittingLogger(EC2AutoScalingStrategy.class);
@ -73,7 +73,6 @@ public class EC2AutoScalingStrategy implements ScalingStrategy<Instance>
WorkerSetupData setupData = workerSetupManager.getWorkerSetupData();
EC2NodeData workerConfig = setupData.getNodeData();
log.info("Creating new instance(s)...");
RunInstancesResult result = amazonEC2Client.runInstances(
new RunInstancesRequest(
workerConfig.getAmiId(),
@ -131,6 +130,10 @@ public class EC2AutoScalingStrategy implements ScalingStrategy<Instance>
@Override
public AutoScalingData<Instance> terminate(List<String> ids)
{
if (ids.isEmpty()) {
return new AutoScalingData<Instance>(Lists.<String>newArrayList(), Lists.<Instance>newArrayList());
}
DescribeInstancesResult result = amazonEC2Client.describeInstances(
new DescribeInstancesRequest()
.withFilters(
@ -184,7 +187,7 @@ public class EC2AutoScalingStrategy implements ScalingStrategy<Instance>
}
@Override
public List<String> ipLookup(List<String> ips)
public List<String> ipToIdLookup(List<String> ips)
{
DescribeInstancesResult result = amazonEC2Client.describeInstances(
new DescribeInstancesRequest()

View File

@ -26,9 +26,9 @@ import java.util.List;
/**
* This class just logs when scaling should occur.
*/
public class NoopScalingStrategy implements ScalingStrategy<String>
public class NoopAutoScalingStrategy implements AutoScalingStrategy<String>
{
private static final EmittingLogger log = new EmittingLogger(NoopScalingStrategy.class);
private static final EmittingLogger log = new EmittingLogger(NoopAutoScalingStrategy.class);
@Override
public AutoScalingData<String> provision()
@ -45,7 +45,7 @@ public class NoopScalingStrategy implements ScalingStrategy<String>
}
@Override
public List<String> ipLookup(List<String> ips)
public List<String> ipToIdLookup(List<String> ips)
{
log.info("I'm not a real strategy so I'm returning what I got %s", ips);
return ips;

View File

@ -0,0 +1,135 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.merger.coordinator.scaling;
import com.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.PeriodGranularity;
import com.metamx.druid.merger.coordinator.TaskRunner;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.joda.time.Period;
import java.util.concurrent.ScheduledExecutorService;
/**
* The ResourceManagementScheduler schedules a check for when worker nodes should potentially be created or destroyed.
* It uses a {@link TaskRunner} to return all pending tasks in the system and the status of the worker nodes in
* the system.
* The ResourceManagementScheduler does not contain the logic to decide whether provision or termination should actually
* occur. That decision is made in the {@link ResourceManagementStrategy}.
*/
public class ResourceManagementScheduler
{
private static final Logger log = new Logger(ResourceManagementScheduler.class);
private final TaskRunner taskRunner;
private final ResourceManagementStrategy resourceManagementStrategy;
private final ResourceManagementSchedulerConfig config;
private final ScheduledExecutorService exec;
private final Object lock = new Object();
private volatile boolean started = false;
public ResourceManagementScheduler(
TaskRunner taskRunner,
ResourceManagementStrategy resourceManagementStrategy,
ResourceManagementSchedulerConfig config,
ScheduledExecutorService exec
)
{
this.taskRunner = taskRunner;
this.resourceManagementStrategy = resourceManagementStrategy;
this.config = config;
this.exec = exec;
}
@LifecycleStart
public void start()
{
synchronized (lock) {
if (started) {
return;
}
ScheduledExecutors.scheduleAtFixedRate(
exec,
config.getProvisionResourcesDuration(),
new Runnable()
{
@Override
public void run()
{
resourceManagementStrategy.doProvision(
taskRunner.getPendingTasks(),
taskRunner.getWorkers()
);
}
}
);
// Schedule termination of worker nodes periodically
Period period = new Period(config.getTerminateResourcesDuration());
PeriodGranularity granularity = new PeriodGranularity(period, config.getTerminateResourcesOriginDateTime(), null);
final long startTime = granularity.next(granularity.truncate(new DateTime().getMillis()));
ScheduledExecutors.scheduleAtFixedRate(
exec,
new Duration(
System.currentTimeMillis(),
startTime
),
config.getTerminateResourcesDuration(),
new Runnable()
{
@Override
public void run()
{
resourceManagementStrategy.doTerminate(
taskRunner.getPendingTasks(),
taskRunner.getWorkers()
);
}
}
);
started = true;
}
}
@LifecycleStop
public void stop()
{
synchronized (lock) {
if (!started) {
return;
}
exec.shutdown();
started = false;
}
}
public ScalingStats getStats()
{
return resourceManagementStrategy.getStats();
}
}

View File

@ -0,0 +1,42 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.merger.coordinator.scaling;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import org.skife.config.Config;
import org.skife.config.Default;
/**
*/
public abstract class ResourceManagementSchedulerConfig
{
@Config("druid.indexer.provisionResources.duration")
@Default("PT1M")
public abstract Duration getProvisionResourcesDuration();
@Config("druid.indexer.terminateResources.duration")
@Default("PT1H")
public abstract Duration getTerminateResourcesDuration();
@Config("druid.indexer.terminateResources.originDateTime")
@Default("2012-01-01T00:55:00.000Z")
public abstract DateTime getTerminateResourcesOriginDateTime();
}

View File

@ -17,17 +17,13 @@
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.loading;
package com.metamx.druid.merger.coordinator.scaling;
import org.skife.config.Config;
import com.metamx.druid.merger.coordinator.TaskRunner;
/**
*/
public abstract class QueryableLoaderConfig extends S3SegmentGetterConfig
public interface ResourceManagementSchedulerFactory
{
@Config("druid.queryable.factory")
public String getQueryableFactoryType()
{
return "mmap";
}
public ResourceManagementScheduler build(TaskRunner runner);
}

View File

@ -0,0 +1,38 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.merger.coordinator.scaling;
import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem;
import com.metamx.druid.merger.coordinator.ZkWorker;
import java.util.Collection;
/**
* The ResourceManagementStrategy decides if worker nodes should be provisioned or determined
* based on the available tasks in the system and the state of the workers in the system.
*/
public interface ResourceManagementStrategy
{
public boolean doProvision(Collection<TaskRunnerWorkItem> runningTasks, Collection<ZkWorker> zkWorkers);
public boolean doTerminate(Collection<TaskRunnerWorkItem> runningTasks, Collection<ZkWorker> zkWorkers);
public ScalingStats getStats();
}

View File

@ -0,0 +1,112 @@
package com.metamx.druid.merger.coordinator.scaling;
import com.google.common.collect.Lists;
import com.google.common.collect.MinMaxPriorityQueue;
import org.joda.time.DateTime;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
/**
*/
public class ScalingStats
{
public static enum EVENT
{
PROVISION,
TERMINATE
}
private static final Comparator<ScalingEvent> comparator = new Comparator<ScalingEvent>()
{
@Override
public int compare(ScalingEvent s1, ScalingEvent s2)
{
return -s1.getTimestamp().compareTo(s2.getTimestamp());
}
};
private final MinMaxPriorityQueue<ScalingEvent> recentNodes;
public ScalingStats(int capacity)
{
this.recentNodes = MinMaxPriorityQueue
.orderedBy(comparator)
.maximumSize(capacity)
.create();
}
public void addProvisionEvent(AutoScalingData data)
{
recentNodes.add(
new ScalingEvent(
data,
new DateTime(),
EVENT.PROVISION
)
);
}
public void addTerminateEvent(AutoScalingData data)
{
recentNodes.add(
new ScalingEvent(
data,
new DateTime(),
EVENT.TERMINATE
)
);
}
public List<ScalingEvent> toList()
{
List<ScalingEvent> retVal = Lists.newArrayList(recentNodes);
Collections.sort(retVal, comparator);
return retVal;
}
public static class ScalingEvent
{
private final AutoScalingData data;
private final DateTime timestamp;
private final EVENT event;
private ScalingEvent(
AutoScalingData data,
DateTime timestamp,
EVENT event
)
{
this.data = data;
this.timestamp = timestamp;
this.event = event;
}
public AutoScalingData getData()
{
return data;
}
public DateTime getTimestamp()
{
return timestamp;
}
public EVENT getEvent()
{
return event;
}
@Override
public String toString()
{
return "ScalingEvent{" +
"data=" + data +
", timestamp=" + timestamp +
", event=" + event +
'}';
}
}
}

View File

@ -0,0 +1,233 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.merger.coordinator.scaling;
import com.google.common.base.Function;
import com.google.common.base.Predicate;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.metamx.common.guava.FunctionalIterable;
import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem;
import com.metamx.druid.merger.coordinator.ZkWorker;
import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager;
import com.metamx.emitter.EmittingLogger;
import org.joda.time.DateTime;
import org.joda.time.Duration;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.ConcurrentSkipListSet;
/**
*/
public class SimpleResourceManagementStrategy implements ResourceManagementStrategy
{
private static final EmittingLogger log = new EmittingLogger(SimpleResourceManagementStrategy.class);
private final AutoScalingStrategy autoScalingStrategy;
private final SimpleResourceManagmentConfig config;
private final WorkerSetupManager workerSetupManager;
private final ScalingStats scalingStats;
private final ConcurrentSkipListSet<String> currentlyProvisioning = new ConcurrentSkipListSet<String>();
private final ConcurrentSkipListSet<String> currentlyTerminating = new ConcurrentSkipListSet<String>();
private volatile DateTime lastProvisionTime = new DateTime();
private volatile DateTime lastTerminateTime = new DateTime();
public SimpleResourceManagementStrategy(
AutoScalingStrategy autoScalingStrategy,
SimpleResourceManagmentConfig config,
WorkerSetupManager workerSetupManager
)
{
this.autoScalingStrategy = autoScalingStrategy;
this.config = config;
this.workerSetupManager = workerSetupManager;
this.scalingStats = new ScalingStats(config.getNumEventsToTrack());
}
@Override
public boolean doProvision(Collection<TaskRunnerWorkItem> pendingTasks, Collection<ZkWorker> zkWorkers)
{
List<String> workerNodeIds = autoScalingStrategy.ipToIdLookup(
Lists.newArrayList(
Iterables.transform(
zkWorkers,
new Function<ZkWorker, String>()
{
@Override
public String apply(ZkWorker input)
{
return input.getWorker().getIp();
}
}
)
)
);
for (String workerNodeId : workerNodeIds) {
currentlyProvisioning.remove(workerNodeId);
}
boolean nothingProvisioning = currentlyProvisioning.isEmpty();
if (nothingProvisioning) {
if (hasTaskPendingBeyondThreshold(pendingTasks)) {
AutoScalingData provisioned = autoScalingStrategy.provision();
if (provisioned != null) {
currentlyProvisioning.addAll(provisioned.getNodeIds());
lastProvisionTime = new DateTime();
scalingStats.addProvisionEvent(provisioned);
return true;
}
}
} else {
Duration durSinceLastProvision = new Duration(new DateTime(), lastProvisionTime);
if (durSinceLastProvision.isLongerThan(config.getMaxScalingDuration())) {
log.makeAlert("Worker node provisioning taking too long")
.addData("millisSinceLastProvision", durSinceLastProvision.getMillis())
.addData("provisioningCount", currentlyProvisioning.size())
.emit();
}
log.info(
"%s still provisioning. Wait for all provisioned nodes to complete before requesting new worker.",
currentlyProvisioning
);
}
return false;
}
@Override
public boolean doTerminate(Collection<TaskRunnerWorkItem> pendingTasks, Collection<ZkWorker> zkWorkers)
{
List<String> workerNodeIds = autoScalingStrategy.ipToIdLookup(
Lists.newArrayList(
Iterables.transform(
zkWorkers,
new Function<ZkWorker, String>()
{
@Override
public String apply(ZkWorker input)
{
return input.getWorker().getIp();
}
}
)
)
);
for (String workerNodeId : workerNodeIds) {
currentlyTerminating.remove(workerNodeId);
}
boolean nothingTerminating = currentlyTerminating.isEmpty();
if (nothingTerminating) {
final int minNumWorkers = workerSetupManager.getWorkerSetupData().getMinNumWorkers();
if (zkWorkers.size() <= minNumWorkers) {
return false;
}
List<ZkWorker> thoseLazyWorkers = Lists.newArrayList(
FunctionalIterable
.create(zkWorkers)
.filter(
new Predicate<ZkWorker>()
{
@Override
public boolean apply(ZkWorker input)
{
return input.getRunningTasks().isEmpty()
&& System.currentTimeMillis() - input.getLastCompletedTaskTime().getMillis()
>= config.getMaxWorkerIdleTimeMillisBeforeDeletion();
}
}
)
);
if (thoseLazyWorkers.isEmpty()) {
return false;
}
AutoScalingData terminated = autoScalingStrategy.terminate(
Lists.transform(
thoseLazyWorkers.subList(minNumWorkers, thoseLazyWorkers.size() - 1),
new Function<ZkWorker, String>()
{
@Override
public String apply(ZkWorker input)
{
return input.getWorker().getIp();
}
}
)
);
if (terminated != null) {
currentlyTerminating.addAll(terminated.getNodeIds());
lastTerminateTime = new DateTime();
scalingStats.addTerminateEvent(terminated);
return true;
}
} else {
Duration durSinceLastTerminate = new Duration(new DateTime(), lastTerminateTime);
if (durSinceLastTerminate.isLongerThan(config.getMaxScalingDuration())) {
log.makeAlert("Worker node termination taking too long")
.addData("millisSinceLastTerminate", durSinceLastTerminate.getMillis())
.addData("terminatingCount", currentlyTerminating.size())
.emit();
}
log.info(
"%s still terminating. Wait for all nodes to terminate before trying again.",
currentlyTerminating
);
}
return false;
}
@Override
public ScalingStats getStats()
{
return scalingStats;
}
private boolean hasTaskPendingBeyondThreshold(Collection<TaskRunnerWorkItem> pendingTasks)
{
long now = System.currentTimeMillis();
for (TaskRunnerWorkItem pendingTask : pendingTasks) {
if (new Duration(pendingTask.getQueueInsertionTime().getMillis(), now).isEqual(config.getMaxPendingTaskDuration())
||
new Duration(
pendingTask.getQueueInsertionTime().getMillis(), now
).isLongerThan(config.getMaxPendingTaskDuration())) {
return true;
}
}
return false;
}
}

View File

@ -0,0 +1,45 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package com.metamx.druid.merger.coordinator.scaling;
import org.joda.time.Duration;
import org.skife.config.Config;
import org.skife.config.Default;
/**
*/
public abstract class SimpleResourceManagmentConfig
{
@Config("druid.indexer.maxWorkerIdleTimeMillisBeforeDeletion")
@Default("600000")
public abstract int getMaxWorkerIdleTimeMillisBeforeDeletion();
@Config("druid.indexer.maxScalingDuration")
@Default("PT15M")
public abstract Duration getMaxScalingDuration();
@Config("druid.indexer.numEventsToTrack")
@Default("50")
public abstract int getNumEventsToTrack();
@Config("druid.indexer.maxPendingTaskDuration")
@Default("PT30S")
public abstract Duration getMaxPendingTaskDuration();
}

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