mirror of https://github.com/apache/druid.git
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:
commit
e67a8757ba
|
@ -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>
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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>()
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -0,0 +1,8 @@
|
|||
package com.metamx.druid;
|
||||
|
||||
import org.joda.time.Interval;
|
||||
|
||||
public interface LogicalSegment
|
||||
{
|
||||
public Interval getInterval();
|
||||
}
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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) {
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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")
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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()
|
||||
);
|
||||
|
|
|
@ -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(),
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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
|
||||
{
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -0,0 +1,6 @@
|
|||
package com.metamx.druid.merger.common.actions;
|
||||
|
||||
public interface TaskActionClient
|
||||
{
|
||||
public <RetType> RetType submit(TaskAction<RetType> taskAction);
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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()
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -82,8 +82,8 @@ public class DefaultMergeTask extends MergeTask
|
|||
}
|
||||
|
||||
@Override
|
||||
public Type getType()
|
||||
public String getType()
|
||||
{
|
||||
return Task.Type.MERGE;
|
||||
return "merge";
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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!");
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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?");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -44,6 +44,11 @@ public class RetryPolicy
|
|||
this.retryCount = 0;
|
||||
}
|
||||
|
||||
public Duration getRetryDelay()
|
||||
{
|
||||
return currRetryDelay;
|
||||
}
|
||||
|
||||
public Duration getAndIncrementRetryDelay()
|
||||
{
|
||||
Duration retVal = new Duration(currRetryDelay);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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 +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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()));
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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 +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -43,4 +43,13 @@ public class AutoScalingData<T>
|
|||
{
|
||||
return nodes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "AutoScalingData{" +
|
||||
"nodeIds=" + nodeIds +
|
||||
", nodes=" + nodes +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
|
@ -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()
|
||||
|
|
|
@ -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;
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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();
|
||||
}
|
|
@ -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 +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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
Loading…
Reference in New Issue