diff --git a/client/pom.xml b/client/pom.xml index 302778496bc..3af4d9b489a 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.10-SNAPSHOT + 0.3.4-SNAPSHOT diff --git a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java index 0612fb9337c..6d896d8bf0f 100644 --- a/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java +++ b/client/src/main/java/com/metamx/druid/client/CachingClusteredClient.java @@ -139,18 +139,24 @@ public class CachingClusteredClient implements QueryRunner // build set of segments to query Set> segments = Sets.newLinkedHashSet(); + List> serversLookup = Lists.newLinkedList(); + for (Interval interval : rewrittenQuery.getIntervals()) { - List> serversLookup = timeline.lookup(interval); + serversLookup.addAll(timeline.lookup(interval)); + } - for (TimelineObjectHolder holder : serversLookup) { - for (PartitionChunk chunk : holder.getObject()) { - ServerSelector selector = chunk.getObject(); - final SegmentDescriptor descriptor = new SegmentDescriptor( - holder.getInterval(), holder.getVersion(), chunk.getChunkNumber() - ); + // Let tool chest filter out unneeded segments + final List> filteredServersLookup = + toolChest.filterSegments(query, serversLookup); - segments.add(Pair.of(selector, descriptor)); - } + for (TimelineObjectHolder holder : filteredServersLookup) { + for (PartitionChunk chunk : holder.getObject()) { + ServerSelector selector = chunk.getObject(); + final SegmentDescriptor descriptor = new SegmentDescriptor( + holder.getInterval(), holder.getVersion(), chunk.getChunkNumber() + ); + + segments.add(Pair.of(selector, descriptor)); } } diff --git a/client/src/main/java/com/metamx/druid/client/ServerInventoryManager.java b/client/src/main/java/com/metamx/druid/client/ServerInventoryManager.java index bd32a62791d..891d41aaec9 100644 --- a/client/src/main/java/com/metamx/druid/client/ServerInventoryManager.java +++ b/client/src/main/java/com/metamx/druid/client/ServerInventoryManager.java @@ -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 { - private static final Map removedSegments = Maps.newHashMap(); + private static final Map removedSegments = new ConcurrentHashMap(); public ServerInventoryManager( ServerInventoryManagerConfig config, diff --git a/client/src/main/java/com/metamx/druid/initialization/Initialization.java b/client/src/main/java/com/metamx/druid/initialization/Initialization.java index 4865208e88e..ba3b42d56e1 100644 --- a/client/src/main/java/com/metamx/druid/initialization/Initialization.java +++ b/client/src/main/java/com/metamx/druid/initialization/Initialization.java @@ -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; /** */ diff --git a/client/src/main/java/com/metamx/druid/merge/ClientKillQuery.java b/client/src/main/java/com/metamx/druid/merge/ClientKillQuery.java new file mode 100644 index 00000000000..02e0a7bd141 --- /dev/null +++ b/client/src/main/java/com/metamx/druid/merge/ClientKillQuery.java @@ -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; + } +} diff --git a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java index 854e2b97c9b..e2d227729d3 100644 --- a/client/src/main/java/com/metamx/druid/query/QueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/QueryToolChest.java @@ -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> +public abstract class QueryToolChest> { - public QueryRunner mergeResults(QueryRunner runner); + public abstract QueryRunner mergeResults(QueryRunner 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> * @param seqOfSequences * @return */ - public Sequence mergeSequences(Sequence> seqOfSequences); - public ServiceMetricEvent.Builder makeMetricBuilder(QueryType query); - public Function makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn); - public TypeReference getResultTypeReference(); - public CacheStrategy getCacheStrategy(QueryType query); - public QueryRunner preMergeQueryDecoration(QueryRunner runner); - public QueryRunner postMergeQueryDecoration(QueryRunner runner); + public abstract Sequence mergeSequences(Sequence> seqOfSequences); + public abstract ServiceMetricEvent.Builder makeMetricBuilder(QueryType query); + public abstract Function makeMetricManipulatorFn(QueryType query, MetricManipulationFn fn); + public abstract TypeReference getResultTypeReference(); + + public CacheStrategy getCacheStrategy(QueryType query) { + return null; + } + + public QueryRunner preMergeQueryDecoration(QueryRunner runner) { + return runner; + } + + public QueryRunner postMergeQueryDecoration(QueryRunner runner) { + return runner; + } + + public List filterSegments(QueryType query, List segments) { + return segments; + } } diff --git a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java index e11f2885fd8..51570e30955 100644 --- a/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/group/GroupByQueryQueryToolChest.java @@ -54,7 +54,7 @@ import java.util.Properties; /** */ -public class GroupByQueryQueryToolChest implements QueryToolChest +public class GroupByQueryQueryToolChest extends QueryToolChest { private static final TypeReference TYPE_REFERENCE = new TypeReference(){}; @@ -177,22 +177,4 @@ public class GroupByQueryQueryToolChest implements QueryToolChest getCacheStrategy(GroupByQuery query) - { - return null; - } - - @Override - public QueryRunner preMergeQueryDecoration(QueryRunner runner) - { - return runner; - } - - @Override - public QueryRunner postMergeQueryDecoration(QueryRunner runner) - { - return runner; - } } diff --git a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index e7df4959b35..6ede9c7f247 100644 --- a/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -51,7 +51,7 @@ import java.util.Map; import java.util.Set; -public class SegmentMetadataQueryQueryToolChest implements QueryToolChest +public class SegmentMetadataQueryQueryToolChest extends QueryToolChest { private static final TypeReference TYPE_REFERENCE = new TypeReference(){}; private static final byte[] SEGMENT_METADATA_CACHE_PREFIX = new byte[]{0x4}; @@ -220,18 +220,6 @@ public class SegmentMetadataQueryQueryToolChest implements QueryToolChest preMergeQueryDecoration(QueryRunner runner) - { - return runner; - } - - @Override - public QueryRunner postMergeQueryDecoration(QueryRunner runner) - { - return runner; - } - private Ordering getOrdering() { return new Ordering() diff --git a/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java b/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java index b813b7febbf..ea6cdbe72f2 100644 --- a/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java +++ b/client/src/main/java/com/metamx/druid/query/search/FragmentSearchQuerySpec.java @@ -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; } } diff --git a/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java b/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java index 1d7a836a3ab..f2ffbc9d1c4 100644 --- a/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java +++ b/client/src/main/java/com/metamx/druid/query/search/InsensitiveContainsSearchQuerySpec.java @@ -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); } @@ -80,8 +83,8 @@ public class InsensitiveContainsSearchQuerySpec implements SearchQuerySpec public String toString() { return "InsensitiveContainsSearchQuerySpec{" + - "value=" + value + - ", sortSpec=" + sortSpec + + "value=" + value + + ", sortSpec=" + sortSpec + "}"; } } diff --git a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java index 3313573adf8..f58f31ebad7 100644 --- a/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/search/SearchQueryQueryToolChest.java @@ -65,7 +65,7 @@ import java.util.Set; /** */ -public class SearchQueryQueryToolChest implements QueryToolChest, SearchQuery> +public class SearchQueryQueryToolChest extends QueryToolChest, SearchQuery> { private static final byte SEARCH_QUERY = 0x2; @@ -263,12 +263,6 @@ public class SearchQueryQueryToolChest implements QueryToolChest> postMergeQueryDecoration(final QueryRunner> runner) - { - return runner; - } - private static class SearchThresholdAdjustingQueryRunner implements QueryRunner> { private final QueryRunner> runner; diff --git a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 7e902115ef8..dcf09526a31 100644 --- a/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -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, TimeBoundaryQuery> + extends QueryToolChest, TimeBoundaryQuery> { private static final byte TIMEBOUNDARY_QUERY = 0x3; @@ -58,6 +58,16 @@ public class TimeBoundaryQueryQueryToolChest { }; + @Override + public List filterSegments(TimeBoundaryQuery query, List input) + { + if(input.size() <= 1) { + return input; + } + + return Lists.newArrayList(input.get(0), input.get(input.size() - 1)); + } + @Override public QueryRunner> mergeResults( final QueryRunner> runner @@ -169,18 +179,6 @@ public class TimeBoundaryQueryQueryToolChest }; } - @Override - public QueryRunner> preMergeQueryDecoration(QueryRunner> runner) - { - return runner; - } - - @Override - public QueryRunner> postMergeQueryDecoration(QueryRunner> runner) - { - return runner; - } - public Ordering> getOrdering() { return Ordering.natural(); diff --git a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 622626dfd57..bef34d9a2da 100644 --- a/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/client/src/main/java/com/metamx/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -61,7 +61,7 @@ import java.util.Map; /** */ -public class TimeseriesQueryQueryToolChest implements QueryToolChest, TimeseriesQuery> +public class TimeseriesQueryQueryToolChest extends QueryToolChest, TimeseriesQuery> { private static final byte TIMESERIES_QUERY = 0x0; @@ -259,12 +259,6 @@ public class TimeseriesQueryQueryToolChest implements QueryToolChest>(runner, Period.months(1)); } - @Override - public QueryRunner> postMergeQueryDecoration(QueryRunner> runner) - { - return runner; - } - public Ordering> getOrdering() { return Ordering.natural(); diff --git a/common/pom.xml b/common/pom.xml index f2f878a6da1..03483c15caa 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.10-SNAPSHOT + 0.3.4-SNAPSHOT @@ -80,8 +80,12 @@ jackson-databind - com.fasterxml.jackson.datatype - jackson-datatype-joda + com.fasterxml.jackson.datatype + jackson-datatype-guava + + + com.fasterxml.jackson.datatype + jackson-datatype-joda org.jdbi diff --git a/common/src/main/java/com/metamx/druid/LogicalSegment.java b/common/src/main/java/com/metamx/druid/LogicalSegment.java new file mode 100644 index 00000000000..5550fb44966 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/LogicalSegment.java @@ -0,0 +1,8 @@ +package com.metamx.druid; + +import org.joda.time.Interval; + +public interface LogicalSegment +{ + public Interval getInterval(); +} diff --git a/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java b/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java index 403fe8bb2cb..d93e5ed8f6b 100644 --- a/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java +++ b/common/src/main/java/com/metamx/druid/TimelineObjectHolder.java @@ -24,7 +24,7 @@ import org.joda.time.Interval; /** */ -public class TimelineObjectHolder +public class TimelineObjectHolder implements LogicalSegment { private final Interval interval; private final VersionType version; @@ -41,6 +41,7 @@ public class TimelineObjectHolder this.object = object; } + @Override public Interval getInterval() { return interval; diff --git a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java index 84514bd6c4c..12079a77959 100644 --- a/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java +++ b/common/src/main/java/com/metamx/druid/jackson/DefaultObjectMapper.java @@ -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); diff --git a/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java b/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java new file mode 100644 index 00000000000..c34b8e7e960 --- /dev/null +++ b/common/src/main/java/com/metamx/druid/utils/CompressionUtils.java @@ -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); + } + } + +} diff --git a/common/src/main/java/com/metamx/druid/utils/JodaUtils.java b/common/src/main/java/com/metamx/druid/utils/JodaUtils.java index 53b14618001..c7feeadc44c 100644 --- a/common/src/main/java/com/metamx/druid/utils/JodaUtils.java +++ b/common/src/main/java/com/metamx/druid/utils/JodaUtils.java @@ -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 intervals) { + return Iterables.any(intervals, new Predicate() + { + @Override + public boolean apply(@Nullable Interval input) + { + return input.overlaps(i); + } + }); + } + public static DateTime minDateTime(DateTime... times) { if (times == null) { diff --git a/druid-services/pom.xml b/druid-services/pom.xml index 90716448ca1..b5b2a7e460c 100644 --- a/druid-services/pom.xml +++ b/druid-services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.2.10-SNAPSHOT + 0.3.4-SNAPSHOT com.metamx druid - 0.2.10-SNAPSHOT + 0.3.4-SNAPSHOT @@ -66,4 +66,4 @@ - \ No newline at end of file + diff --git a/examples/pom.xml b/examples/pom.xml index bfbdeb2f735..ae597675a62 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.10-SNAPSHOT + 0.3.4-SNAPSHOT diff --git a/examples/rand/pom.xml b/examples/rand/pom.xml index 20bf45f8cae..b9ff5296fde 100644 --- a/examples/rand/pom.xml +++ b/examples/rand/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.10-SNAPSHOT + 0.3.4-SNAPSHOT diff --git a/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java b/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java index ecdf2606a3b..92eb86cc801 100644 --- a/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java +++ b/examples/rand/src/main/java/druid/examples/RealtimeStandaloneMain.java @@ -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 diff --git a/examples/twitter/pom.xml b/examples/twitter/pom.xml index 42be38a8916..88f4beae7da 100644 --- a/examples/twitter/pom.xml +++ b/examples/twitter/pom.xml @@ -9,7 +9,7 @@ com.metamx druid-examples - 0.2.10-SNAPSHOT + 0.3.4-SNAPSHOT diff --git a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java index e936d481489..5f4d25cb95b 100644 --- a/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java +++ b/examples/twitter/src/main/java/druid/examples/RealtimeStandaloneMain.java @@ -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 diff --git a/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java b/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java index 1bef29dfc6c..779ba107dab 100644 --- a/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java +++ b/examples/twitter/src/main/java/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java @@ -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") diff --git a/index-common/pom.xml b/index-common/pom.xml index 49db8c7b958..34264feb745 100644 --- a/index-common/pom.xml +++ b/index-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.10-SNAPSHOT + 0.3.4-SNAPSHOT @@ -94,4 +94,4 @@ - \ No newline at end of file + diff --git a/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java b/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java index 93825a8e9f0..8179cd623e4 100644 --- a/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java +++ b/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java @@ -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(); } } diff --git a/indexer/pom.xml b/indexer/pom.xml index ee05ae99737..778f682a0f2 100644 --- a/indexer/pom.xml +++ b/indexer/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.10-SNAPSHOT + 0.3.4-SNAPSHOT @@ -57,14 +57,6 @@ javax.mail mail - - org.codehaus.jackson - jackson-core-asl - - - org.codehaus.jackson - jackson-mapper-asl - diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java index d4ee1941396..f34ff2988f2 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java @@ -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> dims = Maps.newTreeMap(); for(final String dim : inputRow.getDimensions()) { final Set 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> dimAndValues : dims.entrySet()) { final String dim = dimAndValues.getKey(); @@ -510,9 +512,23 @@ public class DeterminePartitionsJob implements Jobby Context context, SortableBytes keyBytes, Iterable combinedIterable ) throws IOException, InterruptedException { - PeekingIterator iterator = Iterators.peekingIterator(combinedIterable.iterator()); + final DateTime bucket = new DateTime(new String(keyBytes.getGroupKey(), Charsets.UTF_8)); + final PeekingIterator 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() ); diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java index eed2339114b..979e2d989a4 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java @@ -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(), diff --git a/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java b/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java index d8eba264c11..0620ba2bc85 100644 --- a/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java +++ b/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java @@ -379,7 +379,8 @@ public class IndexGeneratorJob implements Jobby ); } else if (outputFS instanceof LocalFileSystem) { loadSpec = ImmutableMap.of( - "type", "test" + "type", "local", + "path", indexOutURI.getPath() ); } else { throw new ISE("Unknown file system[%s]", outputFS.getClass()); diff --git a/merger/pom.xml b/merger/pom.xml index cc230b8e921..e39359574a5 100644 --- a/merger/pom.xml +++ b/merger/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.10-SNAPSHOT + 0.3.4-SNAPSHOT diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskCallback.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskCallback.java similarity index 90% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/TaskCallback.java rename to merger/src/main/java/com/metamx/druid/merger/common/TaskCallback.java index 549d0e7c4a7..edf3c9bae86 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskCallback.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskCallback.java @@ -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 { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskHolder.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskHolder.java deleted file mode 100644 index 9abb60d063a..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskHolder.java +++ /dev/null @@ -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; - } -} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskGroup.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskLock.java similarity index 55% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/TaskGroup.java rename to merger/src/main/java/com/metamx/druid/merger/common/TaskLock.java index 85b2ef62a04..1cb82725482 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskGroup.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskLock.java @@ -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 taskSet = new TreeSet( - new Ordering() - { - @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 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() - { - @Override - public Object apply(Task task) - { - return task.getId(); - } - } - ) - ) - ) .toString(); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java index d6cb18093b8..ec226429dd0 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java @@ -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). + *

+ * 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.emptyList(), - Collections.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.emptyList(), Collections.emptyList(), -1); - } - - public static TaskStatus success(String taskId, List segments) - { - return new TaskStatus(taskId, Status.SUCCESS, ImmutableList.copyOf(segments), Collections.emptyList(), -1); - } - - public static TaskStatus continued(String taskId, List nextTasks) - { - Preconditions.checkArgument(nextTasks.size() > 0, "nextTasks.size() > 0"); - return new TaskStatus( - taskId, - Status.CONTINUED, - Collections.emptyList(), - ImmutableList.copyOf(nextTasks), - -1 - ); + return new TaskStatus(taskId, Status.FAILED, -1); } private final String id; - private final List segments; - private final List 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 segments, - @JsonProperty("nextTasks") List 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 getSegments() - { - return segments; - } - - @JsonProperty("nextTasks") - public List 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(); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java index 0cebe1fc91c..58ad3c5cc43 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java @@ -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 getSegmentGetters(final Task task) + public Map getSegments(final Task task, List segments) + throws SegmentLoadingException { - final S3SegmentGetterConfig getterConfig = new S3SegmentGetterConfig() - { - @Override - public File getCacheDirectory() - { - return new File(config.getTaskDir(task), "fetched_segments"); - } - }; + 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.builder() - .put("s3", new S3SegmentPuller(s3Client, getterConfig)) - .put("s3_union", new S3SegmentPuller(s3Client, getterConfig)) - .put("s3_zip", new S3ZippedSegmentPuller(s3Client, getterConfig)) - .build(); + Map retVal = Maps.newLinkedHashMap(); + for (DataSegment segment : segments) { + retVal.put(segment, loader.getSegmentFiles(segment)); + } + + return retVal; } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClient.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClient.java new file mode 100644 index 00000000000..f6740064f52 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClient.java @@ -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 submit(TaskAction 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; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockAcquireAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockAcquireAction.java new file mode 100644 index 00000000000..f669af33625 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockAcquireAction.java @@ -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> +{ + 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> getReturnTypeReference() + { + return new TypeReference>() {}; + } + + @Override + public Optional perform(TaskActionToolbox toolbox) + { + try { + return toolbox.getTaskLockbox().tryLock(task, interval); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java new file mode 100644 index 00000000000..e0e3eddb71f --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java @@ -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> +{ + private final Task task; + + @JsonCreator + public LockListAction( + @JsonProperty("task") Task task + ) + { + this.task = task; + } + + @JsonProperty + public Task getTask() + { + return task; + } + + public TypeReference> getReturnTypeReference() + { + return new TypeReference>() {}; + } + + @Override + public List perform(TaskActionToolbox toolbox) + { + try { + return toolbox.getTaskLockbox().findLocksForTask(task); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockReleaseAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockReleaseAction.java new file mode 100644 index 00000000000..5c84d024a50 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/LockReleaseAction.java @@ -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 +{ + 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 getReturnTypeReference() + { + return new TypeReference() {}; + } + + @Override + public Void perform(TaskActionToolbox toolbox) + { + try { + toolbox.getTaskLockbox().unlock(task, interval); + return null; + } catch (Exception e) { + throw Throwables.propagate(e); + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java new file mode 100644 index 00000000000..26900e29942 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java @@ -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 submit(TaskAction 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 responseDict = jsonMapper.readValue( + response, + new TypeReference>() {} + ); + + 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); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java new file mode 100644 index 00000000000..75ad4a9161f --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java @@ -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 +{ + private final Task task; + private final Set segments; + + @JsonCreator + public SegmentInsertAction( + @JsonProperty("task") Task task, + @JsonProperty("segments") Set segments + ) + { + this.task = task; + this.segments = ImmutableSet.copyOf(segments); + } + + @JsonProperty + public Task getTask() + { + return task; + } + + @JsonProperty + public Set getSegments() + { + return segments; + } + + public TypeReference getReturnTypeReference() + { + return new TypeReference() {}; + } + + @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); + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUnusedAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUnusedAction.java new file mode 100644 index 00000000000..b20d130064e --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUnusedAction.java @@ -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> +{ + 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> getReturnTypeReference() + { + return new TypeReference>() {}; + } + + @Override + public List perform(TaskActionToolbox toolbox) + { + try { + return toolbox.getMergerDBCoordinator().getUnusedSegmentsForInterval(dataSource, interval); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUsedAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUsedAction.java new file mode 100644 index 00000000000..0395057fe83 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUsedAction.java @@ -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> +{ + 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> getReturnTypeReference() + { + return new TypeReference>() {}; + } + + @Override + public List perform(TaskActionToolbox toolbox) + { + try { + return toolbox.getMergerDBCoordinator().getUsedSegmentsForInterval(dataSource, interval); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java new file mode 100644 index 00000000000..f1b61c58d9f --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java @@ -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 +{ + private final Task task; + private final Set segments; + + @JsonCreator + public SegmentNukeAction( + @JsonProperty("task") Task task, + @JsonProperty("segments") Set segments + ) + { + this.task = task; + this.segments = ImmutableSet.copyOf(segments); + } + + @JsonProperty + public Task getTask() + { + return task; + } + + @JsonProperty + public Set getSegments() + { + return segments; + } + + public TypeReference getReturnTypeReference() + { + return new TypeReference() {}; + } + + @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); + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SpawnTasksAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/SpawnTasksAction.java new file mode 100644 index 00000000000..a7a73d8eac7 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/SpawnTasksAction.java @@ -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 +{ + private final Task task; + private final List newTasks; + + @JsonCreator + public SpawnTasksAction( + @JsonProperty("task") Task task, + @JsonProperty("newTasks") List newTasks + ) + { + this.task = task; + this.newTasks = ImmutableList.copyOf(newTasks); + } + + @JsonProperty + public Task getTask() + { + return task; + } + + @JsonProperty + public List getNewTasks() + { + return newTasks; + } + + public TypeReference getReturnTypeReference() + { + return new TypeReference() {}; + } + + @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); + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskAction.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskAction.java new file mode 100644 index 00000000000..7dedf50aad9 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskAction.java @@ -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 +{ + public Task getTask(); // TODO Look into replacing this with task ID so stuff serializes smaller + public TypeReference getReturnTypeReference(); // T_T + public RetType perform(TaskActionToolbox toolbox); +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClient.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClient.java new file mode 100644 index 00000000000..7baa08fe788 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClient.java @@ -0,0 +1,6 @@ +package com.metamx.druid.merger.common.actions; + +public interface TaskActionClient +{ + public RetType submit(TaskAction taskAction); +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionToolbox.java b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionToolbox.java new file mode 100644 index 00000000000..af8f6dcd40c --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionToolbox.java @@ -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 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 taskLocks = getTaskLockbox().findLocksForTask(task); + for(final DataSegment segment : segments) { + final boolean ok = Iterables.any( + taskLocks, new Predicate() + { + @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; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java b/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java new file mode 100644 index 00000000000..5b7609bd042 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java @@ -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()); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java index 7d456d29e01..c26888c4485 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java @@ -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() diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java index ae383f49f60..bb65225a6a5 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java @@ -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; - 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 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(); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java index b3c90fa1330..ce88b2481e6 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java @@ -62,7 +62,7 @@ public class AppendTask extends MergeTask throws Exception { VersionedIntervalTimeline timeline = new VersionedIntervalTimeline( - Ordering.natural().nullsFirst() + Ordering.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 diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java index 4ba3ab38030..4fe6933c2fe 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/DefaultMergeTask.java @@ -82,8 +82,8 @@ public class DefaultMergeTask extends MergeTask } @Override - public Type getType() + public String getType() { - return Task.Type.MERGE; + return "merge"; } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java index 5f37ad2853e..f9d0de34138 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java @@ -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()); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java index 110fdbf86f6..569aa3e8a29 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java @@ -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 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 dimValues = inputRow.getDimension(dim); - if(!unusableDimensions.contains(dim)) { + if (!unusableDimensions.contains(dim)) { - if(dimValues.size() == 1) { + if (dimValues.size() == 1) { // Track this value TreeMultiset 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() + List nextTasks = Lists.transform( + shardSpecs, + new Function() { @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()); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java index b3da8978bd3..f4daec2d90a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java @@ -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 pushedSegments = new CopyOnWriteArrayList(); - 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; } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java index ca514cec52b..246389470ef 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java @@ -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 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!"); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java new file mode 100644 index 00000000000..84deaee5e33 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java @@ -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 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()); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java index 21f6c1e6416..f317efc7040 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java @@ -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 segmentGetters = toolbox.getSegmentGetters(this); - final Map gettedSegments = Maps.newHashMap(); - for (final DataSegment segment : segments) { - Map loadSpec = segment.getLoadSpec(); - SegmentPuller segmentPuller = segmentGetters.get(loadSpec.get("type")); - gettedSegments.put(segment, segmentPuller.getSegmentFiles(segment)); - } + final Map 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 toIdentifier = new Function() { @@ -216,7 +212,13 @@ public abstract class MergeTask extends AbstractTask } }; - final Set current = ImmutableSet.copyOf(Iterables.transform(context.getCurrentSegments(), toIdentifier)); + final Set current = ImmutableSet.copyOf( + Iterables.transform( + toolbox.getTaskActionClient() + .submit(new SegmentListUsedAction(this, getDataSource(), getFixedInterval().get())), + toIdentifier + ) + ); final Set requested = ImmutableSet.copyOf(Iterables.transform(segments, toIdentifier)); final Set 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 segments) diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java index 60d1cf21e39..5ac4dcf71df 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java @@ -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 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). + * 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; } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java b/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java index 32e8bfd71dd..28d3ab1fec1 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java +++ b/merger/src/main/java/com/metamx/druid/merger/common/task/V8toV9UpgradeTask.java @@ -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(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java index 41032cc831f..4bbf0f37a32 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java @@ -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,44 +75,46 @@ public class DbTaskStorage implements TaskStorage log.info("Inserting task %s with status: %s", task.getId(), status); - dbi.withHandle( - new HandleCallback() - { - @Override - public Void withHandle(Handle handle) throws Exception + try { + dbi.withHandle( + new HandleCallback() { - handle.createStatement( - String.format( - "INSERT INTO %s (id, created_date, payload, status_code, status_payload) VALUES (:id, :created_date, :payload, :status_code, :status_payload)", - dbConnectorConfig.getTaskTable() - ) - ) - .bind("id", task.getId()) - .bind("created_date", new DateTime().toString()) - .bind("payload", jsonMapper.writeValueAsString(task)) - .bind("status_code", status.getStatusCode().toString()) - .bind("status_payload", jsonMapper.writeValueAsString(status)) - .execute(); + @Override + public Void withHandle(Handle handle) throws Exception + { + handle.createStatement( + String.format( + "INSERT INTO %s (id, created_date, payload, status_code, status_payload) VALUES (:id, :created_date, :payload, :status_code, :status_payload)", + dbConnectorConfig.getTaskTable() + ) + ) + .bind("id", task.getId()) + .bind("created_date", new DateTime().toString()) + .bind("payload", jsonMapper.writeValueAsString(task)) + .bind("status_code", status.getStatusCode().toString()) + .bind("status_payload", jsonMapper.writeValueAsString(status)) + .execute(); - return null; + return null; + } } - } - ); + ); + } 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() @@ -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 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() + return dbi.withHandle( + new HandleCallback>() { @Override - public Void withHandle(Handle handle) throws Exception + public Optional withHandle(Handle handle) throws Exception { - handle.createStatement( - String.format( - "UPDATE %s SET version = :version WHERE id = :id", - dbConnectorConfig.getTaskTable() + final List> dbTasks = + handle.createQuery( + String.format( + "SELECT payload FROM %s WHERE id = :id", + dbConnectorConfig.getTaskTable() + ) ) - ) - .bind("id", taskid) - .bind("version", version) - .execute(); + .bind("id", taskid) + .list(); - return null; + if(dbTasks.size() == 0) { + return Optional.absent(); + } else { + final Map 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 getVersion(final String taskid) - { - return dbi.withHandle( - new HandleCallback>() - { - @Override - public Optional withHandle(Handle handle) throws Exception - { - final List> 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 dbStatus = Iterables.getOnlyElement(dbStatuses); - return Optional.fromNullable((String) dbStatus.get("version")); - } - } - } - ); - } - @Override public List 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() + { + @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 taskLocks = getLocksWithIds(taskid); + + for(final Map.Entry 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() + { + @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 getLocks(String taskid) + { + return ImmutableList.copyOf( + Iterables.transform( + getLocksWithIds(taskid).entrySet(), new Function, TaskLock>() + { + @Override + public TaskLock apply(Map.Entry e) + { + return e.getValue(); + } + } + ) + ); + } + + @Override + public void addAuditLog(final TaskAction taskAction) + { + Preconditions.checkNotNull(taskAction, "taskAction"); + + log.info("Logging action for task[%s]: %s", taskAction.getTask().getId(), taskAction); + + dbi.withHandle( + new HandleCallback() + { + @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 getAuditLogs(final String taskid) + { + return dbi.withHandle( + new HandleCallback>() + { + @Override + public List withHandle(Handle handle) throws Exception + { + final List> 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, TaskAction>() + { + @Override + public TaskAction apply(Map row) + { + try { + return jsonMapper.readValue(row.get("payload").toString(), TaskAction.class); + } catch(Exception e) { + throw Throwables.propagate(e); + } + } + } + ); + } + } + ); + } + + private Map getLocksWithIds(final String taskid) + { + return dbi.withHandle( + new HandleCallback>() + { + @Override + public Map withHandle(Handle handle) throws Exception + { + final List> 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 retMap = Maps.newHashMap(); + for(final Map row : dbTaskLocks) { + retMap.put((Long)row.get("id"), jsonMapper.readValue(row.get("lock_payload").toString(), TaskLock.class)); + } + return retMap; + } + } + ); + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java index fc03504792f..573152fce09 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskRunner.java @@ -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 getRunningTasks() + { + return Lists.newArrayList(); + } + + @Override + public Collection getPendingTasks() + { + return Lists.newArrayList(); + } + + @Override + public Collection getWorkers() + { + return Lists.newArrayList(); + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java index 16ecd06073f..042f0b8196f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/LocalTaskStorage.java @@ -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,93 +42,178 @@ import java.util.Map; */ public class LocalTaskStorage implements TaskStorage { + private final ReentrantLock giant = new ReentrantLock(); private final Map tasks = Maps.newHashMap(); + private final Multimap taskLocks = HashMultimap.create(); + private final Multimap taskActions = ArrayListMultimap.create(); private static final Logger log = new Logger(LocalTaskStorage.class); @Override public void insert(Task task, TaskStatus status) { - Preconditions.checkNotNull(task, "task"); - Preconditions.checkNotNull(status, "status"); - Preconditions.checkArgument( - task.getId().equals(status.getId()), - "Task/Status ID mismatch[%s/%s]", - task.getId(), - status.getId() - ); - Preconditions.checkState(!tasks.containsKey(task.getId()), "Task ID must not already be present: %s", task.getId()); - log.info("Inserting task %s with status: %s", task.getId(), status); - tasks.put(task.getId(), new TaskStuff(task, status)); + giant.lock(); + + try { + Preconditions.checkNotNull(task, "task"); + Preconditions.checkNotNull(status, "status"); + Preconditions.checkArgument( + task.getId().equals(status.getId()), + "Task/Status ID mismatch[%s/%s]", + task.getId(), + status.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 getTask(String taskid) { - Preconditions.checkNotNull(taskid, "taskid"); - Preconditions.checkNotNull(status, "status"); - Preconditions.checkState(tasks.containsKey(taskid), "Task ID must already be present: %s", taskid); - log.info("Updating task %s to status: %s", taskid, status); - tasks.put(taskid, tasks.get(taskid).withStatus(status)); + 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 getStatus(String taskid) { - Preconditions.checkNotNull(taskid, "taskid"); - if(tasks.containsKey(taskid)) { - return Optional.of(tasks.get(taskid).status); - } else { - return Optional.absent(); - } - } + giant.lock(); - @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 getVersion(String taskid) - { - Preconditions.checkNotNull(taskid, "taskid"); - if(tasks.containsKey(taskid)) { - return tasks.get(taskid).version; - } else { - return Optional.absent(); + try { + Preconditions.checkNotNull(taskid, "taskid"); + if(tasks.containsKey(taskid)) { + return Optional.of(tasks.get(taskid).getStatus()); + } else { + return Optional.absent(); + } + } finally { + giant.unlock(); } } @Override public List getRunningTasks() { - final ImmutableList.Builder listBuilder = ImmutableList.builder(); - for(final TaskStuff taskStuff : tasks.values()) { - if(taskStuff.status.isRunnable()) { - listBuilder.add(taskStuff.task); - } - } + giant.lock(); - return listBuilder.build(); + try { + final ImmutableList.Builder listBuilder = ImmutableList.builder(); + for(final TaskStuff taskStuff : tasks.values()) { + 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 getLocks(final String taskid) + { + giant.lock(); + + try { + return ImmutableList.copyOf(taskLocks.get(taskid)); + } finally { + giant.unlock(); + } + } + + @Override + public void addAuditLog(TaskAction taskAction) + { + giant.lock(); + + try { + taskActions.put(taskAction.getTask().getId(), taskAction); + } finally { + giant.unlock(); + } + } + + @Override + public List 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 version; private TaskStuff(Task task, TaskStatus status) - { - this(task, status, Optional.absent()); - } - - private TaskStuff(Task task, TaskStatus status, Optional 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); } } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java index f226a19be4f..9338bc930c9 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java @@ -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,132 +68,205 @@ public class MergerDBCoordinator this.dbi = dbi; } - public List getSegmentsForInterval(final String dataSource, final Interval interval) throws IOException + public List 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 - // XXX Could be reading from a cache if we can assume we're the only one editing the DB - - final VersionedIntervalTimeline timeline = dbi.withHandle( - new HandleCallback>() + final VersionedIntervalTimeline timeline = dbi.withHandle( + new HandleCallback>() + { + @Override + public VersionedIntervalTimeline withHandle(Handle handle) throws Exception { - @Override - public VersionedIntervalTimeline withHandle(Handle handle) throws Exception - { - final VersionedIntervalTimeline timeline = new VersionedIntervalTimeline( - Ordering.natural() - ); + final VersionedIntervalTimeline timeline = new VersionedIntervalTimeline( + Ordering.natural() + ); - final ResultIterator> dbSegments = - handle.createQuery( - String.format( - "SELECT payload FROM %s WHERE used = 1 AND dataSource = :dataSource", - dbConnectorConfig.getSegmentTable() - ) - ) - .bind("dataSource", dataSource) - .iterator(); - - while (dbSegments.hasNext()) { - - final Map dbSegment = dbSegments.next(); - - DataSegment segment = jsonMapper.readValue( - (String) dbSegment.get("payload"), - DataSegment.class - ); - - timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); - - } - - dbSegments.close(); - - return timeline; - - } - } - ); - - final List segments = Lists.transform( - timeline.lookup(interval), - new Function, DataSegment>() - { - @Override - public DataSegment apply(@Nullable TimelineObjectHolder input) - { - return input.getObject().getChunk(0).getObject(); - } - } - ); - - return segments; - - } - } - - public void announceHistoricalSegment(final DataSegment segment) throws Exception - { - synchronized (lock) { - try { - List> exists = dbi.withHandle( - new HandleCallback>>() - { - @Override - public List> withHandle(Handle handle) throws Exception - { - return handle.createQuery( + final ResultIterator> dbSegments = + handle.createQuery( String.format( - "SELECT id FROM %s WHERE id = ':identifier'", - dbConnectorConfig.getSegmentTable() - ) - ).bind( - "identifier", - segment.getIdentifier() - ).list(); - } - } - ); - - if (!exists.isEmpty()) { - log.info("Found [%s] in DB, not updating DB", segment.getIdentifier()); - return; - } - - dbi.withHandle( - new HandleCallback() - { - @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)", + "SELECT payload FROM %s WHERE used = 1 AND dataSource = :dataSource", dbConnectorConfig.getSegmentTable() ) ) - .bind("id", segment.getIdentifier()) - .bind("dataSource", segment.getDataSource()) - .bind("created_date", new DateTime().toString()) - .bind("start", segment.getInterval().getStart().toString()) - .bind("end", segment.getInterval().getEnd().toString()) - .bind("partitioned", segment.getShardSpec().getPartitionNum()) - .bind("version", segment.getVersion()) - .bind("used", true) - .bind("payload", jsonMapper.writeValueAsString(segment)) - .execute(); + .bind("dataSource", dataSource) + .iterator(); + + while (dbSegments.hasNext()) { + + final Map dbSegment = dbSegments.next(); + + DataSegment segment = jsonMapper.readValue( + (String) dbSegment.get("payload"), + DataSegment.class + ); + + timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); - 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); + dbSegments.close(); + + return timeline; + + } + } + ); + + final List segments = Lists.transform( + timeline.lookup(interval), + new Function, DataSegment>() + { + @Override + public DataSegment apply(TimelineObjectHolder input) + { + return input.getObject().getChunk(0).getObject(); + } + } + ); + + return segments; + } + + public void announceHistoricalSegments(final Set segments) throws Exception + { + dbi.inTransaction( + new TransactionCallback() + { + @Override + public Void inTransaction(Handle handle, TransactionStatus transactionStatus) throws Exception + { + for(final DataSegment segment : segments) { + announceHistoricalSegment(handle, segment); + } + + return null; + } + } + ); + } + + + private void announceHistoricalSegment(final Handle handle, final DataSegment segment) throws Exception + { + try { + final List> exists = handle.createQuery( + String.format( + "SELECT id FROM %s WHERE id = ':identifier'", + dbConnectorConfig.getSegmentTable() + ) + ).bind( + "identifier", + segment.getIdentifier() + ).list(); + + if (!exists.isEmpty()) { + log.info("Found [%s] in DB, not updating DB", segment.getIdentifier()); + return; } + + 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)", + dbConnectorConfig.getSegmentTable() + ) + ) + .bind("id", segment.getIdentifier()) + .bind("dataSource", segment.getDataSource()) + .bind("created_date", new DateTime().toString()) + .bind("start", segment.getInterval().getStart().toString()) + .bind("end", segment.getInterval().getEnd().toString()) + .bind("partitioned", segment.getShardSpec().getPartitionNum()) + .bind("version", segment.getVersion()) + .bind("used", true) + .bind("payload", jsonMapper.writeValueAsString(segment)) + .execute(); + + log.info("Published segment [%s] to DB", segment.getIdentifier()); + } + catch (Exception e) { + log.error(e, "Exception inserting into DB"); + throw e; } } + + public void deleteSegments(final Set segments) throws Exception + { + dbi.inTransaction( + new TransactionCallback() + { + @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 getUnusedSegmentsForInterval(final String dataSource, final Interval interval) + { + List matchingSegments = dbi.withHandle( + new HandleCallback>() + { + @Override + public List 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.newArrayList(), + new Folder3, Map>() + { + @Override + public List fold( + List accumulator, + Map 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; + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java index 4157d6a66ca..06dd63730cc 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java @@ -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. *

- * 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. *

* 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 zkWorkers = new ConcurrentHashMap(); - // all tasks that are assigned or need to be assigned - private final Map tasks = new ConcurrentHashMap(); + private final Map zkWorkers = new ConcurrentHashMap(); + // 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 currentlyProvisioning = new ConcurrentSkipListSet(); - private final ConcurrentSkipListSet currentlyTerminating = new ConcurrentSkipListSet(); 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 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() - { - @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 getWorkers() { - return started; - } - - public int getNumWorkers() - { - return zkWorkers.size(); + return zkWorkers.values(); } @Override - public void run(Task task, TaskContext context, TaskCallback callback) + public Collection getRunningTasks() { - if (tasks.containsKey(task.getId())) { - throw new ISE("Assigned a task[%s] that already exists, WTF is happening?!", task.getId()); - } - TaskWrapper taskWrapper = new TaskWrapper( - task, context, callback, retryPolicyFactory.makeRetryPolicy() - ); - tasks.put(taskWrapper.getTask().getId(), taskWrapper); - assignTask(taskWrapper); + return runningTasks.values(); } - private void assignTask(TaskWrapper taskWrapper) + @Override + public Collection getPendingTasks() { - WorkerWrapper workerWrapper = findWorkerRunningTask(taskWrapper); + return pendingTasks.values(); + } - // 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()); + public ZkWorker findWorkerRunningTask(String taskId) + { + for (ZkWorker zkWorker : zkWorkers.values()) { + if (zkWorker.getRunningTasks().contains(taskId)) { + return zkWorker; + } + } + return null; + } - TaskStatus taskStatus = jsonMapper.readValue( - workerWrapper.getStatusCache() - .getCurrentData( - JOINER.join(config.getStatusPath(), worker.getHost(), taskWrapper.getTask().getId()) - ) - .getData(), - TaskStatus.class - ); + /** + * 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() + ); + addPendingTask(taskRunnerWorkItem); + } - if (taskStatus.isComplete()) { - TaskCallback callback = taskWrapper.getCallback(); - if (callback != null) { - callback.notify(taskStatus); + 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() + { + @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 copy = Lists.newArrayList(pendingTasks.values()); + for (TaskRunnerWorkItem taskWrapper : copy) { + assignTask(taskWrapper); + } + + return null; } - 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); - } + ); + + try { + future.get(); + } + 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.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,146 +414,153 @@ 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)) { - final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); - final TaskStatus taskStatus; - // This can fail if a worker writes a bogus status. Retry if so. - try { - taskStatus = jsonMapper.readValue( - event.getData().getData(), TaskStatus.class - ); + try { + if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED) || + event.getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED)) { + final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); + final TaskStatus taskStatus; - if (!taskStatus.getId().equals(taskId)) { - // Sanity check - throw new ISE( - "Worker[%s] status id does not match payload id: %s != %s", - worker.getHost(), - taskId, - taskStatus.getId() - ); - } - } - 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)); - throw Throwables.propagate(e); - } - - log.info( - "Worker[%s] wrote %s status for task: %s", - worker.getHost(), - taskStatus.getStatusCode(), - taskId + // This can fail if a worker writes a bogus status. Retry if so. + try { + taskStatus = jsonMapper.readValue( + event.getData().getData(), TaskStatus.class ); - statusLock.notify(); - - if (taskStatus.isComplete()) { - // Worker is done with this task - workerWrapper.setLastCompletedTaskTime(new DateTime()); - final TaskWrapper taskWrapper = tasks.get(taskId); - - if (taskWrapper == null) { - log.warn( - "WTF?! Worker[%s] completed a task I didn't know about: %s", - worker.getHost(), - taskId - ); - } else { - final TaskCallback callback = taskWrapper.getCallback(); - - // Cleanup - if (callback != null) { - callback.notify(taskStatus); - } - tasks.remove(taskId); - cf.delete().guaranteed().inBackground().forPath(event.getData().getPath()); - } + if (!taskStatus.getId().equals(taskId)) { + // Sanity check + throw new ISE( + "Worker[%s] status id does not match payload id: %s != %s", + worker.getHost(), + taskId, + taskStatus.getId() + ); } } + catch (Exception e) { + log.warn(e, "Worker[%s] wrote bogus status for task: %s", worker.getHost(), taskId); + retryTask(runningTasks.get(taskId), worker.getHost()); + throw Throwables.propagate(e); + } + + log.info( + "Worker[%s] wrote %s status for task: %s", + worker.getHost(), + taskStatus.getStatusCode(), + taskId + ); + + // Synchronizing state with ZK + synchronized (statusLock) { + statusLock.notify(); + } + + final TaskRunnerWorkItem taskRunnerWorkItem = runningTasks.get(taskId); + if (taskRunnerWorkItem == null) { + log.warn( + "WTF?! Worker[%s] announcing a status for a task I didn't know about: %s", + worker.getHost(), + taskId + ); + } + + if (taskStatus.isComplete()) { + if (taskRunnerWorkItem != null) { + final TaskCallback callback = taskRunnerWorkItem.getCallback(); + if (callback != null) { + callback.notify(taskStatus); + } + } + + // 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()); + } } - catch (Exception e) { - log.makeAlert(e, "Failed to handle new worker status") - .addData("worker", worker.getHost()) - .addData("znode", event.getData().getPath()) - .emit(); - } + } + catch (Exception e) { + log.makeAlert(e, "Failed to handle new worker status") + .addData("worker", worker.getHost()) + .addData("znode", event.getData().getPath()) + .emit(); } } } ); - 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 tasksToRetry = Sets.newHashSet(workerWrapper.getRunningTasks()); - tasksToRetry.addAll(cf.getChildren().forPath(JOINER.join(config.getTaskPath(), worker.getHost()))); + Set 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()); } } - zkWorkers.remove(worker.getHost()); } - private WorkerWrapper findWorkerForTask() + private ZkWorker findWorkerForTask() { try { - final MinMaxPriorityQueue workerQueue = MinMaxPriorityQueue.orderedBy( - new Comparator() + final MinMaxPriorityQueue workerQueue = MinMaxPriorityQueue.orderedBy( + new Comparator() { @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() + new Predicate() { @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?"); - } - } - } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java index fbb27d76c18..632a1fcc985 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/RetryPolicy.java @@ -44,6 +44,11 @@ public class RetryPolicy this.retryCount = 0; } + public Duration getRetryDelay() + { + return currRetryDelay; + } + public Duration getAndIncrementRetryDelay() { Duration retVal = new Duration(currRetryDelay); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskExistsException.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskExistsException.java new file mode 100644 index 00000000000..ddc146c3167 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskExistsException.java @@ -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; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java new file mode 100644 index 00000000000..b62bca604d4 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java @@ -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> 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 tryLock(final Task task, final Interval interval) + { + return tryLock(task, interval, Optional.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 tryLock(final Task task, final Interval interval, final Optional 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 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(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 findLocksForTask(final Task task) + { + giant.lock(); + + try { + return Lists.transform( + findLockPossesForTask(task), new Function() + { + @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 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 findLockPossesForTask(final Task task) + { + giant.lock(); + + try { + final Iterable 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 dsRunning = running.get(task.getDataSource()); + if(dsRunning == null) { + searchSpace = ImmutableList.of(); + } else { + searchSpace = dsRunning.values(); + } + } + + return ImmutableList.copyOf( + Iterables.filter( + searchSpace, new Predicate() + { + @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 findLockPossesForInterval(final String dataSource, final Interval interval) + { + giant.lock(); + + try { + final NavigableMap 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 dsLockbox = dsRunning.navigableKeySet(); + final Iterable 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() + { + @Override + public boolean apply(@Nullable Interval searchInterval) + { + return searchInterval != null && searchInterval.overlaps(interval); + } + } + ) + .transform( + new Function() + { + @Override + public TaskLockPosse apply(Interval interval) + { + return dsRunning.get(interval); + } + } + ) + ); + } + } + finally { + giant.unlock(); + } + } + + private static class TaskLockPosse + { + final private TaskLock taskLock; + final private Set taskIds; + + public TaskLockPosse(TaskLock taskLock) + { + this.taskLock = taskLock; + taskIds = Sets.newHashSet(); + } + + public TaskLock getTaskLock() + { + return taskLock; + } + + public Set getTaskIds() + { + return taskIds; + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMaster.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java similarity index 74% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMaster.java rename to merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java index 0481e8f21b6..b9fefecf013 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMaster.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java @@ -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; + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java index f674b8744cd..377004d473f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java @@ -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. - * + *

* 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. - * + *

+ * 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. + *

+ * For persistence, the queue saves new tasks from {@link #add} and task status updates from {@link #notify} using a + * {@link TaskStorage} obj + *

* 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 queue = Lists.newLinkedList(); - private final Map> 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 runningTasks = taskStorage.getRunningTasks(); + + for(final Task task : runningTasks) { + queue.add(task); + } + + // Get all locks, along with which tasks they belong to + final Multimap 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 byVersionOrdering = new Ordering() + { + @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 taskAndLock : tasksByLock.entries()) { + final Task task = taskAndLock.getValue(); + final TaskLock savedTaskLock = taskAndLock.getKey(); + + final Optional 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> runningTasks = Lists.transform( - taskStorage.getRunningTasks(), - new Function>() - { - @Override - public Pair apply(Task task) - { - return Pair.of(task, taskStorage.getVersion(task.getId()).orNull()); - } - } - ); - - // Sort by version, with nulls last - final Ordering> byVersionOrdering = new Ordering>() - { - final private Ordering baseOrdering = Ordering.natural().nullsLast(); - - @Override - public int compare( - Pair left, Pair right - ) - { - return baseOrdering.compare(left.rhs, right.rhs); - } - }; - - for(final Pair taskAndVersion : byVersionOrdering.sortedCopy(runningTasks)) { - final Task task = taskAndVersion.lhs; - final String preferredVersion = taskAndVersion.rhs; - - queue.add(task); - - if(preferredVersion != null) { - final Optional 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!"); - taskStorage.insert(task, TaskStatus.running(task.getId())); + // 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(); + log.info("Waiting for work..."); + + 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 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 maybeLock = taskLockbox.tryLock(task, task.getFixedInterval().get()); + if(maybeLock.isPresent()) { + log.info("Task claimed with fixed interval lock: %s", task.getId()); + queue.remove(task); + 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); - log.info("Task claimed: %s", task); - return new VersionedTaskWrapper(task, maybeVersion.get()); + return task; } } - log.info("No doable work found."); return null; } finally { @@ -255,312 +298,61 @@ 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 maybeTaskGroup = Lists.newArrayList( - FunctionalIterable.create(findLocks(dataSource, interval)) - .filter( - new Predicate() - { - @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 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); - - 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 getStatus(final String taskid) - { - giant.lock(); - - try { - final Optional statusOptional = taskStorage.getStatus(taskid); - if(statusOptional.isPresent()) { - // See if we can collapse this down - return Optional.of(collapseStatus(statusOptional.get())); - } else { - return statusOptional; - } - } - finally { - giant.unlock(); - } - } - - private TaskStatus collapseStatus(TaskStatus status) - { - - if (status.isContinued()) { - - int nSubtasks = 0; - int nSuccesses = 0; - List 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 tryLock(final Task task) - { - return tryLock(task, Optional.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 tryLock(final Task task, final Optional preferredVersion) - { - giant.lock(); - - try { - - final String dataSource = task.getDataSource(); - final Interval interval = task.getInterval(); - - final List 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; + if(taskStatus.isComplete()) { + if(didPersistStatus) { + log.info("Task done: %s", task); + taskLockbox.unlock(task); + workMayBeAvailable.signalAll(); } else { - return Optional.absent(); + // 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); } - - } else { - - // No existing locks. We can make a new one. - if (!running.containsKey(dataSource)) { - running.put(dataSource, new TreeMap(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 findLocks(final String dataSource, final Interval interval) - { - giant.lock(); - - try { - final NavigableMap 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 dsLockbox = dsRunning.navigableKeySet(); - final Iterable 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() - { - @Override - public boolean apply(@Nullable Interval searchInterval) - { - return searchInterval != null && searchInterval.overlaps(interval); - } - } - ) - .transform( - new Function() - { - @Override - public TaskGroup apply(Interval interval) - { - return dsRunning.get(interval); - } - } - ) - ); } } finally { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java index 5362741ac92..756c4b793fd 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java @@ -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 getRunningTasks(); + + public Collection getPendingTasks(); + + public Collection getWorkers(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerWorkItem.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerWorkItem.java new file mode 100644 index 00000000000..9e32c1bc926 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerWorkItem.java @@ -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 +{ + 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 + + '}'; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/VersionedTaskWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerWorkQueue.java similarity index 70% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/VersionedTaskWrapper.java rename to merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerWorkQueue.java index 4eb0726ef69..f0dc161ad1a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/VersionedTaskWrapper.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerWorkQueue.java @@ -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 { - 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())); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java index 13308db86a2..2e21f52876b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java @@ -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 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 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 getVersion(String taskid); + public void addAuditLog(TaskAction taskAction); + + /** + * Returns all actions taken by a task. + */ + public List getAuditLogs(String taskid); /** * Returns a list of currently-running tasks as stored in the storage facility, in no particular order. */ public List getRunningTasks(); + + /** + * Returns a list of locks for a particular task. + */ + public List getLocks(String taskid); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorageQueryAdapter.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorageQueryAdapter.java new file mode 100644 index 00000000000..da8269f140f --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorageQueryAdapter.java @@ -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 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> getSameGroupChildStatuses(final String taskid) + { + final Optional taskOptional = storage.getTask(taskid); + final Optional statusOptional = storage.getStatus(taskid); + final ImmutableMap.Builder> resultBuilder = ImmutableMap.builder(); + + resultBuilder.put(taskid, statusOptional); + + final Iterable nextTasks = FunctionalIterable + .create(storage.getAuditLogs(taskid)).filter( + new Predicate() + { + @Override + public boolean apply(TaskAction taskAction) + { + return taskAction instanceof SpawnTasksAction; + } + } + ).transformCat( + new Function>() + { + @Override + public Iterable 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 getSameGroupMergedStatus(final String taskid) + { + final Map> statuses = getSameGroupChildStatuses(taskid); + + int nSuccesses = 0; + int nFailures = 0; + int nTotal = 0; + + for(final Optional statusOption : statuses.values()) { + nTotal ++; + + if(statusOption.isPresent()) { + final TaskStatus status = statusOption.get(); + + if(status.isSuccess()) { + nSuccesses ++; + } else if(status.isFailure()) { + nFailures ++; + } + } + } + + final Optional 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 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 taskOptional = storage.getTask(taskid); + final Set segments = Sets.newHashSet(); + final List 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; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java deleted file mode 100644 index c757bb2dc33..00000000000 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskWrapper.java +++ /dev/null @@ -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; - } -} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/ZkWorker.java similarity index 84% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java rename to merger/src/main/java/com/metamx/druid/merger/coordinator/ZkWorker.java index 8203781bbf9..ff5ad95bf04 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/WorkerWrapper.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/ZkWorker.java @@ -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 + + '}'; + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerCoordinatorConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerCoordinatorConfig.java index 6d6218c3bff..b727f805bb2 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerCoordinatorConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerCoordinatorConfig.java @@ -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 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(); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java index 5b342883bb2..4e2265789cf 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java @@ -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(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java index 44b3a1d4c8c..14e1656e8b8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java @@ -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(); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java index 78326d3a3cc..14135174e59 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java @@ -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())); + // Retry would be nice, but only after we have a way to throttle and limit them. Just fail for now. + 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; - - // 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) { - 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() - ) - ); + 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) { + log.info("Abandoning task due to shutdown: %s", task.getId()); + return; + } + + queue.notify(task, statusFromRunner); + + // 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 (statusFromRunner.isFailure()) { + log.makeAlert("Failed to index") + .addData("task", task.getId()) + .addData("type", task.getType()) + .addData("dataSource", task.getDataSource()) + .addData("interval", task.getFixedInterval()) + .emit(); } - if (!task.getInterval().contains(segment.getInterval())) { - throw new IllegalStateException( - String.format( - "Segment for task[%s] has invalid interval: %s", - task.getId(), - segment.getIdentifier() - ) - ); - } - - 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); + log.info( + "Task %s: %s (%d run duration)", + statusFromRunner.getStatusCode(), + task, + statusFromRunner.getDuration() + ); } } 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()) { - log.makeAlert("Failed to index") + log.makeAlert(e, "Failed to handle task callback") .addData("task", task.getId()) - .addData("type", task.getType().toString()) - .addData("dataSource", task.getDataSource()) - .addData("interval", task.getInterval()) + .addData("statusCode", statusFromRunner.getStatusCode()) .emit(); } - - log.info( - "Task %s: %s (%d segments) (%d run duration)", - status.getStatusCode(), - task, - status.getSegments().size(), - status.getDuration() - ); - } } ); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorInfoResource.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorInfoResource.java new file mode 100644 index 00000000000..7e90ade74a6 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorInfoResource.java @@ -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(); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java index 7d075cbe9e8..f684981cf2d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java @@ -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 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 (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") + ) + ); + } + + public void initializeTaskToolbox() { if (taskToolbox == null) { - final RestS3Service s3Client = 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), + 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; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java index e4acd93514f..fd3e02bcad3 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorResource.java @@ -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 status = tasks.getStatus(taskid); + final Optional 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 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 status = taskStorageQueryAdapter.getSameGroupMergedStatus(taskid); + final Set segments = taskStorageQueryAdapter.getSameGroupNewSegments(taskid); + + final Map ret = jsonMapper.convertValue(status, new TypeReference>(){}); + 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 Response doAction(final TaskAction action) + { + final T ret = taskMasterLifecycle.getTaskToolbox().getTaskActionClient().submit(action); + final Map retMap = Maps.newHashMap(); + retMap.put("result", ret); + + return Response.ok().entity(retMap).build(); + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java index 1012beb830b..f5bdfd6cbb5 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java @@ -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); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java index 5a1bb4980e5..0ca74a9b38c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java @@ -43,4 +43,13 @@ public class AutoScalingData { return nodes; } + + @Override + public String toString() + { + return "AutoScalingData{" + + "nodeIds=" + nodeIds + + ", nodes=" + nodes + + '}'; + } } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingStrategy.java similarity index 81% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java rename to merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingStrategy.java index 150de1357e0..7ab92a0b985 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingStrategy.java @@ -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 +public interface AutoScalingStrategy { public AutoScalingData provision(); @@ -31,8 +32,8 @@ public interface ScalingStrategy /** * Provides a lookup of ip addresses to node ids - * @param ips - * @return + * @param ips - nodes ips + * @return node ids */ - public List ipLookup(List ips); + public List ipToIdLookup(List ips); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java index 16b629d9abb..47ef22152dd 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java @@ -44,7 +44,7 @@ import java.util.List; /** */ -public class EC2AutoScalingStrategy implements ScalingStrategy +public class EC2AutoScalingStrategy implements AutoScalingStrategy { private static final EmittingLogger log = new EmittingLogger(EC2AutoScalingStrategy.class); @@ -73,7 +73,6 @@ public class EC2AutoScalingStrategy implements ScalingStrategy 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 @Override public AutoScalingData terminate(List ids) { + if (ids.isEmpty()) { + return new AutoScalingData(Lists.newArrayList(), Lists.newArrayList()); + } + DescribeInstancesResult result = amazonEC2Client.describeInstances( new DescribeInstancesRequest() .withFilters( @@ -184,7 +187,7 @@ public class EC2AutoScalingStrategy implements ScalingStrategy } @Override - public List ipLookup(List ips) + public List ipToIdLookup(List ips) { DescribeInstancesResult result = amazonEC2Client.describeInstances( new DescribeInstancesRequest() diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopAutoScalingStrategy.java similarity index 90% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java rename to merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopAutoScalingStrategy.java index 2b412ca6202..d4a5f355c6f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopScalingStrategy.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopAutoScalingStrategy.java @@ -26,9 +26,9 @@ import java.util.List; /** * This class just logs when scaling should occur. */ -public class NoopScalingStrategy implements ScalingStrategy +public class NoopAutoScalingStrategy implements AutoScalingStrategy { - private static final EmittingLogger log = new EmittingLogger(NoopScalingStrategy.class); + private static final EmittingLogger log = new EmittingLogger(NoopAutoScalingStrategy.class); @Override public AutoScalingData provision() @@ -45,7 +45,7 @@ public class NoopScalingStrategy implements ScalingStrategy } @Override - public List ipLookup(List ips) + public List ipToIdLookup(List ips) { log.info("I'm not a real strategy so I'm returning what I got %s", ips); return ips; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementScheduler.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementScheduler.java new file mode 100644 index 00000000000..45c0060fda6 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementScheduler.java @@ -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(); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerConfig.java new file mode 100644 index 00000000000..924fe897490 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerConfig.java @@ -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(); +} diff --git a/server/src/main/java/com/metamx/druid/loading/QueryableLoaderConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerFactory.java similarity index 77% rename from server/src/main/java/com/metamx/druid/loading/QueryableLoaderConfig.java rename to merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerFactory.java index c6e2a72c931..8d33c8be220 100644 --- a/server/src/main/java/com/metamx/druid/loading/QueryableLoaderConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerFactory.java @@ -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); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementStrategy.java new file mode 100644 index 00000000000..7e2d666c3bd --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementStrategy.java @@ -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 runningTasks, Collection zkWorkers); + + public boolean doTerminate(Collection runningTasks, Collection zkWorkers); + + public ScalingStats getStats(); +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStats.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStats.java new file mode 100644 index 00000000000..7d65b9aa2ef --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStats.java @@ -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 comparator = new Comparator() + + { + @Override + public int compare(ScalingEvent s1, ScalingEvent s2) + { + return -s1.getTimestamp().compareTo(s2.getTimestamp()); + } + }; + + private final MinMaxPriorityQueue 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 toList() + { + List 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 + + '}'; + } + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java new file mode 100644 index 00000000000..1cd3f8e46e6 --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java @@ -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 currentlyProvisioning = new ConcurrentSkipListSet(); + private final ConcurrentSkipListSet currentlyTerminating = new ConcurrentSkipListSet(); + + 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 pendingTasks, Collection zkWorkers) + { + List workerNodeIds = autoScalingStrategy.ipToIdLookup( + Lists.newArrayList( + Iterables.transform( + zkWorkers, + new Function() + { + @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 pendingTasks, Collection zkWorkers) + { + List workerNodeIds = autoScalingStrategy.ipToIdLookup( + Lists.newArrayList( + Iterables.transform( + zkWorkers, + new Function() + { + @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 thoseLazyWorkers = Lists.newArrayList( + FunctionalIterable + .create(zkWorkers) + .filter( + new Predicate() + { + @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() + { + @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 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; + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagmentConfig.java b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagmentConfig.java new file mode 100644 index 00000000000..19fc2fdcedb --- /dev/null +++ b/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagmentConfig.java @@ -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(); +} diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java index c41f4ac8be4..d7836d46faa 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/TaskMonitor.java @@ -21,11 +21,9 @@ package com.metamx.druid.merger.worker; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.druid.merger.common.TaskHolder; 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.TaskContext; import com.metamx.emitter.EmittingLogger; import com.netflix.curator.framework.CuratorFramework; import com.netflix.curator.framework.recipes.cache.PathChildrenCache; @@ -33,7 +31,6 @@ import com.netflix.curator.framework.recipes.cache.PathChildrenCacheEvent; import com.netflix.curator.framework.recipes.cache.PathChildrenCacheListener; import org.apache.commons.io.FileUtils; - import java.io.File; import java.util.concurrent.ExecutorService; @@ -84,14 +81,14 @@ public class TaskMonitor throws Exception { if (pathChildrenCacheEvent.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) { - final TaskHolder taskHolder = toolbox.getObjectMapper().readValue( + final Task task = toolbox.getObjectMapper().readValue( cf.getData().forPath(pathChildrenCacheEvent.getData().getPath()), - TaskHolder.class + Task.class ); - final Task task = taskHolder.getTask(); - final TaskContext taskContext = taskHolder.getTaskContext(); if (workerCuratorCoordinator.statusExists(task.getId())) { + log.warn("Got task %s that I am already running...", task.getId()); + workerCuratorCoordinator.unannounceTask(task.getId()); return; } @@ -110,7 +107,7 @@ public class TaskMonitor try { workerCuratorCoordinator.unannounceTask(task.getId()); workerCuratorCoordinator.announceStatus(TaskStatus.running(task.getId())); - taskStatus = task.run(taskContext, toolbox); + taskStatus = task.run(toolbox); } catch (Exception e) { log.makeAlert(e, "Failed to run task") diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java b/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java index 5b5f3a0a6e7..afb217c5e77 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java @@ -39,6 +39,10 @@ public abstract class WorkerConfig @Config("druid.worker.version") public abstract String getVersion(); + @Config("druid.worker.masterService") + public abstract String getMasterService(); + + @Config("druid.worker.capacity") public int getCapacity() { return Runtime.getRuntime().availableProcessors() - 1; diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java index 237ae09d854..d02ffa5d9e3 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java @@ -35,18 +35,22 @@ import com.metamx.druid.http.StatusServlet; import com.metamx.druid.initialization.CuratorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; +import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.loading.S3DataSegmentPusher; +import com.metamx.druid.loading.S3DataSegmentPusherConfig; +import com.metamx.druid.loading.S3SegmentKiller; +import com.metamx.druid.loading.SegmentKiller; import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.actions.RemoteTaskActionClient; import com.metamx.druid.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.config.IndexerCoordinatorConfig; import com.metamx.druid.merger.worker.TaskMonitor; import com.metamx.druid.merger.worker.Worker; import com.metamx.druid.merger.worker.WorkerCuratorCoordinator; import com.metamx.druid.merger.worker.config.WorkerConfig; -import com.metamx.druid.loading.S3SegmentPusher; -import com.metamx.druid.loading.S3SegmentPusherConfig; -import com.metamx.druid.loading.SegmentPusher; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitters; @@ -61,8 +65,8 @@ 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 com.netflix.curator.x.discovery.ServiceDiscovery; +import com.netflix.curator.x.discovery.ServiceProvider; import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.security.AWSCredentials; @@ -96,12 +100,16 @@ public class WorkerNode extends RegisteringNode private final Properties props; private final ConfigurationObjectFactory configFactory; + private RestS3Service s3Service = null; private List monitors = null; + private HttpClient httpClient = null; private ServiceEmitter emitter = null; - private IndexerCoordinatorConfig coordinatorConfig = null; // FIXME needed for task toolbox, but shouldn't be + private TaskConfig taskConfig = null; private WorkerConfig workerConfig = null; private TaskToolbox taskToolbox = null; private CuratorFramework curatorFramework = null; + private ServiceDiscovery serviceDiscovery = null; + private ServiceProvider coordinatorServiceProvider = null; private WorkerCuratorCoordinator workerCuratorCoordinator = null; private TaskMonitor taskMonitor = null; private Server server = null; @@ -123,12 +131,24 @@ public class WorkerNode extends RegisteringNode this.configFactory = configFactory; } + public WorkerNode setHttpClient(HttpClient httpClient) + { + this.httpClient = httpClient; + return this; + } + public WorkerNode setEmitter(ServiceEmitter emitter) { this.emitter = emitter; return this; } + public WorkerNode setS3Service(RestS3Service s3Service) + { + this.s3Service = s3Service; + return this; + } + public WorkerNode setTaskToolbox(TaskToolbox taskToolbox) { this.taskToolbox = taskToolbox; @@ -141,6 +161,18 @@ public class WorkerNode extends RegisteringNode return this; } + public WorkerNode setCoordinatorServiceProvider(ServiceProvider coordinatorServiceProvider) + { + this.coordinatorServiceProvider = coordinatorServiceProvider; + return this; + } + + public WorkerNode setServiceDiscovery(ServiceDiscovery serviceDiscovery) + { + this.serviceDiscovery = serviceDiscovery; + return this; + } + public WorkerNode setWorkerCuratorCoordinator(WorkerCuratorCoordinator workerCuratorCoordinator) { this.workerCuratorCoordinator = workerCuratorCoordinator; @@ -155,13 +187,17 @@ public class WorkerNode extends RegisteringNode public void init() throws Exception { + initializeHttpClient(); initializeEmitter(); + initializeS3Service(); initializeMonitors(); initializeMergerConfig(); + initializeCuratorFramework(); + initializeServiceDiscovery(); + initializeCoordinatorServiceProvider(); initializeTaskToolbox(); initializeJacksonInjections(); initializeJacksonSubtypes(); - initializeCuratorFramework(); initializeCuratorCoordinator(); initializeTaskMonitor(); initializeServer(); @@ -234,7 +270,7 @@ public class WorkerNode extends RegisteringNode { InjectableValues.Std injectables = new InjectableValues.Std(); - injectables.addValue("s3Client", taskToolbox.getS3Client()) + injectables.addValue("s3Client", s3Service) .addValue("segmentPusher", taskToolbox.getSegmentPusher()); jsonMapper.setInjectableValues(injectables); @@ -245,13 +281,18 @@ public class WorkerNode extends RegisteringNode jsonMapper.registerSubtypes(StaticS3FirehoseFactory.class); } + private void initializeHttpClient() + { + if (httpClient == null) { + httpClient = HttpClientInit.createClient( + HttpClientConfig.builder().withNumConnections(1).build(), lifecycle + ); + } + } + private void initializeEmitter() { if (emitter == null) { - final HttpClient httpClient = HttpClientInit.createClient( - HttpClientConfig.builder().withNumConnections(1).build(), lifecycle - ); - emitter = new ServiceEmitter( PropUtils.getProperty(props, "druid.service"), PropUtils.getProperty(props, "druid.host"), @@ -261,6 +302,18 @@ public class WorkerNode extends RegisteringNode EmittingLogger.registerEmitter(emitter); } + private void initializeS3Service() throws S3ServiceException + { + if(s3Service == null) { + s3Service = new RestS3Service( + new AWSCredentials( + PropUtils.getProperty(props, "com.metamx.aws.accessKey"), + PropUtils.getProperty(props, "com.metamx.aws.secretKey") + ) + ); + } + } + private void initializeMonitors() { if (monitors == null) { @@ -272,8 +325,8 @@ public class WorkerNode extends RegisteringNode private void initializeMergerConfig() { - if (coordinatorConfig == null) { - coordinatorConfig = configFactory.build(IndexerCoordinatorConfig.class); + if (taskConfig == null) { + taskConfig = configFactory.build(TaskConfig.class); } if (workerConfig == null) { @@ -284,28 +337,58 @@ public class WorkerNode extends RegisteringNode public void initializeTaskToolbox() throws S3ServiceException { if (taskToolbox == null) { - final RestS3Service s3Client = 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), + final DataSegmentPusher dataSegmentPusher = new S3DataSegmentPusher( + s3Service, + configFactory.build(S3DataSegmentPusherConfig.class), + jsonMapper + ); + final SegmentKiller segmentKiller = new S3SegmentKiller( + s3Service + ); + taskToolbox = new TaskToolbox( + taskConfig, + new RemoteTaskActionClient(httpClient, coordinatorServiceProvider, jsonMapper), + emitter, + s3Service, + dataSegmentPusher, + segmentKiller, jsonMapper ); - taskToolbox = new TaskToolbox(coordinatorConfig, emitter, s3Client, segmentPusher, jsonMapper); } } public void initializeCuratorFramework() throws IOException { - final CuratorConfig curatorConfig = configFactory.build(CuratorConfig.class); - curatorFramework = Initialization.makeCuratorFrameworkClient( - curatorConfig, - lifecycle - ); + if (curatorFramework == null) { + final CuratorConfig curatorConfig = configFactory.build(CuratorConfig.class); + curatorFramework = Initialization.makeCuratorFrameworkClient( + curatorConfig, + lifecycle + ); + } + } + + public void initializeServiceDiscovery() throws Exception + { + if (serviceDiscovery == null) { + final ServiceDiscoveryConfig config = configFactory.build(ServiceDiscoveryConfig.class); + this.serviceDiscovery = Initialization.makeServiceDiscoveryClient( + curatorFramework, + config, + lifecycle + ); + } + } + + public void initializeCoordinatorServiceProvider() + { + if (coordinatorServiceProvider == null) { + this.coordinatorServiceProvider = Initialization.makeServiceProvider( + workerConfig.getMasterService(), + serviceDiscovery, + lifecycle + ); + } } public void initializeCuratorCoordinator() diff --git a/merger/src/test/java/com/metamx/druid/merger/TestTask.java b/merger/src/test/java/com/metamx/druid/merger/TestTask.java new file mode 100644 index 00000000000..c23b498f739 --- /dev/null +++ b/merger/src/test/java/com/metamx/druid/merger/TestTask.java @@ -0,0 +1,81 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.merger; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.metamx.druid.aggregation.AggregatorFactory; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.merger.common.TaskStatus; +import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.task.DefaultMergeTask; + +import java.util.List; + +/** + */ +@JsonTypeName("test") +public class TestTask extends DefaultMergeTask +{ + private final String id; + private final TaskStatus status; + + @JsonCreator + public TestTask( + @JsonProperty("id") String id, + @JsonProperty("dataSource") String dataSource, + @JsonProperty("segments") List segments, + @JsonProperty("aggregations") List aggregators, + @JsonProperty("taskStatus") TaskStatus status + ) + { + super(dataSource, segments, aggregators); + + this.id = id; + this.status = status; + } + + @Override + @JsonProperty + public String getId() + { + return id; + } + + @Override + @JsonProperty + public String getType() + { + return "test"; + } + + @JsonProperty + public TaskStatus getStatus() + { + return status; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + return status; + } +} diff --git a/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java b/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java index fcecac91d5b..0f1a7a66964 100644 --- a/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskTest.java @@ -19,9 +19,10 @@ package com.metamx.druid.merger.common.task; +import com.google.common.base.Charsets; import com.google.common.collect.ImmutableList; +import com.google.common.hash.Hashing; import com.metamx.druid.client.DataSegment; -import org.apache.commons.codec.digest.DigestUtils; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; @@ -51,9 +52,9 @@ public class MergeTaskTest } @Override - public Type getType() + public String getType() { - return Type.TEST; + return "test"; } }; @@ -66,17 +67,18 @@ public class MergeTaskTest @Test public void testInterval() { - Assert.assertEquals(new Interval("2012-01-03/2012-01-07"), testMergeTask.getInterval()); + Assert.assertEquals(new Interval("2012-01-03/2012-01-07"), testMergeTask.getFixedInterval().get()); } @Test public void testID() { - final String desiredPrefix = "merge_foo_" + DigestUtils.shaHex( + final String desiredPrefix = "merge_foo_" + Hashing.sha1().hashString( "2012-01-03T00:00:00.000Z_2012-01-05T00:00:00.000Z_V1_0" + "_2012-01-04T00:00:00.000Z_2012-01-06T00:00:00.000Z_V1_0" + "_2012-01-05T00:00:00.000Z_2012-01-07T00:00:00.000Z_V1_0" - ) + "_"; + , Charsets.UTF_8 + ).toString().toLowerCase() + "_"; Assert.assertEquals( desiredPrefix, testMergeTask.getId().substring(0, desiredPrefix.length()) diff --git a/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java b/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java new file mode 100644 index 00000000000..51310bb2ef0 --- /dev/null +++ b/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java @@ -0,0 +1,111 @@ +package com.metamx.druid.merger.common.task; + +import com.google.common.collect.ImmutableList; +import com.metamx.common.Granularity; +import com.metamx.druid.QueryGranularity; +import com.metamx.druid.aggregation.AggregatorFactory; +import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.indexer.granularity.UniformGranularitySpec; +import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.realtime.Schema; +import com.metamx.druid.shard.NoneShardSpec; +import junit.framework.Assert; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.joda.time.Interval; +import org.junit.Test; + +public class TaskSerdeTest +{ + @Test + public void testIndexTaskSerde() throws Exception + { + final Task task = new IndexTask( + "foo", + new UniformGranularitySpec(Granularity.DAY, ImmutableList.of(new Interval("2010-01-01/P2D"))), + new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, + QueryGranularity.NONE, + 10000, + null, + -1 + ); + + final ObjectMapper jsonMapper = new DefaultObjectMapper(); + final String json = jsonMapper.writeValueAsString(task); + final Task task2 = jsonMapper.readValue(json, Task.class); + + Assert.assertEquals(task.getId(), task2.getId()); + Assert.assertEquals(task.getGroupId(), task2.getGroupId()); + Assert.assertEquals(task.getDataSource(), task2.getDataSource()); + Assert.assertEquals(task.getFixedInterval(), task2.getFixedInterval()); + Assert.assertEquals(task.getFixedInterval().get(), task2.getFixedInterval().get()); + } + + @Test + public void testIndexGeneratorTaskSerde() throws Exception + { + final Task task = new IndexGeneratorTask( + "foo", + new Interval("2010-01-01/P1D"), + null, + new Schema( + "foo", + new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, + QueryGranularity.NONE, + new NoneShardSpec() + ), + -1 + ); + + final ObjectMapper jsonMapper = new DefaultObjectMapper(); + final String json = jsonMapper.writeValueAsString(task); + final Task task2 = jsonMapper.readValue(json, Task.class); + + Assert.assertEquals(task.getId(), task2.getId()); + Assert.assertEquals(task.getGroupId(), task2.getGroupId()); + Assert.assertEquals(task.getDataSource(), task2.getDataSource()); + Assert.assertEquals(task.getFixedInterval(), task2.getFixedInterval()); + Assert.assertEquals(task.getFixedInterval().get(), task2.getFixedInterval().get()); + } + + @Test + public void testAppendTaskSerde() throws Exception + { + final Task task = new AppendTask( + "foo", + ImmutableList.of( + DataSegment.builder().dataSource("foo").interval(new Interval("2010-01-01/P1D")).version("1234").build() + ) + ); + + final ObjectMapper jsonMapper = new DefaultObjectMapper(); + final String json = jsonMapper.writeValueAsString(task); + final Task task2 = jsonMapper.readValue(json, Task.class); + + Assert.assertEquals(task.getId(), task2.getId()); + Assert.assertEquals(task.getGroupId(), task2.getGroupId()); + Assert.assertEquals(task.getDataSource(), task2.getDataSource()); + Assert.assertEquals(task.getFixedInterval(), task2.getFixedInterval()); + Assert.assertEquals(task.getFixedInterval().get(), task2.getFixedInterval().get()); + } + + @Test + public void testDeleteTaskSerde() throws Exception + { + final Task task = new DeleteTask( + "foo", + new Interval("2010-01-01/P1D") + ); + + final ObjectMapper jsonMapper = new DefaultObjectMapper(); + final String json = jsonMapper.writeValueAsString(task); + System.out.println(json); + final Task task2 = jsonMapper.readValue(json, Task.class); + + Assert.assertEquals(task.getId(), task2.getId()); + Assert.assertEquals(task.getGroupId(), task2.getGroupId()); + Assert.assertEquals(task.getDataSource(), task2.getDataSource()); + Assert.assertEquals(task.getFixedInterval(), task2.getFixedInterval()); + Assert.assertEquals(task.getFixedInterval().get(), task2.getFixedInterval().get()); + } +} diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java index ded6b55d64d..60309f1e368 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java @@ -1,27 +1,21 @@ package com.metamx.druid.merger.coordinator; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.base.Throwables; import com.google.common.collect.Lists; -import com.google.common.collect.Sets; import com.metamx.common.ISE; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.merger.TestTask; +import com.metamx.druid.merger.common.TaskCallback; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; import com.metamx.druid.merger.common.config.IndexerZkConfig; -import com.metamx.druid.merger.common.task.DefaultMergeTask; -import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; +import com.metamx.druid.merger.common.config.TaskConfig; import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.merger.coordinator.config.RetryPolicyConfig; -import com.metamx.druid.merger.coordinator.scaling.AutoScalingData; -import com.metamx.druid.merger.coordinator.scaling.ScalingStrategy; import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; import com.metamx.druid.merger.worker.TaskMonitor; @@ -34,10 +28,6 @@ import com.netflix.curator.retry.ExponentialBackoffRetry; import com.netflix.curator.test.TestingCluster; import org.apache.commons.lang.mutable.MutableBoolean; import org.apache.zookeeper.CreateMode; - - - - import org.easymock.EasyMock; import org.joda.time.DateTime; import org.joda.time.Duration; @@ -48,8 +38,6 @@ import org.junit.Before; import org.junit.Test; import java.io.File; -import java.util.List; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -117,10 +105,12 @@ public class RemoteTaskRunnerTest null, null, null, - null, + 0, 0 ) - ), Lists.newArrayList() + ), + Lists.newArrayList(), + TaskStatus.success("task1") ); makeRemoteTaskRunner(); @@ -140,36 +130,28 @@ public class RemoteTaskRunnerTest { remoteTaskRunner.run( task1, - new TaskContext(new DateTime().toString(), Sets.newHashSet()), null ); } @Test - public void testAlreadyExecutedTask() throws Exception + public void testExceptionThrownWithExistingTask() throws Exception { - final CountDownLatch latch = new CountDownLatch(1); remoteTaskRunner.run( - new TestTask(task1){ - @Override - public TaskStatus run( - TaskContext context, TaskToolbox toolbox - ) throws Exception - { - latch.await(); - return super.run(context, toolbox); - } - }, - new TaskContext(new DateTime().toString(), Sets.newHashSet()), + new TestTask( + task1.getId(), + task1.getDataSource(), + task1.getSegments(), + Lists.newArrayList(), + TaskStatus.running(task1.getId()) + ), null ); try { - remoteTaskRunner.run(task1, new TaskContext(new DateTime().toString(), Sets.newHashSet()), null); - latch.countDown(); + remoteTaskRunner.run(task1, null); fail("ISE expected"); } catch (ISE expected) { - latch.countDown(); } } @@ -191,12 +173,13 @@ public class RemoteTaskRunnerTest null, null, null, - null, + 0, 0 ) - ), Lists.newArrayList() + ), + Lists.newArrayList(), + TaskStatus.success("foo") ), - new TaskContext(new DateTime().toString(), Sets.newHashSet()), null ); } @@ -207,27 +190,17 @@ public class RemoteTaskRunnerTest } @Test - public void testRunWithExistingCompletedTask() throws Exception + public void testRunWithCallback() throws Exception { - cf.create().creatingParentsIfNeeded().forPath( - String.format("%s/worker1/task1", statusPath), - jsonMapper.writeValueAsBytes( - TaskStatus.success( - "task1", - Lists.newArrayList() - ) - ) - ); - - // Really don't like this way of waiting for the task to appear - while (remoteTaskRunner.getNumWorkers() == 0) { - Thread.sleep(500); - } - final MutableBoolean callbackCalled = new MutableBoolean(false); remoteTaskRunner.run( - task1, - null, + new TestTask( + task1.getId(), + task1.getDataSource(), + task1.getSegments(), + Lists.newArrayList(), + TaskStatus.running(task1.getId()) + ), new TaskCallback() { @Override @@ -238,6 +211,34 @@ public class RemoteTaskRunnerTest } ); + // Really don't like this way of waiting for the task to appear + int count = 0; + while (remoteTaskRunner.findWorkerRunningTask(task1.getId()) == null) { + Thread.sleep(500); + if (count > 10) { + throw new ISE("WTF?! Task still not announced in ZK?"); + } + count++; + } + + Assert.assertTrue(remoteTaskRunner.getRunningTasks().size() == 1); + + // Complete the task + cf.setData().forPath( + String.format("%s/worker1/task1", statusPath), + jsonMapper.writeValueAsBytes(TaskStatus.success(task1.getId())) + ); + + // Really don't like this way of waiting for the task to disappear + count = 0; + while (remoteTaskRunner.findWorkerRunningTask(task1.getId()) != null) { + Thread.sleep(500); + if (count > 10) { + throw new ISE("WTF?! Task still exists in ZK?"); + } + count++; + } + Assert.assertTrue("TaskCallback was not called!", callbackCalled.booleanValue()); } @@ -281,38 +282,8 @@ public class RemoteTaskRunnerTest cf, workerCuratorCoordinator, new TaskToolbox( - new IndexerCoordinatorConfig() + new TaskConfig() { - @Override - public String getServerName() - { - return "worker1"; - } - - @Override - public String getLeaderLatchPath() - { - return null; - } - - @Override - public int getNumLocalThreads() - { - return 1; - } - - @Override - public String getRunnerImpl() - { - return null; - } - - @Override - public String getStorageImpl() - { - return null; - } - @Override public File getBaseTaskDir() { @@ -325,30 +296,11 @@ public class RemoteTaskRunnerTest } @Override - public boolean isWhitelistEnabled() - { - return false; - } - - @Override - public String getWhitelistDatasourcesString() - { - return null; - } - - @Override - public long getRowFlushBoundary() + public int getDefaultRowFlushBoundary() { return 0; } - - - @Override - public String getStrategyImpl() - { - return null; - } - }, null, null, null, jsonMapper + }, null, null, null, null, null, jsonMapper ), Executors.newSingleThreadExecutor() ); @@ -368,7 +320,7 @@ public class RemoteTaskRunnerTest null, null ) - ); + ).atLeastOnce(); EasyMock.replay(workerSetupManager); remoteTaskRunner = new RemoteTaskRunner( @@ -378,7 +330,6 @@ public class RemoteTaskRunnerTest pathChildrenCache, scheduledExec, new RetryPolicyFactory(new TestRetryPolicyConfig()), - new TestScalingStrategy(), workerSetupManager ); @@ -388,8 +339,13 @@ public class RemoteTaskRunnerTest String.format("%s/worker1", announcementsPath), jsonMapper.writeValueAsBytes(worker1) ); - while (remoteTaskRunner.getNumWorkers() == 0) { - Thread.sleep(50); + int count = 0; + while (remoteTaskRunner.getWorkers().size() == 0) { + Thread.sleep(500); + if (count > 10) { + throw new ISE("WTF?! Still can't find worker!"); + } + count++; } } @@ -414,53 +370,8 @@ public class RemoteTaskRunnerTest } } - private static class TestScalingStrategy implements ScalingStrategy - { - @Override - public AutoScalingData provision() - { - return null; - } - - @Override - public AutoScalingData terminate(List nodeIds) - { - return null; - } - - @Override - public List ipLookup(List ips) - { - return ips; - } - } - private static class TestRemoteTaskRunnerConfig extends RemoteTaskRunnerConfig { - @Override - public Duration getTerminateResourcesDuration() - { - return null; - } - - @Override - public DateTime getTerminateResourcesOriginDateTime() - { - return null; - } - - @Override - public int getMaxWorkerIdleTimeMillisBeforeDeletion() - { - return 0; - } - - @Override - public Duration getMaxScalingDuration() - { - return null; - } - @Override public String getAnnouncementPath() { @@ -491,53 +402,4 @@ public class RemoteTaskRunnerTest return 1000; } } - - @JsonTypeName("test") - private static class TestTask extends DefaultMergeTask - { - private final String id; - private final String dataSource; - private final List segments; - private final List aggregators; - - @JsonCreator - public TestTask( - @JsonProperty("id") String id, - @JsonProperty("dataSource") String dataSource, - @JsonProperty("segments") List segments, - @JsonProperty("aggregations") List aggregators - ) - { - super(dataSource, segments, aggregators); - - this.id = id; - this.dataSource = dataSource; - this.segments = segments; - this.aggregators = aggregators; - } - - public TestTask(TestTask task) - { - this(task.id, task.dataSource, task.segments, task.aggregators); - } - - @Override - @JsonProperty - public String getId() - { - return id; - } - - @Override - public Type getType() - { - return Type.TEST; - } - - @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception - { - return TaskStatus.success("task1", Lists.newArrayList()); - } - } } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java new file mode 100644 index 00000000000..69a364e1900 --- /dev/null +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java @@ -0,0 +1,648 @@ +package com.metamx.druid.merger.coordinator; + +import com.google.common.base.Optional; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.Ordering; +import com.google.common.collect.Sets; +import com.google.common.io.Files; +import com.metamx.common.Granularity; +import com.metamx.common.ISE; +import com.metamx.common.guava.Comparators; +import com.metamx.druid.QueryGranularity; +import com.metamx.druid.aggregation.AggregatorFactory; +import com.metamx.druid.aggregation.DoubleSumAggregatorFactory; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.indexer.granularity.UniformGranularitySpec; +import com.metamx.druid.input.InputRow; +import com.metamx.druid.input.MapBasedInputRow; +import com.metamx.druid.jackson.DefaultObjectMapper; +import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.loading.SegmentKiller; +import com.metamx.druid.loading.SegmentLoadingException; +import com.metamx.druid.merger.common.TaskLock; +import com.metamx.druid.merger.common.TaskStatus; +import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.actions.LocalTaskActionClient; +import com.metamx.druid.merger.common.actions.LockAcquireAction; +import com.metamx.druid.merger.common.actions.LockListAction; +import com.metamx.druid.merger.common.actions.LockReleaseAction; +import com.metamx.druid.merger.common.actions.SegmentInsertAction; +import com.metamx.druid.merger.common.actions.TaskActionToolbox; +import com.metamx.druid.merger.common.config.TaskConfig; +import com.metamx.druid.merger.common.task.AbstractTask; +import com.metamx.druid.merger.common.task.IndexTask; +import com.metamx.druid.merger.common.task.KillTask; +import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.merger.coordinator.exec.TaskConsumer; +import com.metamx.druid.realtime.Firehose; +import com.metamx.druid.realtime.FirehoseFactory; +import com.metamx.emitter.EmittingLogger; +import com.metamx.emitter.core.Event; +import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.emitter.service.ServiceEventBuilder; +import org.apache.commons.io.FileUtils; +import org.easymock.EasyMock; +import org.jets3t.service.ServiceException; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Executors; + +public class TaskLifecycleTest +{ + private File tmp = null; + private TaskStorage ts = null; + private TaskLockbox tl = null; + private TaskQueue tq = null; + private TaskRunner tr = null; + private MockMergerDBCoordinator mdc = null; + private TaskToolbox tb = null; + private TaskConsumer tc = null; + TaskStorageQueryAdapter tsqa = null; + + private static final Ordering byIntervalOrdering = new Ordering() + { + @Override + public int compare(DataSegment dataSegment, DataSegment dataSegment2) + { + return Comparators.intervalsByStartThenEnd().compare(dataSegment.getInterval(), dataSegment2.getInterval()); + } + }; + + @Before + public void setUp() + { + EmittingLogger.registerEmitter(EasyMock.createMock(ServiceEmitter.class)); + + tmp = Files.createTempDir(); + + ts = new LocalTaskStorage(); + tl = new TaskLockbox(ts); + tq = new TaskQueue(ts, tl); + mdc = newMockMDC(); + + tb = new TaskToolbox( + new TaskConfig() + { + @Override + public File getBaseTaskDir() + { + return tmp; + } + + @Override + public int getDefaultRowFlushBoundary() + { + return 50000; + } + }, + new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, mdc, newMockEmitter())), + newMockEmitter(), + null, // s3 client + new DataSegmentPusher() + { + @Override + public DataSegment push(File file, DataSegment segment) throws IOException + { + return segment; + } + }, + new SegmentKiller() + { + @Override + public void kill(Collection segments) throws ServiceException + { + + } + }, + new DefaultObjectMapper() + ) + { + @Override + public Map getSegments( + Task task, List segments + ) throws SegmentLoadingException + { + return ImmutableMap.of(); + } + }; + + tr = new LocalTaskRunner( + tb, + Executors.newSingleThreadExecutor() + ); + + tc = new TaskConsumer(tq, tr, tb, newMockEmitter()); + tsqa = new TaskStorageQueryAdapter(ts); + + tq.start(); + tc.start(); + } + + @After + public void tearDown() + { + try { + FileUtils.deleteDirectory(tmp); + } catch(Exception e) { + // suppress + } + tc.stop(); + tq.stop(); + } + + @Test + public void testIndexTask() throws Exception + { + final Task indexTask = new IndexTask( + "foo", + new UniformGranularitySpec(Granularity.DAY, ImmutableList.of(new Interval("2010-01-01/P2D"))), + new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, + QueryGranularity.NONE, + 10000, + newMockFirehoseFactory( + ImmutableList.of( + IR("2010-01-01T01", "x", "y", 1), + IR("2010-01-01T01", "x", "z", 1), + IR("2010-01-02T01", "a", "b", 2), + IR("2010-01-02T01", "a", "c", 1) + ) + ), + -1 + ); + + final TaskStatus mergedStatus = runTask(indexTask); + final TaskStatus status = ts.getStatus(indexTask.getId()).get(); + final List publishedSegments = byIntervalOrdering.sortedCopy(mdc.getPublished()); + final List loggedSegments = byIntervalOrdering.sortedCopy(tsqa.getSameGroupNewSegments(indexTask.getId())); + + Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode()); + Assert.assertEquals("merged statusCode", TaskStatus.Status.SUCCESS, mergedStatus.getStatusCode()); + Assert.assertEquals("segments logged vs published", loggedSegments, publishedSegments); + Assert.assertEquals("num segments published", 2, mdc.getPublished().size()); + Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size()); + + Assert.assertEquals("segment1 datasource", "foo", publishedSegments.get(0).getDataSource()); + Assert.assertEquals("segment1 interval", new Interval("2010-01-01/P1D"), publishedSegments.get(0).getInterval()); + Assert.assertEquals("segment1 dimensions", ImmutableList.of("dim1", "dim2"), publishedSegments.get(0).getDimensions()); + Assert.assertEquals("segment1 metrics", ImmutableList.of("met"), publishedSegments.get(0).getMetrics()); + + Assert.assertEquals("segment2 datasource", "foo", publishedSegments.get(1).getDataSource()); + Assert.assertEquals("segment2 interval", new Interval("2010-01-02/P1D"), publishedSegments.get(1).getInterval()); + Assert.assertEquals("segment2 dimensions", ImmutableList.of("dim1", "dim2"), publishedSegments.get(1).getDimensions()); + Assert.assertEquals("segment2 metrics", ImmutableList.of("met"), publishedSegments.get(1).getMetrics()); + } + + @Test + public void testIndexTaskFailure() throws Exception + { + final Task indexTask = new IndexTask( + "foo", + new UniformGranularitySpec(Granularity.DAY, ImmutableList.of(new Interval("2010-01-01/P1D"))), + new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, + QueryGranularity.NONE, + 10000, + newMockExceptionalFirehoseFactory(), + -1 + ); + + final TaskStatus mergedStatus = runTask(indexTask); + final TaskStatus status = ts.getStatus(indexTask.getId()).get(); + + Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode()); + Assert.assertEquals("merged statusCode", TaskStatus.Status.FAILED, mergedStatus.getStatusCode()); + Assert.assertEquals("num segments published", 0, mdc.getPublished().size()); + Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size()); + } + + @Test + public void testKillTask() throws Exception + { + // TODO: Worst test ever + final Task killTask = new KillTask("foo", new Interval("2010-01-02/P2D")); + + final TaskStatus mergedStatus = runTask(killTask); + Assert.assertEquals("merged statusCode", TaskStatus.Status.SUCCESS, mergedStatus.getStatusCode()); + Assert.assertEquals("num segments published", 0, mdc.getPublished().size()); + Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size()); + } + + @Test + public void testRealtimeishTask() throws Exception + { + class RealtimeishTask extends AbstractTask + { + RealtimeishTask() + { + super("rt1", "rt", "foo", null); + } + + @Override + public String getType() + { + return "realtime_test"; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + final Interval interval1 = new Interval("2010-01-01T00/PT1H"); + final Interval interval2 = new Interval("2010-01-01T01/PT1H"); + + // Sort of similar to what realtime tasks do: + + // Acquire lock for first interval + final Optional lock1 = toolbox.getTaskActionClient().submit(new LockAcquireAction(this, interval1)); + final List locks1 = toolbox.getTaskActionClient().submit(new LockListAction(this)); + + // (Confirm lock sanity) + Assert.assertTrue("lock1 present", lock1.isPresent()); + Assert.assertEquals("lock1 interval", interval1, lock1.get().getInterval()); + Assert.assertEquals("locks1", ImmutableList.of(lock1.get()), locks1); + + // Acquire lock for second interval + final Optional lock2 = toolbox.getTaskActionClient().submit(new LockAcquireAction(this, interval2)); + final List locks2 = toolbox.getTaskActionClient().submit(new LockListAction(this)); + + // (Confirm lock sanity) + Assert.assertTrue("lock2 present", lock2.isPresent()); + Assert.assertEquals("lock2 interval", interval2, lock2.get().getInterval()); + Assert.assertEquals("locks2", ImmutableList.of(lock1.get(), lock2.get()), locks2); + + // Push first segment + toolbox.getTaskActionClient() + .submit( + new SegmentInsertAction( + this, + ImmutableSet.of( + DataSegment.builder() + .dataSource("foo") + .interval(interval1) + .version(lock1.get().getVersion()) + .build() + ) + ) + ); + + // Release first lock + toolbox.getTaskActionClient().submit(new LockReleaseAction(this, interval1)); + final List locks3 = toolbox.getTaskActionClient().submit(new LockListAction(this)); + + // (Confirm lock sanity) + Assert.assertEquals("locks3", ImmutableList.of(lock2.get()), locks3); + + // Push second segment + toolbox.getTaskActionClient() + .submit( + new SegmentInsertAction( + this, + ImmutableSet.of( + DataSegment.builder() + .dataSource("foo") + .interval(interval2) + .version(lock2.get().getVersion()) + .build() + ) + ) + ); + + // Release second lock + toolbox.getTaskActionClient().submit(new LockReleaseAction(this, interval2)); + final List locks4 = toolbox.getTaskActionClient().submit(new LockListAction(this)); + + // (Confirm lock sanity) + Assert.assertEquals("locks4", ImmutableList.of(), locks4); + + // Exit + return TaskStatus.success(getId()); + } + } + + final Task rtishTask = new RealtimeishTask(); + final TaskStatus status = runTask(rtishTask); + + Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode()); + Assert.assertEquals("num segments published", 2, mdc.getPublished().size()); + Assert.assertEquals("num segments nuked", 0, mdc.getNuked().size()); + } + + @Test + public void testSimple() throws Exception + { + final Task task = new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D")) + { + @Override + public String getType() + { + return "test"; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + final TaskLock myLock = Iterables.getOnlyElement( + toolbox.getTaskActionClient() + .submit(new LockListAction(this)) + ); + + final DataSegment segment = DataSegment.builder() + .dataSource("ds") + .interval(new Interval("2012-01-01/P1D")) + .version(myLock.getVersion()) + .build(); + + toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment))); + return TaskStatus.success(getId()); + } + }; + + final TaskStatus status = runTask(task); + + Assert.assertEquals("statusCode", TaskStatus.Status.SUCCESS, status.getStatusCode()); + Assert.assertEquals("segments published", 1, mdc.getPublished().size()); + Assert.assertEquals("segments nuked", 0, mdc.getNuked().size()); + } + + @Test + public void testBadVersion() throws Exception + { + final Task task = new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D")) + { + @Override + public String getType() + { + return "test"; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + final TaskLock myLock = Iterables.getOnlyElement( + toolbox.getTaskActionClient() + .submit(new LockListAction(this)) + ); + + final DataSegment segment = DataSegment.builder() + .dataSource("ds") + .interval(new Interval("2012-01-01/P2D")) + .version(myLock.getVersion()) + .build(); + + toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment))); + return TaskStatus.success(getId()); + } + }; + + final TaskStatus status = runTask(task); + + Assert.assertEquals("statusCode", TaskStatus.Status.FAILED, status.getStatusCode()); + Assert.assertEquals("segments published", 0, mdc.getPublished().size()); + Assert.assertEquals("segments nuked", 0, mdc.getNuked().size()); + } + + @Test + public void testBadInterval() throws Exception + { + final Task task = new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D")) + { + @Override + public String getType() + { + return "test"; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + final TaskLock myLock = Iterables.getOnlyElement( + toolbox.getTaskActionClient() + .submit(new LockListAction(this)) + ); + + final DataSegment segment = DataSegment.builder() + .dataSource("ds") + .interval(new Interval("2012-01-01/P1D")) + .version(myLock.getVersion() + "1!!!1!!") + .build(); + + toolbox.getTaskActionClient().submit(new SegmentInsertAction(this, ImmutableSet.of(segment))); + return TaskStatus.success(getId()); + } + }; + + final TaskStatus status = runTask(task); + + Assert.assertEquals("statusCode", TaskStatus.Status.FAILED, status.getStatusCode()); + Assert.assertEquals("segments published", 0, mdc.getPublished().size()); + Assert.assertEquals("segments nuked", 0, mdc.getNuked().size()); + } + + private TaskStatus runTask(Task task) + { + final long startTime = System.currentTimeMillis(); + + tq.add(task); + + TaskStatus status; + + try { + while ( (status = tsqa.getSameGroupMergedStatus(task.getId()).get()).isRunnable()) { + if(System.currentTimeMillis() > startTime + 10 * 1000) { + throw new ISE("Where did the task go?!: %s", task.getId()); + } + + Thread.sleep(100); + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } + + return status; + } + + private static class MockMergerDBCoordinator extends MergerDBCoordinator + { + final private Set published = Sets.newHashSet(); + final private Set nuked = Sets.newHashSet(); + + private MockMergerDBCoordinator() + { + super(null, null, null); + } + + @Override + public List getUsedSegmentsForInterval(String dataSource, Interval interval) throws IOException + { + return ImmutableList.of(); + } + + @Override + public List getUnusedSegmentsForInterval(String dataSource, Interval interval) + { + return ImmutableList.of(); + } + + @Override + public void announceHistoricalSegments(Set segment) + { + published.addAll(segment); + } + + @Override + public void deleteSegments(Set segment) + { + nuked.addAll(segment); + } + + public Set getPublished() + { + return ImmutableSet.copyOf(published); + } + + public Set getNuked() + { + return ImmutableSet.copyOf(nuked); + } + } + + private static MockMergerDBCoordinator newMockMDC() + { + return new MockMergerDBCoordinator(); + } + + private static ServiceEmitter newMockEmitter() + { + return new ServiceEmitter(null, null, null) + { + @Override + public void emit(Event event) + { + + } + + @Override + public void emit(ServiceEventBuilder builder) + { + + } + }; + } + + private static InputRow IR(String dt, String dim1, String dim2, float met) + { + return new MapBasedInputRow( + new DateTime(dt).getMillis(), + ImmutableList.of("dim1", "dim2"), + ImmutableMap.of( + "dim1", dim1, + "dim2", dim2, + "met", met + ) + ); + } + + private static FirehoseFactory newMockExceptionalFirehoseFactory() + { + return new FirehoseFactory() + { + @Override + public Firehose connect() throws IOException + { + return new Firehose() + { + @Override + public boolean hasMore() + { + return true; + } + + @Override + public InputRow nextRow() + { + throw new RuntimeException("HA HA HA"); + } + + @Override + public Runnable commit() + { + return new Runnable() { + @Override + public void run() + { + + } + }; + } + + @Override + public void close() throws IOException + { + + } + }; + } + }; + } + + private static FirehoseFactory newMockFirehoseFactory(final Iterable inputRows) + { + return new FirehoseFactory() + { + @Override + public Firehose connect() throws IOException + { + final Iterator inputRowIterator = inputRows.iterator(); + + return new Firehose() + { + @Override + public boolean hasMore() + { + return inputRowIterator.hasNext(); + } + + @Override + public InputRow nextRow() + { + return inputRowIterator.next(); + } + + @Override + public Runnable commit() + { + return new Runnable() + { + @Override + public void run() + { + + } + }; + } + + @Override + public void close() throws IOException + { + + } + }; + } + }; + } +} diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java index 1a88f49885e..0d003551ea9 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java @@ -20,14 +20,17 @@ package com.metamx.druid.merger.coordinator; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import com.metamx.druid.client.DataSegment; +import com.metamx.druid.merger.common.TaskLock; import com.metamx.druid.merger.common.TaskStatus; import com.metamx.druid.merger.common.TaskToolbox; +import com.metamx.druid.merger.common.actions.LocalTaskActionClient; +import com.metamx.druid.merger.common.actions.SpawnTasksAction; +import com.metamx.druid.merger.common.actions.TaskActionToolbox; import com.metamx.druid.merger.common.task.AbstractTask; import com.metamx.druid.merger.common.task.Task; -import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; @@ -40,23 +43,21 @@ public class TaskQueueTest @Test public void testEmptyQueue() throws Exception { - final TaskQueue tq = newTaskQueue(); + final TaskStorage ts = new LocalTaskStorage(); + final TaskLockbox tl = new TaskLockbox(ts); + final TaskQueue tq = newTaskQueue(ts, tl); // get task status for nonexistent task - Assert.assertFalse("getStatus", tq.getStatus("foo").isPresent()); + Assert.assertFalse("getStatus", ts.getStatus("foo").isPresent()); // poll on empty queue Assert.assertNull("poll", tq.poll()); } - private static TaskQueue newTaskQueue() + public static TaskQueue newTaskQueue(TaskStorage storage, TaskLockbox lockbox) { - return newTaskQueueWithStorage(new LocalTaskStorage()); - } - - public static TaskQueue newTaskQueueWithStorage(TaskStorage storage) - { - final TaskQueue tq = new TaskQueue(storage); + final TaskQueue tq = new TaskQueue(storage, lockbox); + tq.bootstrap(); tq.start(); return tq; } @@ -64,7 +65,10 @@ public class TaskQueueTest @Test public void testAddRemove() throws Exception { - final TaskQueue tq = newTaskQueue(); + final TaskStorage ts = new LocalTaskStorage(); + final TaskLockbox tl = new TaskLockbox(ts); + final TaskQueue tq = newTaskQueue(ts, tl); + final Task[] tasks = { newTask("T0", "G0", "bar", new Interval("2011/P1Y")), newTask("T1", "G1", "bar", new Interval("2011-03-01/P1D")), @@ -81,13 +85,13 @@ public class TaskQueueTest } // get task status for in-progress task - Assert.assertEquals("T2 status (before finishing)", TaskStatus.Status.RUNNING, tq.getStatus(tasks[2].getId()).get().getStatusCode()); + Assert.assertEquals("T2 status (before finishing)", TaskStatus.Status.RUNNING, ts.getStatus(tasks[2].getId()).get().getStatusCode()); // Can't add tasks with the same id thrown = null; try { tq.add(newTask("T5", "G5", "baz", new Interval("2013-02-01/PT1H"))); - } catch(IllegalStateException e) { + } catch(TaskExistsException e) { thrown = e; } @@ -96,9 +100,9 @@ public class TaskQueueTest // take max number of tasks final List taken = Lists.newArrayList(); while (true) { - final VersionedTaskWrapper taskWrapper = tq.poll(); - if(taskWrapper != null) { - taken.add(taskWrapper.getTask()); + final Task task = tq.poll(); + if(task != null) { + taken.add(task); } else { break; } @@ -114,27 +118,21 @@ public class TaskQueueTest ); // mark one done - tq.done(tasks[2], tasks[2].run(null, null)); + tq.notify(tasks[2], tasks[2].run(null)); // get its status back - Assert.assertEquals("T2 status (after finishing)", TaskStatus.Status.SUCCESS, tq.getStatus(tasks[2].getId()).get().getStatusCode()); + Assert.assertEquals( + "T2 status (after finishing)", + TaskStatus.Status.SUCCESS, + ts.getStatus(tasks[2].getId()).get().getStatusCode() + ); - // Can't do a task twice - thrown = null; - try { - tq.done(tasks[2], tasks[2].run(null, null)); - } catch(IllegalStateException e) { - thrown = e; - } - - Assert.assertNotNull("Exception on twice-done task", thrown); - - // we should be able to get one more task now + // We should be able to get one more task now taken.clear(); while (true) { - final VersionedTaskWrapper taskWrapper = tq.poll(); - if(taskWrapper != null) { - taken.add(taskWrapper.getTask()); + final Task task = tq.poll(); + if(task != null) { + taken.add(task); } else { break; } @@ -156,44 +154,55 @@ public class TaskQueueTest @Test public void testContinues() throws Exception { - final TaskQueue tq = newTaskQueue(); + final TaskStorage ts = new LocalTaskStorage(); + final TaskLockbox tl = new TaskLockbox(ts); + final TaskQueue tq = newTaskQueue(ts, tl); + final TaskToolbox tb = new TaskToolbox( + null, + new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, null, null)), + null, + null, + null, + null, + null + ); final Task t0 = newTask("T0", "G0", "bar", new Interval("2011/P1Y")); final Task t1 = newContinuedTask("T1", "G1", "bar", new Interval("2013/P1Y"), Lists.newArrayList(t0)); tq.add(t1); - Assert.assertTrue("T0 isPresent (#1)", !tq.getStatus("T0").isPresent()); - Assert.assertTrue("T1 isPresent (#1)", tq.getStatus("T1").isPresent()); - Assert.assertTrue("T1 isRunnable (#1)", tq.getStatus("T1").get().isRunnable()); - Assert.assertTrue("T1 isComplete (#1)", !tq.getStatus("T1").get().isComplete()); + Assert.assertTrue("T0 isPresent (#1)", !ts.getStatus("T0").isPresent()); + Assert.assertTrue("T1 isPresent (#1)", ts.getStatus("T1").isPresent()); + Assert.assertTrue("T1 isRunnable (#1)", ts.getStatus("T1").get().isRunnable()); + Assert.assertTrue("T1 isComplete (#1)", !ts.getStatus("T1").get().isComplete()); // should be able to get t1 out - Assert.assertEquals("poll #1", "T1", tq.poll().getTask().getId()); + Assert.assertEquals("poll #1", "T1", tq.poll().getId()); Assert.assertNull("poll #2", tq.poll()); // report T1 done. Should cause T0 to be created - tq.done(t1, t1.run(null, null)); + tq.notify(t1, t1.run(tb)); - Assert.assertTrue("T0 isPresent (#2)", tq.getStatus("T0").isPresent()); - Assert.assertTrue("T0 isRunnable (#2)", tq.getStatus("T0").get().isRunnable()); - Assert.assertTrue("T0 isComplete (#2)", !tq.getStatus("T0").get().isComplete()); - Assert.assertTrue("T1 isPresent (#2)", tq.getStatus("T1").isPresent()); - Assert.assertTrue("T1 isRunnable (#2)", !tq.getStatus("T1").get().isRunnable()); - Assert.assertTrue("T1 isComplete (#2)", tq.getStatus("T1").get().isComplete()); + Assert.assertTrue("T0 isPresent (#2)", ts.getStatus("T0").isPresent()); + Assert.assertTrue("T0 isRunnable (#2)", ts.getStatus("T0").get().isRunnable()); + Assert.assertTrue("T0 isComplete (#2)", !ts.getStatus("T0").get().isComplete()); + Assert.assertTrue("T1 isPresent (#2)", ts.getStatus("T1").isPresent()); + Assert.assertTrue("T1 isRunnable (#2)", !ts.getStatus("T1").get().isRunnable()); + Assert.assertTrue("T1 isComplete (#2)", ts.getStatus("T1").get().isComplete()); // should be able to get t0 out - Assert.assertEquals("poll #3", "T0", tq.poll().getTask().getId()); + Assert.assertEquals("poll #3", "T0", tq.poll().getId()); Assert.assertNull("poll #4", tq.poll()); // report T0 done. Should cause T0, T1 to be marked complete - tq.done(t0, t0.run(null, null)); + tq.notify(t0, t0.run(tb)); - Assert.assertTrue("T0 isPresent (#3)", tq.getStatus("T0").isPresent()); - Assert.assertTrue("T0 isRunnable (#3)", !tq.getStatus("T0").get().isRunnable()); - Assert.assertTrue("T0 isComplete (#3)", tq.getStatus("T0").get().isComplete()); - Assert.assertTrue("T1 isPresent (#3)", tq.getStatus("T1").isPresent()); - Assert.assertTrue("T1 isRunnable (#3)", !tq.getStatus("T1").get().isRunnable()); - Assert.assertTrue("T1 isComplete (#3)", tq.getStatus("T1").get().isComplete()); + Assert.assertTrue("T0 isPresent (#3)", ts.getStatus("T0").isPresent()); + Assert.assertTrue("T0 isRunnable (#3)", !ts.getStatus("T0").get().isRunnable()); + Assert.assertTrue("T0 isComplete (#3)", ts.getStatus("T0").get().isComplete()); + Assert.assertTrue("T1 isPresent (#3)", ts.getStatus("T1").isPresent()); + Assert.assertTrue("T1 isRunnable (#3)", !ts.getStatus("T1").get().isRunnable()); + Assert.assertTrue("T1 isComplete (#3)", ts.getStatus("T1").get().isComplete()); // should be no more events available for polling Assert.assertNull("poll #5", tq.poll()); @@ -202,7 +211,18 @@ public class TaskQueueTest @Test public void testConcurrency() throws Exception { - final TaskQueue tq = newTaskQueue(); + final TaskStorage ts = new LocalTaskStorage(); + final TaskLockbox tl = new TaskLockbox(ts); + final TaskQueue tq = newTaskQueue(ts, tl); + final TaskToolbox tb = new TaskToolbox( + null, + new LocalTaskActionClient(ts, new TaskActionToolbox(tq, tl, null, null)), + null, + null, + null, + null, + null + ); // Imagine a larger task that splits itself up into pieces final Task t1 = newTask("T1", "G0", "bar", new Interval("2011-01-01/P1D")); @@ -219,15 +239,16 @@ public class TaskQueueTest tq.add(t0); - final VersionedTaskWrapper wt0 = tq.poll(); - Assert.assertEquals("wt0 task id", "T0", wt0.getTask().getId()); + final Task wt0 = tq.poll(); + final TaskLock wt0Lock = Iterables.getOnlyElement(tl.findLocksForTask(wt0)); + Assert.assertEquals("wt0 task id", "T0", wt0.getId()); Assert.assertNull("null poll #1", tq.poll()); // Sleep a bit to avoid false test passes Thread.sleep(5); // Finish t0 - tq.done(t0, t0.run(null, null)); + tq.notify(t0, t0.run(tb)); // take max number of tasks final Set taken = Sets.newHashSet(); @@ -236,15 +257,16 @@ public class TaskQueueTest // Sleep a bit to avoid false test passes Thread.sleep(5); - final VersionedTaskWrapper taskWrapper = tq.poll(); + final Task task = tq.poll(); - if(taskWrapper != null) { + if(task != null) { + final TaskLock taskLock = Iterables.getOnlyElement(tl.findLocksForTask(task)); Assert.assertEquals( - String.format("%s version", taskWrapper.getTask().getId()), - wt0.getVersion(), - taskWrapper.getVersion() + String.format("%s version", task.getId()), + wt0Lock.getVersion(), + taskLock.getVersion() ); - taken.add(taskWrapper.getTask().getId()); + taken.add(task.getId()); } else { break; } @@ -254,34 +276,36 @@ public class TaskQueueTest Assert.assertEquals("taken", Sets.newHashSet("T1", "T3"), taken); // Finish t1 - tq.done(t1, t1.run(null, null)); + tq.notify(t1, t1.run(null)); Assert.assertNull("null poll #2", tq.poll()); // Finish t3 - tq.done(t3, t3.run(null, null)); + tq.notify(t3, t3.run(tb)); // We should be able to get t2 now - final VersionedTaskWrapper wt2 = tq.poll(); - Assert.assertEquals("wt2 task id", "T2", wt2.getTask().getId()); - Assert.assertEquals("wt2 group id", "G1", wt2.getTask().getGroupId()); - Assert.assertNotSame("wt2 version", wt0.getVersion(), wt2.getVersion()); + final Task wt2 = tq.poll(); + final TaskLock wt2Lock = Iterables.getOnlyElement(tl.findLocksForTask(wt2)); + Assert.assertEquals("wt2 task id", "T2", wt2.getId()); + Assert.assertEquals("wt2 group id", "G1", wt2.getGroupId()); + Assert.assertNotSame("wt2 version", wt0Lock.getVersion(), wt2Lock.getVersion()); Assert.assertNull("null poll #3", tq.poll()); // Finish t2 - tq.done(t2, t2.run(null, null)); + tq.notify(t2, t2.run(tb)); // We should be able to get t4 // And it should be in group G0, but that group should have a different version than last time // (Since the previous transaction named "G0" has ended and transaction names are not necessarily tied to // one version if they end and are re-started) - final VersionedTaskWrapper wt4 = tq.poll(); - Assert.assertEquals("wt4 task id", "T4", wt4.getTask().getId()); - Assert.assertEquals("wt4 group id", "G0", wt4.getTask().getGroupId()); - Assert.assertNotSame("wt4 version", wt0.getVersion(), wt4.getVersion()); - Assert.assertNotSame("wt4 version", wt2.getVersion(), wt4.getVersion()); + final Task wt4 = tq.poll(); + final TaskLock wt4Lock = Iterables.getOnlyElement(tl.findLocksForTask(wt4)); + Assert.assertEquals("wt4 task id", "T4", wt4.getId()); + Assert.assertEquals("wt4 group id", "G0", wt4.getGroupId()); + Assert.assertNotSame("wt4 version", wt0Lock.getVersion(), wt4Lock.getVersion()); + Assert.assertNotSame("wt4 version", wt2Lock.getVersion(), wt4Lock.getVersion()); // Kind of done testing at this point, but let's finish t4 anyway - tq.done(t4, t4.run(null, null)); + tq.notify(t4, t4.run(tb)); Assert.assertNull("null poll #4", tq.poll()); } @@ -289,21 +313,25 @@ public class TaskQueueTest public void testBootstrap() throws Exception { final TaskStorage storage = new LocalTaskStorage(); + final TaskLockbox lockbox = new TaskLockbox(storage); + storage.insert(newTask("T1", "G1", "bar", new Interval("2011-01-01/P1D")), TaskStatus.running("T1")); storage.insert(newTask("T2", "G2", "bar", new Interval("2011-02-01/P1D")), TaskStatus.running("T2")); - storage.setVersion("T1", "1234"); + storage.addLock("T1", new TaskLock("G1", "bar", new Interval("2011-01-01/P1D"), "1234")); - final TaskQueue tq = newTaskQueueWithStorage(storage); + final TaskQueue tq = newTaskQueue(storage, lockbox); - final VersionedTaskWrapper vt1 = tq.poll(); - Assert.assertEquals("vt1 id", "T1", vt1.getTask().getId()); - Assert.assertEquals("vt1 version", "1234", vt1.getVersion()); + final Task vt1 = tq.poll(); + final TaskLock vt1Lock = Iterables.getOnlyElement(lockbox.findLocksForTask(vt1)); + Assert.assertEquals("vt1 id", "T1", vt1.getId()); + Assert.assertEquals("vt1 version", "1234", vt1Lock.getVersion()); - tq.done(vt1.getTask(), TaskStatus.success("T1", ImmutableList.of())); + tq.notify(vt1, TaskStatus.success("T1")); // re-bootstrap tq.stop(); - storage.setStatus("T2", TaskStatus.failure("T2")); + storage.setStatus(TaskStatus.failure("T2")); + tq.bootstrap(); tq.start(); Assert.assertNull("null poll", tq.poll()); @@ -314,30 +342,15 @@ public class TaskQueueTest return new AbstractTask(id, groupId, dataSource, interval) { @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception + public TaskStatus run(TaskToolbox toolbox) throws Exception { - return TaskStatus.success( - id, - Lists.newArrayList( - new DataSegment( - dataSource, - interval, - new DateTime("2012-01-02").toString(), - null, - null, - null, - null, - null, - -1 - ) - ) - ); + return TaskStatus.success(id); } @Override - public Type getType() + public String getType() { - return Type.TEST; + return "null"; } }; } @@ -353,15 +366,16 @@ public class TaskQueueTest return new AbstractTask(id, groupId, dataSource, interval) { @Override - public Type getType() + public String getType() { - return Type.TEST; + return "null"; } @Override - public TaskStatus run(TaskContext context, TaskToolbox toolbox) throws Exception + public TaskStatus run(TaskToolbox toolbox) throws Exception { - return TaskStatus.continued(id, nextTasks); + toolbox.getTaskActionClient().submit(new SpawnTasksAction(this, nextTasks)); + return TaskStatus.success(id); } }; } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java new file mode 100644 index 00000000000..36b6bd41632 --- /dev/null +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java @@ -0,0 +1,288 @@ +/* + * 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.collect.Lists; +import com.google.common.collect.Sets; +import com.metamx.druid.aggregation.AggregatorFactory; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.merger.TestTask; +import com.metamx.druid.merger.common.TaskStatus; +import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem; +import com.metamx.druid.merger.coordinator.ZkWorker; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; +import com.metamx.druid.merger.coordinator.setup.WorkerSetupManager; +import com.metamx.druid.merger.worker.Worker; +import junit.framework.Assert; +import org.easymock.EasyMock; +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.joda.time.Interval; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.List; +import java.util.Set; + +/** + */ +public class SimpleResourceManagementStrategyTest +{ + private AutoScalingStrategy autoScalingStrategy; + private WorkerSetupManager workerSetupManager; + private Task testTask; + private SimpleResourceManagementStrategy simpleResourceManagementStrategy; + + @Before + public void setUp() throws Exception + { + workerSetupManager = EasyMock.createMock(WorkerSetupManager.class); + autoScalingStrategy = EasyMock.createMock(AutoScalingStrategy.class); + + testTask = new TestTask( + "task1", + "dummyDs", + Lists.newArrayList( + new DataSegment( + "dummyDs", + new Interval("2012-01-01/2012-01-02"), + new DateTime().toString(), + null, + null, + null, + null, + 0, + 0 + ) + ), + Lists.newArrayList(), + TaskStatus.success("task1") + ); + simpleResourceManagementStrategy = new SimpleResourceManagementStrategy( + autoScalingStrategy, + new SimpleResourceManagmentConfig() + { + @Override + public int getMaxWorkerIdleTimeMillisBeforeDeletion() + { + return 0; + } + + @Override + public Duration getMaxScalingDuration() + { + return null; + } + + @Override + public int getNumEventsToTrack() + { + return 1; + } + + @Override + public Duration getMaxPendingTaskDuration() + { + return new Duration(0); + } + }, + workerSetupManager + ); + } + + @Test + public void testSuccessfulProvision() throws Exception + { + EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) + .andReturn(Lists.newArrayList()); + EasyMock.expect(autoScalingStrategy.provision()).andReturn( + new AutoScalingData(Lists.newArrayList(), Lists.newArrayList()) + ); + EasyMock.replay(autoScalingStrategy); + + boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( + Arrays.asList( + new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime()) + ), + Arrays.asList( + new TestZkWorker(testTask) + ) + ); + + Assert.assertTrue(provisionedSomething); + Assert.assertTrue(simpleResourceManagementStrategy.getStats().toList().size() == 1); + Assert.assertTrue( + simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.PROVISION + ); + + EasyMock.verify(autoScalingStrategy); + } + + @Test + public void testSomethingProvisioning() throws Exception + { + EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) + .andReturn(Lists.newArrayList()).times(2); + EasyMock.expect(autoScalingStrategy.provision()).andReturn( + new AutoScalingData(Lists.newArrayList("fake"), Lists.newArrayList("faker")) + ); + EasyMock.replay(autoScalingStrategy); + + boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( + Arrays.asList( + new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime()) + ), + Arrays.asList( + new TestZkWorker(testTask) + ) + ); + + Assert.assertTrue(provisionedSomething); + Assert.assertTrue(simpleResourceManagementStrategy.getStats().toList().size() == 1); + DateTime createdTime = simpleResourceManagementStrategy.getStats().toList().get(0).getTimestamp(); + Assert.assertTrue( + simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.PROVISION + ); + + provisionedSomething = simpleResourceManagementStrategy.doProvision( + Arrays.asList( + new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime()) + ), + Arrays.asList( + new TestZkWorker(testTask) + ) + ); + + Assert.assertFalse(provisionedSomething); + Assert.assertTrue( + simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.PROVISION + ); + DateTime anotherCreatedTime = simpleResourceManagementStrategy.getStats().toList().get(0).getTimestamp(); + Assert.assertTrue( + createdTime.equals(anotherCreatedTime) + ); + + EasyMock.verify(autoScalingStrategy); + } + + @Test + public void testDoSuccessfulTerminate() throws Exception + { + EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn(new WorkerSetupData("0", 0, null, null)); + EasyMock.replay(workerSetupManager); + + EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) + .andReturn(Lists.newArrayList()); + EasyMock.expect(autoScalingStrategy.terminate(EasyMock.>anyObject())).andReturn( + new AutoScalingData(Lists.newArrayList(), Lists.newArrayList()) + ); + EasyMock.replay(autoScalingStrategy); + + boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate( + Arrays.asList( + new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime()) + ), + Arrays.asList( + new TestZkWorker(null) + ) + ); + + Assert.assertTrue(terminatedSomething); + Assert.assertTrue(simpleResourceManagementStrategy.getStats().toList().size() == 1); + Assert.assertTrue( + simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.TERMINATE + ); + + EasyMock.verify(workerSetupManager); + EasyMock.verify(autoScalingStrategy); + } + + @Test + public void testSomethingTerminating() throws Exception + { + EasyMock.expect(workerSetupManager.getWorkerSetupData()).andReturn(new WorkerSetupData("0", 0, null, null)); + EasyMock.replay(workerSetupManager); + + EasyMock.expect(autoScalingStrategy.ipToIdLookup(EasyMock.>anyObject())) + .andReturn(Lists.newArrayList()).times(2); + EasyMock.expect(autoScalingStrategy.terminate(EasyMock.>anyObject())).andReturn( + new AutoScalingData(Lists.newArrayList("foobar"), Lists.newArrayList("foobrick")) + ); + EasyMock.replay(autoScalingStrategy); + + boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate( + Arrays.asList( + new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime()) + ), + Arrays.asList( + new TestZkWorker(null) + ) + ); + + Assert.assertTrue(terminatedSomething); + Assert.assertTrue(simpleResourceManagementStrategy.getStats().toList().size() == 1); + Assert.assertTrue( + simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.TERMINATE + ); + + terminatedSomething = simpleResourceManagementStrategy.doTerminate( + Arrays.asList( + new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime()) + ), + Arrays.asList( + new TestZkWorker(null) + ) + ); + + Assert.assertFalse(terminatedSomething); + Assert.assertTrue(simpleResourceManagementStrategy.getStats().toList().size() == 1); + Assert.assertTrue( + simpleResourceManagementStrategy.getStats().toList().get(0).getEvent() == ScalingStats.EVENT.TERMINATE + ); + + EasyMock.verify(workerSetupManager); + EasyMock.verify(autoScalingStrategy); + } + + private static class TestZkWorker extends ZkWorker + { + private final Task testTask; + + private TestZkWorker( + Task testTask + ) + { + super(new Worker("host", "ip", 3, "version"), null, null); + + this.testTask = testTask; + } + + @Override + public Set getRunningTasks() + { + if (testTask == null) { + return Sets.newHashSet(); + } + return Sets.newHashSet(testTask.getId()); + } + } +} diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/TestAutoScalingStrategy.java b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/TestAutoScalingStrategy.java new file mode 100644 index 00000000000..8213da61848 --- /dev/null +++ b/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/TestAutoScalingStrategy.java @@ -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 java.util.List; + +/** + */ +public class TestAutoScalingStrategy implements AutoScalingStrategy +{ + @Override + public AutoScalingData provision() + { + return null; + } + + @Override + public AutoScalingData terminate(List ids) + { + return null; + } + + @Override + public List ipToIdLookup(List ips) + { + return null; + } +} diff --git a/pom.xml b/pom.xml index 3166c2e845b..1f1afee50e3 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.2.10-SNAPSHOT + 0.3.4-SNAPSHOT druid druid @@ -177,6 +177,11 @@ jackson-databind 2.1.4-mmx-2 + + com.fasterxml.jackson.datatype + jackson-datatype-guava + 2.1.2 + com.fasterxml.jackson.datatype jackson-datatype-joda @@ -192,6 +197,16 @@ jackson-jaxrs-json-provider 2.1.3 + + org.codehaus.jackson + jackson-core-asl + 1.9.11 + + + org.codehaus.jackson + jackson-mapper-asl + 1.9.11 + javax.inject javax.inject diff --git a/realtime/pom.xml b/realtime/pom.xml index 0c3e8ab773c..0e48bc168c0 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.10-SNAPSHOT + 0.3.4-SNAPSHOT diff --git a/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java b/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java index d98997b5051..efc0c7a598f 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/FireDepartmentConfig.java @@ -19,6 +19,7 @@ package com.metamx.druid.realtime; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; @@ -31,6 +32,7 @@ public class FireDepartmentConfig private final int maxRowsInMemory; private final Period intermediatePersistPeriod; + @JsonCreator public FireDepartmentConfig( @JsonProperty("maxRowsInMemory") int maxRowsInMemory, @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java index d8fc7ebeea7..087e87d107f 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeNode.java @@ -46,9 +46,11 @@ import com.metamx.druid.http.QueryServlet; import com.metamx.druid.http.StatusServlet; import com.metamx.druid.initialization.Initialization; 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.LocalDataSegmentPusher; +import com.metamx.druid.loading.LocalDataSegmentPusherConfig; +import com.metamx.druid.loading.S3DataSegmentPusher; +import com.metamx.druid.loading.S3DataSegmentPusherConfig; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.service.ServiceEmitter; @@ -86,7 +88,7 @@ public class RealtimeNode extends BaseServerNode private final Map injectablesMap = Maps.newLinkedHashMap(); private MetadataUpdater metadataUpdater = null; - private SegmentPusher segmentPusher = null; + private DataSegmentPusher dataSegmentPusher = null; private List fireDepartments = null; private ServerView view = null; @@ -117,10 +119,10 @@ public class RealtimeNode extends BaseServerNode return this; } - public RealtimeNode setSegmentPusher(SegmentPusher segmentPusher) + public RealtimeNode setDataSegmentPusher(DataSegmentPusher dataSegmentPusher) { - Preconditions.checkState(this.segmentPusher == null, "Cannot set segmentPusher once it has already been set."); - this.segmentPusher = segmentPusher; + Preconditions.checkState(this.dataSegmentPusher == null, "Cannot set segmentPusher once it has already been set."); + this.dataSegmentPusher = dataSegmentPusher; return this; } @@ -144,10 +146,10 @@ public class RealtimeNode extends BaseServerNode return metadataUpdater; } - public SegmentPusher getSegmentPusher() + public DataSegmentPusher getDataSegmentPusher() { initializeSegmentPusher(); - return segmentPusher; + return dataSegmentPusher; } public List getFireDepartments() @@ -220,7 +222,7 @@ public class RealtimeNode extends BaseServerNode } injectables.put("queryRunnerFactoryConglomerate", getConglomerate()); - injectables.put("segmentPusher", segmentPusher); + injectables.put("segmentPusher", dataSegmentPusher); injectables.put("metadataUpdater", metadataUpdater); injectables.put("serverView", view); injectables.put("serviceEmitter", getEmitter()); @@ -256,22 +258,32 @@ public class RealtimeNode extends BaseServerNode private void initializeSegmentPusher() { - if (segmentPusher == null) { + if (dataSegmentPusher == null) { final Properties props = getProps(); - final RestS3Service s3Client; - try { - s3Client = new RestS3Service( - new AWSCredentials( - PropUtils.getProperty(props, "com.metamx.aws.accessKey"), - PropUtils.getProperty(props, "com.metamx.aws.secretKey") - ) + if (Boolean.parseBoolean(props.getProperty("druid.pusher.local", "false"))) { + dataSegmentPusher = new LocalDataSegmentPusher( + getConfigFactory().build(LocalDataSegmentPusherConfig.class), getJsonMapper() ); } - catch (S3ServiceException e) { - throw Throwables.propagate(e); - } + else { - segmentPusher = new S3SegmentPusher(s3Client, getConfigFactory().build(S3SegmentPusherConfig.class), getJsonMapper()); + final RestS3Service s3Client; + try { + s3Client = new RestS3Service( + new AWSCredentials( + PropUtils.getProperty(props, "com.metamx.aws.accessKey"), + PropUtils.getProperty(props, "com.metamx.aws.secretKey") + ) + ); + } + catch (S3ServiceException e) { + throw Throwables.propagate(e); + } + + dataSegmentPusher = new S3DataSegmentPusher( + s3Client, getConfigFactory().build(S3DataSegmentPusherConfig.class), getJsonMapper() + ); + } } } diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java index 47a4ef0f40f..775dc7d5305 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimePlumberSchool.java @@ -48,7 +48,7 @@ import com.metamx.druid.index.Segment; import com.metamx.druid.index.v1.IndexGranularity; 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.MetricsEmittingQueryRunner; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryRunnerFactory; @@ -95,7 +95,7 @@ public class RealtimePlumberSchool implements PlumberSchool private volatile RejectionPolicyFactory rejectionPolicyFactory = null; private volatile QueryRunnerFactoryConglomerate conglomerate = null; - private volatile SegmentPusher segmentPusher = null; + private volatile DataSegmentPusher dataSegmentPusher = null; private volatile MetadataUpdater metadataUpdater = null; private volatile ServerView serverView = null; private ServiceEmitter emitter; @@ -130,9 +130,9 @@ public class RealtimePlumberSchool implements PlumberSchool } @JacksonInject("segmentPusher") - public void setSegmentPusher(SegmentPusher segmentPusher) + public void setDataSegmentPusher(DataSegmentPusher dataSegmentPusher) { - this.segmentPusher = segmentPusher; + this.dataSegmentPusher = dataSegmentPusher; } @JacksonInject("metadataUpdater") @@ -307,7 +307,7 @@ public class RealtimePlumberSchool implements PlumberSchool } } - final File mergedFile; + File mergedFile = null; try { List indexes = Lists.newArrayList(); for (FireHydrant fireHydrant : sink) { @@ -325,7 +325,7 @@ public class RealtimePlumberSchool implements PlumberSchool QueryableIndex index = IndexIO.loadIndex(mergedFile); - DataSegment segment = segmentPusher.push( + DataSegment segment = dataSegmentPusher.push( mergedFile, sink.getSegment().withDimensions(Lists.newArrayList(index.getAvailableDimensions())) ); @@ -337,6 +337,19 @@ public class RealtimePlumberSchool implements PlumberSchool .addData("interval", interval) .emit(); } + + + if (mergedFile != null) { + try { + if (mergedFile != null) { + log.info("Deleting Index File[%s]", mergedFile); + FileUtils.deleteDirectory(mergedFile); + } + } + catch (IOException e) { + log.warn(e, "Error deleting directory[%s]", mergedFile); + } + } } } ); @@ -512,7 +525,7 @@ public class RealtimePlumberSchool implements PlumberSchool private void verifyState() { Preconditions.checkNotNull(conglomerate, "must specify a queryRunnerFactoryConglomerate to do this action."); - Preconditions.checkNotNull(segmentPusher, "must specify a segmentPusher to do this action."); + Preconditions.checkNotNull(dataSegmentPusher, "must specify a segmentPusher to do this action."); Preconditions.checkNotNull(metadataUpdater, "must specify a metadataUpdater to do this action."); Preconditions.checkNotNull(serverView, "must specify a serverView to do this action."); Preconditions.checkNotNull(emitter, "must specify a serviceEmitter to do this action."); diff --git a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java b/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java index 5a74b17e223..007ea188b82 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/S3SegmentPusher.java @@ -20,20 +20,22 @@ package com.metamx.druid.realtime; import com.fasterxml.jackson.databind.ObjectMapper; -import com.metamx.druid.loading.S3SegmentPusherConfig; -import com.metamx.druid.loading.SegmentPusher; +import com.metamx.druid.loading.DataSegmentPusher; +import com.metamx.druid.loading.S3DataSegmentPusherConfig; import org.jets3t.service.impl.rest.httpclient.RestS3Service; /** * A placeholder class to make the move of the SegmentPushers to a new package backwards compatible + * + * Exists in 0.2, can be removed from 0.3 on */ @Deprecated -public class S3SegmentPusher extends com.metamx.druid.loading.S3SegmentPusher implements SegmentPusher +public class S3SegmentPusher extends com.metamx.druid.loading.S3DataSegmentPusher implements DataSegmentPusher { public S3SegmentPusher( RestS3Service s3Client, - S3SegmentPusherConfig config, + S3DataSegmentPusherConfig config, ObjectMapper jsonMapper ) { diff --git a/server/pom.xml b/server/pom.xml index 8e715da4e8e..e373da72f3d 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.2.10-SNAPSHOT + 0.3.4-SNAPSHOT diff --git a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java index 3bde07908c7..38e7d1e4a39 100644 --- a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java +++ b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java @@ -30,7 +30,7 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.collect.CountingMap; import com.metamx.druid.index.Segment; import com.metamx.druid.loading.SegmentLoader; -import com.metamx.druid.loading.StorageAdapterLoadingException; +import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.partition.PartitionChunk; import com.metamx.druid.partition.PartitionHolder; import com.metamx.druid.query.BySegmentQueryRunner; @@ -104,24 +104,24 @@ public class ServerManager implements QuerySegmentWalker } } - public void loadSegment(final DataSegment segment) throws StorageAdapterLoadingException + public void loadSegment(final DataSegment segment) throws SegmentLoadingException { final Segment adapter; try { adapter = segmentLoader.getSegment(segment); } - catch (StorageAdapterLoadingException e) { + catch (SegmentLoadingException e) { try { segmentLoader.cleanup(segment); } - catch (StorageAdapterLoadingException e1) { + catch (SegmentLoadingException e1) { // ignore } throw e; } if (adapter == null) { - throw new StorageAdapterLoadingException("Null adapter from loadSpec[%s]", segment.getLoadSpec()); + throw new SegmentLoadingException("Null adapter from loadSpec[%s]", segment.getLoadSpec()); } synchronized (lock) { @@ -139,7 +139,7 @@ public class ServerManager implements QuerySegmentWalker ); if ((entry != null) && (entry.getChunk(segment.getShardSpec().getPartitionNum()) != null)) { log.info("Told to load a adapter for a segment[%s] that already exists", segment.getIdentifier()); - throw new StorageAdapterLoadingException("Segment already exists[%s]", segment.getIdentifier()); + throw new SegmentLoadingException("Segment already exists[%s]", segment.getIdentifier()); } loadedIntervals.add( @@ -154,7 +154,7 @@ public class ServerManager implements QuerySegmentWalker } } - public void dropSegment(final DataSegment segment) throws StorageAdapterLoadingException + public void dropSegment(final DataSegment segment) throws SegmentLoadingException { String dataSource = segment.getDataSource(); synchronized (lock) { diff --git a/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java b/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java index 57d64e0ba32..1951205975c 100644 --- a/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java +++ b/server/src/main/java/com/metamx/druid/coordination/ZkCoordinator.java @@ -29,7 +29,7 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidServer; -import com.metamx.druid.loading.StorageAdapterLoadingException; +import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.emitter.service.AlertEvent; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.phonebook.PhoneBook; @@ -245,14 +245,14 @@ public class ZkCoordinator implements DataSegmentChangeHandler } catch (IOException e) { removeSegment(segment); - throw new StorageAdapterLoadingException( + throw new SegmentLoadingException( "Failed to write to disk segment info cache file[%s]", segmentInfoCacheFile ); } yp.announce(servedSegmentsLocation, segment.getIdentifier(), segment); } - catch (StorageAdapterLoadingException e) { + catch (SegmentLoadingException e) { log.error(e, "Failed to load segment[%s]", segment); emitter.emit( new AlertEvent.Builder().build( diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java index e76b372978f..da1dbcc51e1 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseRuleManager.java @@ -191,7 +191,11 @@ public class DatabaseRuleManager public Map> withHandle(Handle handle) throws Exception { return handle.createQuery( - String.format("SELECT dataSource, payload FROM %s", config.getRuleTable()) + // Return latest version rule by dataSource + String.format( + "SELECT %1$s.dataSource, %1$s.payload FROM %1$s INNER JOIN(SELECT dataSource, max(version) as version, payload FROM %1$s GROUP BY dataSource) ds ON %1$s.datasource = ds.datasource and %1$s.version = ds.version", + config.getRuleTable() + ) ).fold( Maps.>newHashMap(), new Folder3>, Map>() diff --git a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java index b17a2f63299..004f5ce926c 100644 --- a/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java +++ b/server/src/main/java/com/metamx/druid/db/DatabaseSegmentManager.java @@ -24,6 +24,7 @@ import com.google.common.base.Function; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; +import com.metamx.common.MapUtils; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; @@ -46,6 +47,7 @@ import org.skife.jdbi.v2.tweak.HandleCallback; import javax.annotation.Nullable; import java.sql.SQLException; +import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Map; @@ -344,6 +346,44 @@ public class DatabaseSegmentManager return dataSources.get().values(); } + public Collection getAllDatasourceNames() + { + synchronized (lock) { + return dbi.withHandle( + new HandleCallback>() + { + @Override + public List withHandle(Handle handle) throws Exception + { + return handle.createQuery( + String.format("SELECT DISTINCT(datasource) FROM %s", config.getSegmentTable()) + ) + .fold( + Lists.newArrayList(), + new Folder3, Map>() + { + @Override + public ArrayList fold( + ArrayList druidDataSources, + Map stringObjectMap, + FoldController foldController, + StatementContext statementContext + ) throws SQLException + { + druidDataSources.add( + MapUtils.getString(stringObjectMap, "datasource") + ); + return druidDataSources; + } + } + ); + + } + } + ); + } + } + public void poll() { try { diff --git a/server/src/main/java/com/metamx/druid/http/ComputeNode.java b/server/src/main/java/com/metamx/druid/http/ComputeNode.java index 2230932d9a1..08b3eb93da3 100644 --- a/server/src/main/java/com/metamx/druid/http/ComputeNode.java +++ b/server/src/main/java/com/metamx/druid/http/ComputeNode.java @@ -39,7 +39,7 @@ import com.metamx.druid.coordination.ZkCoordinatorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerInit; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.loading.QueryableLoaderConfig; +import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.metrics.ServerMonitor; import com.metamx.druid.query.MetricsEmittingExecutorService; @@ -172,7 +172,7 @@ public class ComputeNode extends BaseServerNode ); setSegmentLoader( - ServerInit.makeDefaultQueryableLoader(s3Client, getConfigFactory().build(QueryableLoaderConfig.class)) + ServerInit.makeDefaultQueryableLoader(s3Client, getConfigFactory().build(SegmentLoaderConfig.class)) ); } catch (S3ServiceException e) { diff --git a/server/src/main/java/com/metamx/druid/http/InfoResource.java b/server/src/main/java/com/metamx/druid/http/InfoResource.java index 800e3a93e46..e434bff995d 100644 --- a/server/src/main/java/com/metamx/druid/http/InfoResource.java +++ b/server/src/main/java/com/metamx/druid/http/InfoResource.java @@ -33,7 +33,10 @@ import com.metamx.druid.client.ServerInventoryManager; import com.metamx.druid.coordination.DruidClusterInfo; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseSegmentManager; +import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.rules.Rule; +import com.metamx.druid.merge.ClientKillQuery; +import org.joda.time.Interval; import javax.annotation.Nullable; import javax.inject.Inject; @@ -58,6 +61,7 @@ import java.util.TreeSet; @Path("/info") public class InfoResource { + private final DruidMaster master; private final ServerInventoryManager serverInventoryManager; private final DatabaseSegmentManager databaseSegmentManager; private final DatabaseRuleManager databaseRuleManager; @@ -65,12 +69,14 @@ public class InfoResource @Inject public InfoResource( + DruidMaster master, ServerInventoryManager serverInventoryManager, DatabaseSegmentManager databaseSegmentManager, DatabaseRuleManager databaseRuleManager, DruidClusterInfo druidClusterInfo ) { + this.master = master; this.serverInventoryManager = serverInventoryManager; this.databaseSegmentManager = databaseSegmentManager; this.databaseRuleManager = databaseRuleManager; @@ -362,11 +368,17 @@ public class InfoResource @DELETE @Path("/datasources/{dataSourceName}") public Response deleteDataSource( - @PathParam("dataSourceName") final String dataSourceName + @PathParam("dataSourceName") final String dataSourceName, + @QueryParam("kill") final String kill, + @QueryParam("interval") final String interval ) { - if (!databaseSegmentManager.removeDatasource(dataSourceName)) { - return Response.status(Response.Status.NOT_FOUND).build(); + if (kill != null && Boolean.valueOf(kill)) { + master.killSegments(new ClientKillQuery(dataSourceName, new Interval(interval))); + } else { + if (!databaseSegmentManager.removeDatasource(dataSourceName)) { + return Response.status(Response.Status.NOT_FOUND).build(); + } } return Response.status(Response.Status.OK).build(); @@ -548,10 +560,14 @@ public class InfoResource @Path("/db/datasources") @Produces("application/json") public Response getDatabaseDataSources( - @QueryParam("full") String full + @QueryParam("full") String full, + @QueryParam("includeDisabled") String includeDisabled ) { Response.ResponseBuilder builder = Response.status(Response.Status.OK); + if (includeDisabled != null) { + return builder.entity(databaseSegmentManager.getAllDatasourceNames()).build(); + } if (full != null) { return builder.entity(databaseSegmentManager.getInventory()).build(); } diff --git a/server/src/main/java/com/metamx/druid/http/MasterMain.java b/server/src/main/java/com/metamx/druid/http/MasterMain.java index 5ba559d4656..64e63176d0c 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -191,7 +191,9 @@ public class MasterMain emitter, scheduledExecutorFactory, new ConcurrentHashMap(), - serviceProvider + serviceProvider, + httpClient, + new ToStringResponseHandler(Charsets.UTF_8) ); lifecycle.addManagedInstance(master); diff --git a/server/src/main/java/com/metamx/druid/http/MasterResource.java b/server/src/main/java/com/metamx/druid/http/MasterResource.java index ca992815eb6..b725ed7f358 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterResource.java +++ b/server/src/main/java/com/metamx/druid/http/MasterResource.java @@ -21,13 +21,13 @@ package com.metamx.druid.http; import com.metamx.druid.master.DruidMaster; import com.metamx.druid.master.LoadPeonCallback; +import com.metamx.druid.merge.ClientKillQuery; import javax.inject.Inject; import javax.ws.rs.Consumes; import javax.ws.rs.GET; import javax.ws.rs.POST; import javax.ws.rs.Path; -import javax.ws.rs.PathParam; import javax.ws.rs.Produces; import javax.ws.rs.core.Response; import java.util.List; diff --git a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java index 1d727f9abe3..7cd6caf3c1b 100644 --- a/server/src/main/java/com/metamx/druid/initialization/ServerInit.java +++ b/server/src/main/java/com/metamx/druid/initialization/ServerInit.java @@ -26,16 +26,16 @@ import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.DruidProcessingConfig; import com.metamx.druid.loading.DelegatingSegmentLoader; +import com.metamx.druid.loading.LocalDataSegmentPuller; import com.metamx.druid.loading.MMappedQueryableIndexFactory; import com.metamx.druid.loading.QueryableIndexFactory; -import com.metamx.druid.loading.S3SegmentPuller; +import com.metamx.druid.loading.S3DataSegmentPuller; +import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.loading.SingleSegmentLoader; import com.metamx.druid.query.group.GroupByQueryEngine; import com.metamx.druid.query.group.GroupByQueryEngineConfig; import com.metamx.druid.Query; import com.metamx.druid.collect.StupidPool; -import com.metamx.druid.loading.QueryableLoaderConfig; -import com.metamx.druid.loading.S3ZippedSegmentPuller; import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.group.GroupByQuery; @@ -64,24 +64,22 @@ public class ServerInit public static SegmentLoader makeDefaultQueryableLoader( RestS3Service s3Client, - QueryableLoaderConfig config + SegmentLoaderConfig config ) { DelegatingSegmentLoader delegateLoader = new DelegatingSegmentLoader(); - final S3SegmentPuller segmentGetter = new S3SegmentPuller(s3Client, config); - final S3ZippedSegmentPuller zippedGetter = new S3ZippedSegmentPuller(s3Client, config); - final QueryableIndexFactory factory; - if ("mmap".equals(config.getQueryableFactoryType())) { - factory = new MMappedQueryableIndexFactory(); - } else { - throw new ISE("Unknown queryableFactoryType[%s]", config.getQueryableFactoryType()); - } + final S3DataSegmentPuller segmentGetter = new S3DataSegmentPuller(s3Client); + final QueryableIndexFactory factory = new MMappedQueryableIndexFactory(); + + SingleSegmentLoader s3segmentLoader = new SingleSegmentLoader(segmentGetter, factory, config); + SingleSegmentLoader localSegmentLoader = new SingleSegmentLoader(new LocalDataSegmentPuller(), factory, config); delegateLoader.setLoaderTypes( ImmutableMap.builder() - .put("s3", new SingleSegmentLoader(segmentGetter, factory)) - .put("s3_zip", new SingleSegmentLoader(zippedGetter, factory)) + .put("s3", s3segmentLoader) + .put("s3_zip", s3segmentLoader) + .put("local", localSegmentLoader) .build() ); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java b/server/src/main/java/com/metamx/druid/loading/DataSegmentPuller.java similarity index 51% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java rename to server/src/main/java/com/metamx/druid/loading/DataSegmentPuller.java index d3a32aca462..b821c653a6e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskContext.java +++ b/server/src/main/java/com/metamx/druid/loading/DataSegmentPuller.java @@ -17,43 +17,31 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator; +package com.metamx.druid.loading; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.client.DataSegment; - - -import java.util.Set; +import java.io.File; /** - * Information gathered by the coordinator, after acquiring a lock, that may be useful to a task. */ -public class TaskContext +public interface DataSegmentPuller { - final String version; - final Set currentSegments; + /** + * Pull down segment files for the given DataSegment and put them in the given directory. + * + * @param segment The segment to pull down files for + * @param dir The directory to store the files in + * @throws SegmentLoadingException if there are any errors + */ + public void getSegmentFiles(DataSegment segment, File dir) throws SegmentLoadingException; - @JsonCreator - public TaskContext( - @JsonProperty("version") String version, - @JsonProperty("currentSegments") Set currentSegments - ) - { - this.version = version; - this.currentSegments = currentSegments; - } - - @JsonProperty - public String getVersion() - { - return version; - } - - @JsonProperty - public Set getCurrentSegments() - { - return currentSegments; - } + /** + * Returns the last modified time of the given segment. + * + * @param segment The segment to check the last modified time for + * @return the last modified time in millis from the epoch + * @throws SegmentLoadingException if there are any errors + */ + public long getLastModified(DataSegment segment) throws SegmentLoadingException; } diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusher.java similarity index 96% rename from server/src/main/java/com/metamx/druid/loading/SegmentPusher.java rename to server/src/main/java/com/metamx/druid/loading/DataSegmentPusher.java index 3700215efc1..5369480d6b9 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusher.java @@ -24,7 +24,7 @@ import com.metamx.druid.client.DataSegment; import java.io.File; import java.io.IOException; -public interface SegmentPusher +public interface DataSegmentPusher { public DataSegment push(File file, DataSegment segment) throws IOException; } diff --git a/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java new file mode 100644 index 00000000000..e72bd787bb3 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java @@ -0,0 +1,44 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.loading; + +import com.google.common.base.Joiner; +import com.metamx.druid.client.DataSegment; + +/** + */ +public class DataSegmentPusherUtil +{ + private static final Joiner JOINER = Joiner.on("/").skipNulls(); + + public static String getStorageDir(DataSegment segment) + { + return JOINER.join( + segment.getDataSource(), + String.format( + "%s_%s", + segment.getInterval().getStart(), + segment.getInterval().getEnd() + ), + segment.getVersion(), + segment.getShardSpec().getPartitionNum() + ); + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java index d576e59ae82..0f8e1e7074f 100644 --- a/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/DelegatingSegmentLoader.java @@ -44,24 +44,24 @@ public class DelegatingSegmentLoader implements SegmentLoader } @Override - public Segment getSegment(DataSegment segment) throws StorageAdapterLoadingException + public Segment getSegment(DataSegment segment) throws SegmentLoadingException { return getLoader(segment.getLoadSpec()).getSegment(segment); } @Override - public void cleanup(DataSegment segment) throws StorageAdapterLoadingException + public void cleanup(DataSegment segment) throws SegmentLoadingException { getLoader(segment.getLoadSpec()).cleanup(segment); } - private SegmentLoader getLoader(Map loadSpec) throws StorageAdapterLoadingException + private SegmentLoader getLoader(Map loadSpec) throws SegmentLoadingException { String type = MapUtils.getString(loadSpec, "type"); SegmentLoader loader = loaderTypes.get(type); if (loader == null) { - throw new StorageAdapterLoadingException("Unknown loader type[%s]. Known types are %s", type, loaderTypes.keySet()); + throw new SegmentLoadingException("Unknown loader type[%s]. Known types are %s", type, loaderTypes.keySet()); } return loader; } diff --git a/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPuller.java new file mode 100644 index 00000000000..8cdb8e0a7a2 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPuller.java @@ -0,0 +1,105 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.loading; + +import com.google.common.io.Files; +import com.metamx.common.MapUtils; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.utils.CompressionUtils; + +import java.io.File; +import java.io.IOException; +import java.util.Map; + +/** + */ +public class LocalDataSegmentPuller implements DataSegmentPuller +{ + private static final Logger log = new Logger(LocalDataSegmentPuller.class); + + @Override + public void getSegmentFiles(DataSegment segment, File dir) throws SegmentLoadingException + { + final File path = getFile(segment); + + if (path.isDirectory()) { + if (path.equals(dir)) { + log.info("Asked to load [%s] into itself, done!", dir); + return; + } + + log.info("Copying files from [%s] to [%s]", path, dir); + File file = null; + try { + final File[] files = path.listFiles(); + for (int i = 0; i < files.length; ++i) { + file = files[i]; + Files.copy(file, new File(dir, file.getName())); + } + } + catch (IOException e) { + throw new SegmentLoadingException(e, "Unable to copy file[%s].", file); + } + } else { + if (!path.getName().endsWith(".zip")) { + throw new SegmentLoadingException("File is not a zip file[%s]", path); + } + + log.info("Unzipping local file[%s] to [%s]", path, dir); + try { + CompressionUtils.unzip(path, dir); + } + catch (IOException e) { + throw new SegmentLoadingException(e, "Unable to unzip file[%s]", path); + } + } + } + + @Override + public long getLastModified(DataSegment segment) throws SegmentLoadingException + { + final File file = getFile(segment); + + long lastModified = Long.MAX_VALUE; + if (file.isDirectory()) { + for (File childFile : file.listFiles()) { + lastModified = Math.min(childFile.lastModified(), lastModified); + } + } + else { + lastModified = file.lastModified(); + } + + return lastModified; + } + + private File getFile(DataSegment segment) throws SegmentLoadingException + { + final Map loadSpec = segment.getLoadSpec(); + final File path = new File(MapUtils.getString(loadSpec, "path")); + + if (!path.exists()) { + throw new SegmentLoadingException("Asked to load path[%s], but it doesn't exist.", path); + } + + return path; + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java new file mode 100644 index 00000000000..1493b162572 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusher.java @@ -0,0 +1,96 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.loading; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.google.common.io.ByteStreams; +import com.google.common.io.Files; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.v1.IndexIO; +import com.metamx.druid.utils.CompressionUtils; + +import java.io.File; +import java.io.IOException; + +/** + */ +public class LocalDataSegmentPusher implements DataSegmentPusher +{ + private static final Logger log = new Logger(LocalDataSegmentPusher.class); + + private final LocalDataSegmentPusherConfig config; + private final ObjectMapper jsonMapper; + + public LocalDataSegmentPusher( + LocalDataSegmentPusherConfig config, + ObjectMapper jsonMapper + ) + { + this.config = config; + this.jsonMapper = jsonMapper; + } + + @Override + public DataSegment push(File dataSegmentFile, DataSegment segment) throws IOException + { + File outDir = new File(config.getStorageDirectory(), DataSegmentPusherUtil.getStorageDir(segment)); + + if (dataSegmentFile.equals(outDir)) { + long size = 0; + for (File file : dataSegmentFile.listFiles()) { + size += file.length(); + } + + return createDescriptorFile( + segment.withLoadSpec(makeLoadSpec(outDir)) + .withSize(size) + .withBinaryVersion(IndexIO.getVersionFromDir(dataSegmentFile)), + outDir + ); + } + + outDir.mkdirs(); + File outFile = new File(outDir, "index.zip"); + log.info("Compressing files from[%s] to [%s]", dataSegmentFile, outFile); + long size = CompressionUtils.zip(dataSegmentFile, outFile); + + return createDescriptorFile( + segment.withLoadSpec(makeLoadSpec(outFile)) + .withSize(size) + .withBinaryVersion(IndexIO.getVersionFromDir(dataSegmentFile)), + outDir + ); + } + + private DataSegment createDescriptorFile(DataSegment segment, File outDir) throws IOException + { + File descriptorFile = new File(outDir, "descriptor.json"); + log.info("Creating descriptor file at[%s]", descriptorFile); + Files.copy(ByteStreams.newInputStreamSupplier(jsonMapper.writeValueAsBytes(segment)), descriptorFile); + return segment; + } + + private ImmutableMap makeLoadSpec(File outFile) + { + return ImmutableMap.of("type", "local", "path", outFile.toString()); + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentGetterConfig.java b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusherConfig.java similarity index 86% rename from server/src/main/java/com/metamx/druid/loading/S3SegmentGetterConfig.java rename to server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusherConfig.java index c2a4c7f6308..d33a9a5130b 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentGetterConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/LocalDataSegmentPusherConfig.java @@ -25,8 +25,8 @@ import java.io.File; /** */ -public abstract class S3SegmentGetterConfig +public abstract class LocalDataSegmentPusherConfig { - @Config("druid.paths.indexCache") - public abstract File getCacheDirectory(); + @Config("druid.pusher.local.storageDirectory") + public abstract File getStorageDirectory(); } diff --git a/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java b/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java index 648813d62ac..9896c3f800b 100644 --- a/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java +++ b/server/src/main/java/com/metamx/druid/loading/MMappedQueryableIndexFactory.java @@ -34,7 +34,7 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory private static final Logger log = new Logger(MMappedQueryableIndexFactory.class); @Override - public QueryableIndex factorize(File parentDir) throws StorageAdapterLoadingException + public QueryableIndex factorize(File parentDir) throws SegmentLoadingException { try { if (! IndexIO.canBeMapped(parentDir)) { @@ -46,11 +46,11 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory IndexIO.storeLatest(IndexIO.readIndex(parentDir), canBeMappedDir); if (! IndexIO.canBeMapped(canBeMappedDir)) { - throw new StorageAdapterLoadingException("WTF!? newly written file[%s] cannot be mapped!?", canBeMappedDir); + throw new SegmentLoadingException("WTF!? newly written file[%s] cannot be mapped!?", canBeMappedDir); } for (File file : canBeMappedDir.listFiles()) { if (! file.renameTo(new File(parentDir, file.getName()))) { - throw new StorageAdapterLoadingException("Couldn't rename[%s] to [%s]", canBeMappedDir, parentDir); + throw new SegmentLoadingException("Couldn't rename[%s] to [%s]", canBeMappedDir, parentDir); } } FileUtils.deleteDirectory(canBeMappedDir); @@ -66,7 +66,7 @@ public class MMappedQueryableIndexFactory implements QueryableIndexFactory catch (IOException e2) { log.error(e, "Problem deleting parentDir[%s]", parentDir); } - throw new StorageAdapterLoadingException(e, e.getMessage()); + throw new SegmentLoadingException(e, "%s", e.getMessage()); } } } diff --git a/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java b/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java index d7f60309aa6..276bbc2028a 100644 --- a/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java +++ b/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java @@ -27,5 +27,5 @@ import java.io.File; */ public interface QueryableIndexFactory { - public QueryableIndex factorize(File parentDir) throws StorageAdapterLoadingException; + public QueryableIndex factorize(File parentDir) throws SegmentLoadingException; } diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java new file mode 100644 index 00000000000..011e1633ca1 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPuller.java @@ -0,0 +1,170 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + +package com.metamx.druid.loading; + +import com.google.common.io.ByteStreams; +import com.google.common.io.Closeables; +import com.google.common.io.Files; +import com.google.inject.Inject; +import com.metamx.common.ISE; +import com.metamx.common.MapUtils; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.common.s3.S3Utils; +import com.metamx.druid.utils.CompressionUtils; +import org.apache.commons.io.FileUtils; +import org.jets3t.service.S3ServiceException; +import org.jets3t.service.ServiceException; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; +import org.jets3t.service.model.S3Bucket; +import org.jets3t.service.model.S3Object; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.util.Map; +import java.util.zip.GZIPInputStream; + +/** + */ +public class S3DataSegmentPuller implements DataSegmentPuller +{ + private static final Logger log = new Logger(S3DataSegmentPuller.class); + + private static final String BUCKET = "bucket"; + private static final String KEY = "key"; + + private final RestS3Service s3Client; + + @Inject + public S3DataSegmentPuller( + RestS3Service s3Client + ) + { + this.s3Client = s3Client; + } + + @Override + public void getSegmentFiles(DataSegment segment, File outDir) throws SegmentLoadingException + { + S3Coords s3Coords = new S3Coords(segment); + + log.info("Pulling index at path[%s] to outDir[%s]", s3Coords, outDir); + + if (!isObjectInBucket(s3Coords)) { + throw new SegmentLoadingException("IndexFile[%s] does not exist.", s3Coords); + } + + if (!outDir.exists()) { + outDir.mkdirs(); + } + + if (!outDir.isDirectory()) { + throw new ISE("outDir[%s] must be a directory.", outDir); + } + + long startTime = System.currentTimeMillis(); + S3Object s3Obj = null; + + try { + s3Obj = s3Client.getObject(new S3Bucket(s3Coords.bucket), s3Coords.path); + + InputStream in = null; + try { + in = s3Obj.getDataInputStream(); + final String key = s3Obj.getKey(); + if (key.endsWith(".zip")) { + CompressionUtils.unzip(in, outDir); + } else if (key.endsWith(".gz")) { + final File outFile = new File(outDir, toFilename(key, ".gz")); + ByteStreams.copy(new GZIPInputStream(in), Files.newOutputStreamSupplier(outFile)); + } else { + ByteStreams.copy(in, Files.newOutputStreamSupplier(new File(outDir, toFilename(key, "")))); + } + log.info("Pull of file[%s] completed in %,d millis", s3Obj, System.currentTimeMillis() - startTime); + } + catch (IOException e) { + FileUtils.deleteDirectory(outDir); + throw new SegmentLoadingException(e, "Problem decompressing object[%s]", s3Obj); + } + finally { + Closeables.closeQuietly(in); + } + } + catch (Exception e) { + throw new SegmentLoadingException(e, e.getMessage()); + } + finally { + S3Utils.closeStreamsQuietly(s3Obj); + } + + } + + private String toFilename(String key, final String suffix) + { + String filename = key.substring(key.lastIndexOf("/") + 1); // characters after last '/' + filename = filename.substring(0, filename.length() - suffix.length()); // remove the suffix from the end + return filename; + } + + private boolean isObjectInBucket(S3Coords coords) throws SegmentLoadingException + { + try { + return s3Client.isObjectInBucket(coords.bucket, coords.path); + } + catch (ServiceException e) { + throw new SegmentLoadingException(e, "S3 fail! Key[%s]", coords); + } + } + + @Override + public long getLastModified(DataSegment segment) throws SegmentLoadingException + { + S3Coords coords = new S3Coords(segment); + try { + S3Object objDetails = s3Client.getObjectDetails(new S3Bucket(coords.bucket), coords.path); + return objDetails.getLastModifiedDate().getTime(); + } + catch (S3ServiceException e) { + throw new SegmentLoadingException(e, e.getMessage()); + } + } + + private static class S3Coords + { + String bucket; + String path; + + public S3Coords(DataSegment segment) + { + Map loadSpec = segment.getLoadSpec(); + bucket = MapUtils.getString(loadSpec, BUCKET); + path = MapUtils.getString(loadSpec, KEY); + if (path.startsWith("/")) { + path = path.substring(1); + } + } + + public String toString() + { + return String.format("s3://%s/%s", bucket, path); + } + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java similarity index 67% rename from server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java rename to server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java index 5af4b905719..273a07d36f3 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java @@ -22,38 +22,34 @@ package com.metamx.druid.loading; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; import com.google.common.collect.ImmutableMap; -import com.google.common.io.Closeables; -import com.metamx.common.ISE; -import com.metamx.common.StreamUtils; +import com.google.common.io.ByteStreams; +import com.google.common.io.Files; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IndexIO; +import com.metamx.druid.utils.CompressionUtils; import com.metamx.emitter.EmittingLogger; -import org.apache.commons.io.FileUtils; -import org.apache.commons.io.IOUtils; - import org.jets3t.service.S3ServiceException; import org.jets3t.service.acl.gs.GSAccessControlList; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import org.jets3t.service.model.S3Object; -import java.io.*; +import java.io.File; +import java.io.IOException; import java.security.NoSuchAlgorithmException; -import java.util.zip.ZipEntry; -import java.util.zip.ZipOutputStream; -public class S3SegmentPusher implements SegmentPusher +public class S3DataSegmentPusher implements DataSegmentPusher { - private static final EmittingLogger log = new EmittingLogger(S3SegmentPusher.class); + private static final EmittingLogger log = new EmittingLogger(S3DataSegmentPusher.class); private static final Joiner JOINER = Joiner.on("/").skipNulls(); private final RestS3Service s3Client; - private final S3SegmentPusherConfig config; + private final S3DataSegmentPusherConfig config; private final ObjectMapper jsonMapper; - public S3SegmentPusher( - RestS3Service s3Client, - S3SegmentPusherConfig config, - ObjectMapper jsonMapper + public S3DataSegmentPusher( + RestS3Service s3Client, + S3DataSegmentPusherConfig config, + ObjectMapper jsonMapper ) { this.s3Client = s3Client; @@ -67,35 +63,11 @@ public class S3SegmentPusher implements SegmentPusher log.info("Uploading [%s] to S3", indexFilesDir); String outputKey = JOINER.join( config.getBaseKey().isEmpty() ? null : config.getBaseKey(), - segment.getDataSource(), - String.format( - "%s_%s", - segment.getInterval().getStart(), - segment.getInterval().getEnd() - ), - segment.getVersion(), - segment.getShardSpec().getPartitionNum() + DataSegmentPusherUtil.getStorageDir(segment) ); - long indexSize = 0; final File zipOutFile = File.createTempFile("druid", "index.zip"); - ZipOutputStream zipOut = null; - try { - zipOut = new ZipOutputStream(new FileOutputStream(zipOutFile)); - File[] indexFiles = indexFilesDir.listFiles(); - for (File indexFile : indexFiles) { - log.info("Adding indexFile[%s] with size[%,d]. Total size[%,d]", indexFile, indexFile.length(), indexSize); - if (indexFile.length() >= Integer.MAX_VALUE) { - throw new ISE("indexFile[%s] too large [%,d]", indexFile, indexFile.length()); - } - zipOut.putNextEntry(new ZipEntry(indexFile.getName())); - IOUtils.copy(new FileInputStream(indexFile), zipOut); - indexSize += indexFile.length(); - } - } - finally { - Closeables.closeQuietly(zipOut); - } + long indexSize = CompressionUtils.zip(indexFilesDir, zipOutFile); try { S3Object toPush = new S3Object(zipOutFile); @@ -119,7 +91,7 @@ public class S3SegmentPusher implements SegmentPusher .withBinaryVersion(IndexIO.getVersionFromDir(indexFilesDir)); File descriptorFile = File.createTempFile("druid", "descriptor.json"); - StreamUtils.copyToFileAndClose(new ByteArrayInputStream(jsonMapper.writeValueAsBytes(segment)), descriptorFile); + Files.copy(ByteStreams.newInputStreamSupplier(jsonMapper.writeValueAsBytes(segment)), descriptorFile); S3Object descriptorObject = new S3Object(descriptorFile); descriptorObject.setBucketName(outputBucket); descriptorObject.setKey(outputKey + "/descriptor.json"); @@ -128,9 +100,6 @@ public class S3SegmentPusher implements SegmentPusher log.info("Pushing %s", descriptorObject); s3Client.putObject(outputBucket, descriptorObject); - log.info("Deleting Index File[%s]", indexFilesDir); - FileUtils.deleteDirectory(indexFilesDir); - log.info("Deleting zipped index File[%s]", zipOutFile); zipOutFile.delete(); diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusherConfig.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java similarity index 95% rename from server/src/main/java/com/metamx/druid/loading/S3SegmentPusherConfig.java rename to server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java index 0bd66a1a913..a2cada422fb 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentPusherConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java @@ -24,7 +24,7 @@ import org.skife.config.Default; /** */ -public abstract class S3SegmentPusherConfig +public abstract class S3DataSegmentPusherConfig { @Config("druid.pusher.s3.bucket") public abstract String getBucket(); diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java new file mode 100644 index 00000000000..46f6acfc629 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/S3SegmentKiller.java @@ -0,0 +1,49 @@ +package com.metamx.druid.loading; + +import com.google.inject.Inject; +import com.metamx.common.MapUtils; +import com.metamx.common.logger.Logger; +import com.metamx.druid.client.DataSegment; +import org.jets3t.service.ServiceException; +import org.jets3t.service.impl.rest.httpclient.RestS3Service; + +import java.util.Collection; +import java.util.Map; + +/** + */ +public class S3SegmentKiller implements SegmentKiller +{ + private static final Logger log = new Logger(S3SegmentKiller.class); + + private final RestS3Service s3Client; + + @Inject + public S3SegmentKiller( + RestS3Service s3Client + ) + { + this.s3Client = s3Client; + } + + + @Override + public void kill(Collection segments) throws ServiceException + { + for (final DataSegment segment : segments) { + Map loadSpec = segment.getLoadSpec(); + String s3Bucket = MapUtils.getString(loadSpec, "bucket"); + String s3Path = MapUtils.getString(loadSpec, "key"); + String s3DescriptorPath = s3Path.substring(0, s3Path.lastIndexOf("/")) + "/descriptor.json"; + + if (s3Client.isObjectInBucket(s3Bucket, s3Path)) { + log.info("Removing index file[s3://%s/%s] from s3!", s3Bucket, s3Path); + s3Client.deleteObject(s3Bucket, s3Path); + } + if (s3Client.isObjectInBucket(s3Bucket, s3DescriptorPath)) { + log.info("Removing descriptor file[s3://%s/%s] from s3!", s3Bucket, s3DescriptorPath); + s3Client.deleteObject(s3Bucket, s3DescriptorPath); + } + } + } +} diff --git a/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java deleted file mode 100644 index 380489548d5..00000000000 --- a/server/src/main/java/com/metamx/druid/loading/S3SegmentPuller.java +++ /dev/null @@ -1,180 +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.loading; - -import com.google.inject.Inject; -import com.metamx.common.MapUtils; -import com.metamx.common.StreamUtils; -import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.common.s3.S3Utils; -import org.apache.commons.io.FileUtils; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Bucket; -import org.jets3t.service.model.S3Object; -import org.joda.time.DateTime; - -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.util.Map; -import java.util.zip.GZIPInputStream; - -/** - */ -public class S3SegmentPuller implements SegmentPuller -{ - private static final Logger log = new Logger(S3SegmentPuller.class); - private static final long DEFAULT_TIMEOUT = 5 * 60 * 1000; - - private static final String BUCKET = "bucket"; - private static final String KEY = "key"; - - private final RestS3Service s3Client; - private final S3SegmentGetterConfig config; - - @Inject - public S3SegmentPuller( - RestS3Service s3Client, - S3SegmentGetterConfig config - ) - { - this.s3Client = s3Client; - this.config = config; - } - - @Override - public File getSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException - { - Map loadSpec = segment.getLoadSpec(); - String s3Bucket = MapUtils.getString(loadSpec, "bucket"); - String s3Path = MapUtils.getString(loadSpec, "key"); - - log.info("Loading index at path[s3://%s/%s]", s3Bucket, s3Path); - - S3Object s3Obj = null; - File tmpFile = null; - try { - if (!s3Client.isObjectInBucket(s3Bucket, s3Path)) { - throw new StorageAdapterLoadingException("IndexFile[s3://%s/%s] does not exist.", s3Bucket, s3Path); - } - - File cacheFile = new File(config.getCacheDirectory(), computeCacheFilePath(s3Bucket, s3Path)); - - if (cacheFile.exists()) { - S3Object objDetails = s3Client.getObjectDetails(new S3Bucket(s3Bucket), s3Path); - DateTime cacheFileLastModified = new DateTime(cacheFile.lastModified()); - DateTime s3ObjLastModified = new DateTime(objDetails.getLastModifiedDate().getTime()); - if (cacheFileLastModified.isAfter(s3ObjLastModified)) { - log.info( - "Found cacheFile[%s] with modified[%s], which is after s3Obj[%s]. Using.", - cacheFile, - cacheFileLastModified, - s3ObjLastModified - ); - return cacheFile.getParentFile(); - } - FileUtils.deleteDirectory(cacheFile.getParentFile()); - } - - long currTime = System.currentTimeMillis(); - - tmpFile = File.createTempFile(s3Bucket, new DateTime().toString()); - log.info( - "Downloading file[s3://%s/%s] to local tmpFile[%s] for cacheFile[%s]", - s3Bucket, s3Path, tmpFile, cacheFile - ); - - s3Obj = s3Client.getObject(new S3Bucket(s3Bucket), s3Path); - StreamUtils.copyToFileAndClose(s3Obj.getDataInputStream(), tmpFile, DEFAULT_TIMEOUT); - final long downloadEndTime = System.currentTimeMillis(); - log.info("Download of file[%s] completed in %,d millis", cacheFile, downloadEndTime - currTime); - - if (!cacheFile.getParentFile().mkdirs()) { - log.info("Unable to make parent file[%s]", cacheFile.getParentFile()); - } - cacheFile.delete(); - - if (s3Path.endsWith("gz")) { - log.info("Decompressing file[%s] to [%s]", tmpFile, cacheFile); - StreamUtils.copyToFileAndClose( - new GZIPInputStream(new FileInputStream(tmpFile)), - cacheFile - ); - if (!tmpFile.delete()) { - log.error("Could not delete tmpFile[%s].", tmpFile); - } - } else { - log.info("Rename tmpFile[%s] to cacheFile[%s]", tmpFile, cacheFile); - if (!tmpFile.renameTo(cacheFile)) { - log.warn("Error renaming tmpFile[%s] to cacheFile[%s]. Copying instead.", tmpFile, cacheFile); - - StreamUtils.copyToFileAndClose(new FileInputStream(tmpFile), cacheFile); - if (!tmpFile.delete()) { - log.error("Could not delete tmpFile[%s].", tmpFile); - } - } - } - - long endTime = System.currentTimeMillis(); - log.info("Local processing of file[%s] done in %,d millis", cacheFile, endTime - downloadEndTime); - - return cacheFile.getParentFile(); - } - catch (Exception e) { - throw new StorageAdapterLoadingException(e, e.getMessage()); - } - finally { - S3Utils.closeStreamsQuietly(s3Obj); - if (tmpFile != null && tmpFile.exists()) { - log.warn("Deleting tmpFile[%s] in finally block. Why?", tmpFile); - tmpFile.delete(); - } - } - } - - private String computeCacheFilePath(String s3Bucket, String s3Path) - { - return String.format( - "%s/%s", s3Bucket, s3Path.endsWith("gz") ? s3Path.substring(0, s3Path.length() - ".gz".length()) : s3Path - ); - } - - @Override - public boolean cleanSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException - { - Map loadSpec = segment.getLoadSpec(); - File cacheFile = new File( - config.getCacheDirectory(), - computeCacheFilePath(MapUtils.getString(loadSpec, BUCKET), MapUtils.getString(loadSpec, KEY)) - ); - - try { - final File parentFile = cacheFile.getParentFile(); - log.info("Recursively deleting file[%s]", parentFile); - FileUtils.deleteDirectory(parentFile); - } - catch (IOException e) { - throw new StorageAdapterLoadingException(e, e.getMessage()); - } - - return true; - } -} diff --git a/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java deleted file mode 100644 index 8fd8ebd4542..00000000000 --- a/server/src/main/java/com/metamx/druid/loading/S3ZippedSegmentPuller.java +++ /dev/null @@ -1,187 +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.loading; - -import com.google.common.io.Closeables; -import com.metamx.common.MapUtils; -import com.metamx.common.StreamUtils; -import com.metamx.common.logger.Logger; -import com.metamx.druid.client.DataSegment; -import com.metamx.druid.common.s3.S3Utils; -import org.apache.commons.io.FileUtils; -import org.apache.commons.io.IOUtils; -import org.jets3t.service.impl.rest.httpclient.RestS3Service; -import org.jets3t.service.model.S3Bucket; -import org.jets3t.service.model.S3Object; -import org.joda.time.DateTime; - -import java.io.BufferedInputStream; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.OutputStream; -import java.util.Map; -import java.util.zip.ZipEntry; -import java.util.zip.ZipInputStream; - -/** - */ -public class S3ZippedSegmentPuller implements SegmentPuller -{ - private static final Logger log = new Logger(S3ZippedSegmentPuller.class); - - private static final String BUCKET = "bucket"; - private static final String KEY = "key"; - - private final RestS3Service s3Client; - private final S3SegmentGetterConfig config; - - public S3ZippedSegmentPuller( - RestS3Service s3Client, - S3SegmentGetterConfig config - ) - { - this.s3Client = s3Client; - this.config = config; - } - - @Override - public File getSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException - { - Map loadSpec = segment.getLoadSpec(); - String s3Bucket = MapUtils.getString(loadSpec, "bucket"); - String s3Path = MapUtils.getString(loadSpec, "key"); - - if (s3Path.startsWith("/")) { - s3Path = s3Path.substring(1); - } - - log.info("Loading index at path[s3://%s/%s]", s3Bucket, s3Path); - - S3Object s3Obj = null; - File tmpFile = null; - try { - if (!s3Client.isObjectInBucket(s3Bucket, s3Path)) { - throw new StorageAdapterLoadingException("IndexFile[s3://%s/%s] does not exist.", s3Bucket, s3Path); - } - - File cacheFile = new File(config.getCacheDirectory(), computeCacheFilePath(s3Bucket, s3Path)); - - if (cacheFile.exists()) { - S3Object objDetails = s3Client.getObjectDetails(new S3Bucket(s3Bucket), s3Path); - DateTime cacheFileLastModified = new DateTime(cacheFile.lastModified()); - DateTime s3ObjLastModified = new DateTime(objDetails.getLastModifiedDate().getTime()); - if (cacheFileLastModified.isAfter(s3ObjLastModified)) { - log.info( - "Found cacheFile[%s] with modified[%s], which is after s3Obj[%s]. Using.", - cacheFile, - cacheFileLastModified, - s3ObjLastModified - ); - return cacheFile; - } - FileUtils.deleteDirectory(cacheFile); - } - - long currTime = System.currentTimeMillis(); - - tmpFile = File.createTempFile(s3Bucket, new DateTime().toString()); - log.info( - "Downloading file[s3://%s/%s] to local tmpFile[%s] for cacheFile[%s]", - s3Bucket, s3Path, tmpFile, cacheFile - ); - - s3Obj = s3Client.getObject(new S3Bucket(s3Bucket), s3Path); - StreamUtils.copyToFileAndClose(s3Obj.getDataInputStream(), tmpFile); - final long downloadEndTime = System.currentTimeMillis(); - log.info("Download of file[%s] completed in %,d millis", cacheFile, downloadEndTime - currTime); - - if (cacheFile.exists()) { - FileUtils.deleteDirectory(cacheFile); - } - cacheFile.mkdirs(); - - ZipInputStream zipIn = null; - OutputStream out = null; - ZipEntry entry = null; - try { - zipIn = new ZipInputStream(new BufferedInputStream(new FileInputStream(tmpFile))); - while ((entry = zipIn.getNextEntry()) != null) { - out = new FileOutputStream(new File(cacheFile, entry.getName())); - IOUtils.copy(zipIn, out); - zipIn.closeEntry(); - Closeables.closeQuietly(out); - out = null; - } - } - finally { - Closeables.closeQuietly(out); - Closeables.closeQuietly(zipIn); - } - - long endTime = System.currentTimeMillis(); - log.info("Local processing of file[%s] done in %,d millis", cacheFile, endTime - downloadEndTime); - - log.info("Deleting tmpFile[%s]", tmpFile); - tmpFile.delete(); - - return cacheFile; - } - catch (Exception e) { - throw new StorageAdapterLoadingException(e, e.getMessage()); - } - finally { - S3Utils.closeStreamsQuietly(s3Obj); - if (tmpFile != null && tmpFile.exists()) { - log.warn("Deleting tmpFile[%s] in finally block. Why?", tmpFile); - tmpFile.delete(); - } - } - } - - private String computeCacheFilePath(String s3Bucket, String s3Path) - { - return new File(String.format("%s/%s", s3Bucket, s3Path)).getParent(); - } - - @Override - public boolean cleanSegmentFiles(DataSegment segment) throws StorageAdapterLoadingException - { - Map loadSpec = segment.getLoadSpec(); - File cacheFile = new File( - config.getCacheDirectory(), - computeCacheFilePath( - MapUtils.getString(loadSpec, BUCKET), - MapUtils.getString(loadSpec, KEY) - ) - ); - - try { - log.info("Deleting directory[%s]", cacheFile); - FileUtils.deleteDirectory(cacheFile); - } - catch (IOException e) { - throw new StorageAdapterLoadingException(e, e.getMessage()); - } - - return true; - } -} diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentKiller.java b/server/src/main/java/com/metamx/druid/loading/SegmentKiller.java new file mode 100644 index 00000000000..8f8746d5324 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/loading/SegmentKiller.java @@ -0,0 +1,14 @@ +package com.metamx.druid.loading; + +import com.metamx.druid.client.DataSegment; +import org.jets3t.service.ServiceException; + +import java.util.Collection; +import java.util.List; + +/** + */ +public interface SegmentKiller +{ + public void kill(Collection segments) throws ServiceException; +} diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java index 1ca54b89106..20fa5592ac2 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentLoader.java @@ -26,6 +26,6 @@ import com.metamx.druid.index.Segment; */ public interface SegmentLoader { - public Segment getSegment(DataSegment loadSpec) throws StorageAdapterLoadingException; - public void cleanup(DataSegment loadSpec) throws StorageAdapterLoadingException; + public Segment getSegment(DataSegment loadSpec) throws SegmentLoadingException; + public void cleanup(DataSegment loadSpec) throws SegmentLoadingException; } diff --git a/server/src/main/java/com/metamx/druid/loading/SegmentPuller.java b/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java similarity index 75% rename from server/src/main/java/com/metamx/druid/loading/SegmentPuller.java rename to server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java index 9cba65f425c..294c91b9a38 100644 --- a/server/src/main/java/com/metamx/druid/loading/SegmentPuller.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentLoaderConfig.java @@ -19,15 +19,20 @@ package com.metamx.druid.loading; -import com.metamx.druid.client.DataSegment; +import org.skife.config.Config; import java.io.File; -import java.util.Map; /** */ -public interface SegmentPuller +public abstract class SegmentLoaderConfig { - public File getSegmentFiles(DataSegment loadSpec) throws StorageAdapterLoadingException; - public boolean cleanSegmentFiles(DataSegment loadSpec) throws StorageAdapterLoadingException; + @Config({"druid.paths.indexCache", "druid.segmentCache.path"}) + public abstract File getCacheDirectory(); + + @Config("druid.segmentCache.deleteOnRemove") + public boolean deleteOnRemove() + { + return true; + } } diff --git a/server/src/main/java/com/metamx/druid/loading/StorageAdapterLoadingException.java b/server/src/main/java/com/metamx/druid/loading/SegmentLoadingException.java similarity index 88% rename from server/src/main/java/com/metamx/druid/loading/StorageAdapterLoadingException.java rename to server/src/main/java/com/metamx/druid/loading/SegmentLoadingException.java index d0f0ba4be93..d52fd6e3a82 100644 --- a/server/src/main/java/com/metamx/druid/loading/StorageAdapterLoadingException.java +++ b/server/src/main/java/com/metamx/druid/loading/SegmentLoadingException.java @@ -21,9 +21,9 @@ package com.metamx.druid.loading; /** */ -public class StorageAdapterLoadingException extends Exception +public class SegmentLoadingException extends Exception { - public StorageAdapterLoadingException( + public SegmentLoadingException( String formatString, Object... objs ) @@ -31,7 +31,7 @@ public class StorageAdapterLoadingException extends Exception super(String.format(formatString, objs)); } - public StorageAdapterLoadingException( + public SegmentLoadingException( Throwable cause, String formatString, Object... objs diff --git a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java index 19c3981e988..7e62f57fbf4 100644 --- a/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java +++ b/server/src/main/java/com/metamx/druid/loading/SingleSegmentLoader.java @@ -19,40 +19,133 @@ package com.metamx.druid.loading; +import com.google.common.base.Joiner; +import com.google.common.base.Throwables; import com.google.inject.Inject; +import com.metamx.common.StreamUtils; +import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.QueryableIndexSegment; import com.metamx.druid.index.Segment; +import org.apache.commons.io.FileUtils; + +import java.io.*; /** */ public class SingleSegmentLoader implements SegmentLoader { - private final SegmentPuller segmentPuller; + private static final Logger log = new Logger(SingleSegmentLoader.class); + + private final DataSegmentPuller dataSegmentPuller; private final QueryableIndexFactory factory; + private final SegmentLoaderConfig config; + private static final Joiner JOINER = Joiner.on("/").skipNulls(); @Inject public SingleSegmentLoader( - SegmentPuller segmentPuller, - QueryableIndexFactory factory + DataSegmentPuller dataSegmentPuller, + QueryableIndexFactory factory, + SegmentLoaderConfig config ) { - this.segmentPuller = segmentPuller; + this.dataSegmentPuller = dataSegmentPuller; this.factory = factory; + this.config = config; } @Override - public Segment getSegment(DataSegment segment) throws StorageAdapterLoadingException + public Segment getSegment(DataSegment segment) throws SegmentLoadingException { - final QueryableIndex index = factory.factorize(segmentPuller.getSegmentFiles(segment)); + File segmentFiles = getSegmentFiles(segment); + final QueryableIndex index = factory.factorize(segmentFiles); return new QueryableIndexSegment(segment.getIdentifier(), index); } - @Override - public void cleanup(DataSegment segment) throws StorageAdapterLoadingException + public File getSegmentFiles(DataSegment segment) throws SegmentLoadingException { - segmentPuller.cleanSegmentFiles(segment); + File localStorageDir = new File(config.getCacheDirectory(), DataSegmentPusherUtil.getStorageDir(segment)); + if (localStorageDir.exists()) { + long localLastModified = localStorageDir.lastModified(); + long remoteLastModified = dataSegmentPuller.getLastModified(segment); + if (remoteLastModified > 0 && localLastModified >= remoteLastModified) { + log.info( + "Found localStorageDir[%s] with modified[%s], which is same or after remote[%s]. Using.", + localStorageDir, localLastModified, remoteLastModified + ); + return localStorageDir; + } + } + + if (localStorageDir.exists()) { + try { + FileUtils.deleteDirectory(localStorageDir); + } + catch (IOException e) { + log.warn(e, "Exception deleting previously existing local dir[%s]", localStorageDir); + } + } + if (!localStorageDir.mkdirs()) { + log.info("Unable to make parent file[%s]", localStorageDir); + } + + dataSegmentPuller.getSegmentFiles(segment, localStorageDir); + + return localStorageDir; + } + + private File getLocalStorageDir(DataSegment segment) + { + String outputKey = JOINER.join( + segment.getDataSource(), + String.format( + "%s_%s", + segment.getInterval().getStart(), + segment.getInterval().getEnd() + ), + segment.getVersion(), + segment.getShardSpec().getPartitionNum() + ); + + return new File(config.getCacheDirectory(), outputKey); + } + + private void moveToCache(File pulledFile, File cacheFile) throws SegmentLoadingException + { + log.info("Rename pulledFile[%s] to cacheFile[%s]", pulledFile, cacheFile); + if (!pulledFile.renameTo(cacheFile)) { + log.warn("Error renaming pulledFile[%s] to cacheFile[%s]. Copying instead.", pulledFile, cacheFile); + + try { + StreamUtils.copyToFileAndClose(new FileInputStream(pulledFile), cacheFile); + } + catch (IOException e) { + throw new SegmentLoadingException( + e, + "Problem moving pulledFile[%s] to cache[%s]", + pulledFile, + cacheFile + ); + } + if (!pulledFile.delete()) { + log.error("Could not delete pulledFile[%s].", pulledFile); + } + } + } + + @Override + public void cleanup(DataSegment segment) throws SegmentLoadingException + { + File cacheFile = getLocalStorageDir(segment).getParentFile(); + + try { + log.info("Deleting directory[%s]", cacheFile); + FileUtils.deleteDirectory(cacheFile); + } + catch (IOException e) { + throw new SegmentLoadingException(e, e.getMessage()); + } } } diff --git a/server/src/main/java/com/metamx/druid/master/DruidMaster.java b/server/src/main/java/com/metamx/druid/master/DruidMaster.java index 3894606a7ff..a21408144b3 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMaster.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMaster.java @@ -22,6 +22,7 @@ package com.metamx.druid.master; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.base.Predicate; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; @@ -43,9 +44,12 @@ import com.metamx.druid.client.ServerInventoryManager; import com.metamx.druid.coordination.DruidClusterInfo; import com.metamx.druid.db.DatabaseRuleManager; import com.metamx.druid.db.DatabaseSegmentManager; +import com.metamx.druid.merge.ClientKillQuery; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; +import com.metamx.http.client.HttpClient; +import com.metamx.http.client.response.HttpResponseHandler; import com.metamx.phonebook.PhoneBook; import com.metamx.phonebook.PhoneBookPeon; import com.netflix.curator.x.discovery.ServiceProvider; @@ -55,6 +59,7 @@ import org.joda.time.DateTime; import org.joda.time.Duration; import javax.annotation.Nullable; +import java.net.URL; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -89,6 +94,9 @@ public class DruidMaster private final Map loadManagementPeons; private final ServiceProvider serviceProvider; + private final HttpClient httpClient; + private final HttpResponseHandler responseHandler; + private final ObjectMapper jsonMapper; public DruidMaster( @@ -102,7 +110,9 @@ public class DruidMaster ServiceEmitter emitter, ScheduledExecutorFactory scheduledExecutorFactory, Map loadManagementPeons, - ServiceProvider serviceProvider + ServiceProvider serviceProvider, + HttpClient httpClient, + HttpResponseHandler responseHandler ) { this.config = config; @@ -123,6 +133,8 @@ public class DruidMaster this.loadManagementPeons = loadManagementPeons; this.serviceProvider = serviceProvider; + this.httpClient = httpClient; + this.responseHandler = responseHandler; } public boolean isClusterMaster() @@ -132,44 +144,47 @@ public class DruidMaster public Map getLoadStatus() { - Map availableSegmentMap = Maps.newHashMap(); - - for (DataSegment segment : getAvailableDataSegments()) { - Integer count = availableSegmentMap.get(segment.getDataSource()); - int newCount = (count == null) ? 0 : count.intValue(); - availableSegmentMap.put(segment.getDataSource(), ++newCount); + // find available segments + Map> availableSegments = Maps.newHashMap(); + for (DataSegment dataSegment : getAvailableDataSegments()) { + Set segments = availableSegments.get(dataSegment.getDataSource()); + if (segments == null) { + segments = Sets.newHashSet(); + availableSegments.put(dataSegment.getDataSource(), segments); + } + segments.add(dataSegment); } - Map> loadedDataSources = Maps.newHashMap(); - for (DruidServer server : serverInventoryManager.getInventory()) { - for (DruidDataSource dataSource : server.getDataSources()) { - if (!loadedDataSources.containsKey(dataSource.getName())) { - TreeSet setToAdd = Sets.newTreeSet(DataSegment.bucketMonthComparator()); - setToAdd.addAll(dataSource.getSegments()); - loadedDataSources.put(dataSource.getName(), setToAdd); - } else { - loadedDataSources.get(dataSource.getName()).addAll(dataSource.getSegments()); + // find segments currently loaded + Map> segmentsInCluster = Maps.newHashMap(); + for (DruidServer druidServer : serverInventoryManager.getInventory()) { + for (DruidDataSource druidDataSource : druidServer.getDataSources()) { + Set segments = segmentsInCluster.get(druidDataSource.getName()); + if (segments == null) { + segments = Sets.newHashSet(); + segmentsInCluster.put(druidDataSource.getName(), segments); } + segments.addAll(druidDataSource.getSegments()); } } - Map loadedSegmentMap = Maps.newHashMap(); - for (Map.Entry> entry : loadedDataSources.entrySet()) { - loadedSegmentMap.put(entry.getKey(), entry.getValue().size()); - } - - Map retVal = Maps.newHashMap(); - - for (Map.Entry entry : availableSegmentMap.entrySet()) { - String key = entry.getKey(); - if (!loadedSegmentMap.containsKey(key) || entry.getValue().doubleValue() == 0.0) { - retVal.put(key, 0.0); - } else { - retVal.put(key, 100 * loadedSegmentMap.get(key).doubleValue() / entry.getValue().doubleValue()); + // compare available segments with currently loaded + Map loadStatus = Maps.newHashMap(); + for (Map.Entry> entry : availableSegments.entrySet()) { + String dataSource = entry.getKey(); + Set segmentsAvailable = entry.getValue(); + Set loadedSegments = segmentsInCluster.get(dataSource); + if (loadedSegments == null) { + loadedSegments = Sets.newHashSet(); } + Set unloadedSegments = Sets.difference(segmentsAvailable, loadedSegments); + loadStatus.put( + dataSource, + 100 * ((double) (segmentsAvailable.size() - unloadedSegments.size()) / (double) segmentsAvailable.size()) + ); } - return retVal; + return loadStatus; } public int lookupSegmentLifetime(DataSegment segment) @@ -334,6 +349,27 @@ public class DruidMaster } } + public void killSegments(ClientKillQuery killQuery) + { + try { + httpClient.post( + new URL( + String.format( + "http://%s:%s/mmx/merger/v1/index", + serviceProvider.getInstance().getAddress(), + serviceProvider.getInstance().getPort() + ) + ) + ) + .setContent("application/json", jsonMapper.writeValueAsBytes(killQuery)) + .go(responseHandler) + .get(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + public Set getAvailableDataSegments() { Set availableSegments = Sets.newTreeSet(Comparators.inverse(DataSegment.bucketMonthComparator())); @@ -593,7 +629,7 @@ public class DruidMaster public DruidMasterRuntimeParams run(DruidMasterRuntimeParams params) { // Display info about all historical servers - Iterable servers =FunctionalIterable + Iterable servers = FunctionalIterable .create(serverInventoryManager.getInventory()) .filter( new Predicate() diff --git a/server/src/main/resources/static/css/enable.css b/server/src/main/resources/static/css/enable.css new file mode 100644 index 00000000000..4c1a4cdb9b1 --- /dev/null +++ b/server/src/main/resources/static/css/enable.css @@ -0,0 +1,3 @@ +#select_datasource { + margin: 20px 0 20px 0; +} \ No newline at end of file diff --git a/server/src/main/resources/static/enable.html b/server/src/main/resources/static/enable.html new file mode 100644 index 00000000000..265ae7780df --- /dev/null +++ b/server/src/main/resources/static/enable.html @@ -0,0 +1,75 @@ + + + + + + Druid Master Console - Enable/Disable Datasources + + + + + + + + + + + + + + +

+ +
Enable/Disable Datasources
+ +
+

Enabled Datasources:

+
    +
+
+ +
+

Disabled Datasources:

+
    +
+
+ +
+ Select Data Source: + +
+ +
+ +
+

Are you sure you want to enable the selected datasource?

+
+ +
+

Are you sure you want to disable the selected datasource?

+
+ +
+
+
+ + \ No newline at end of file diff --git a/server/src/main/resources/static/index.html b/server/src/main/resources/static/index.html index bd36d0c5c53..8a6a68a7faa 100644 --- a/server/src/main/resources/static/index.html +++ b/server/src/main/resources/static/index.html @@ -20,20 +20,29 @@ - Druid Master Console - - - + Druid Master Console + + + - + \ No newline at end of file diff --git a/server/src/main/resources/static/js/enable-0.0.1.js b/server/src/main/resources/static/js/enable-0.0.1.js new file mode 100644 index 00000000000..f25b1a53a54 --- /dev/null +++ b/server/src/main/resources/static/js/enable-0.0.1.js @@ -0,0 +1,97 @@ +$(document).ready(function() { + $("button").button(); + + $("#error_dialog").dialog({ + autoOpen: false, + modal:true, + resizeable: false, + buttons: { + Ok : function() { + $(this).dialog("close"); + } + } + }); + + $("#enable_dialog").dialog({ + autoOpen: false, + modal:true, + resizeable: false, + buttons: { + Yes : function() { + var selected = $('#datasources option:selected').text(); + $.ajax({ + type: 'POST', + url:'/info/datasources/' + selected, + data: JSON.stringify(selected), + contentType:"application/json; charset=utf-8", + dataType:"json", + error: function(xhr, status, error) { + $("#enable_dialog").dialog("close"); + $("#error_dialog").html(xhr.responseText); + $("#error_dialog").dialog("open"); + }, + success: function(data, status, xhr) { + $("#enable_dialog").dialog("close"); + } + }); + }, + Cancel: function() { + $(this).dialog("close"); + } + } + }); + + $("#disable_dialog").dialog({ + autoOpen: false, + modal:true, + resizeable: false, + buttons: { + Yes : function() { + var selected = $('#datasources option:selected').text(); + $.ajax({ + type: 'DELETE', + url:'/info/datasources/' + selected, + data: JSON.stringify(selected), + contentType:"application/json; charset=utf-8", + dataType:"json", + error: function(xhr, status, error) { + $("#disable_dialog").dialog("close"); + $("#error_dialog").html(xhr.responseText); + $("#error_dialog").dialog("open"); + }, + success: function(data, status, xhr) { + $("#disable_dialog").dialog("close"); + } + }); + }, + Cancel: function() { + $(this).dialog("close"); + } + } + }); + + $.getJSON("/info/db/datasources", function(enabled_datasources) { + $.each(enabled_datasources, function(index, datasource) { + $('#enabled_datasources').append($('
  • ' + datasource + '
  • ')); + }); + + $.getJSON("/info/db/datasources?includeDisabled", function(db_datasources) { + var disabled_datasources = _.difference(db_datasources, enabled_datasources); + $.each(disabled_datasources, function(index, datasource) { + $('#disabled_datasources').append($('
  • ' + datasource + '
  • ')); + }); + $.each(db_datasources, function(index, datasource) { + $('#datasources').append($('').attr("value", datasource).text(datasource)); + }); + }); + }); + + + $("#enable").click(function() { + $("#enable_dialog").dialog("open"); + }); + + $('#disable').click(function (){ + $("#disable_dialog").dialog("open") + }); +}); \ No newline at end of file diff --git a/server/src/main/resources/static/js/kill-0.0.1.js b/server/src/main/resources/static/js/kill-0.0.1.js new file mode 100644 index 00000000000..666651128f1 --- /dev/null +++ b/server/src/main/resources/static/js/kill-0.0.1.js @@ -0,0 +1,53 @@ +$(document).ready(function() { + $("button").button(); + + $("#error_dialog").dialog({ + autoOpen: false, + modal:true, + resizeable: false, + buttons: { + Ok : function() { + $(this).dialog("close"); + } + } + }); + + $("#confirm_dialog").dialog({ + autoOpen: false, + modal:true, + resizeable: false, + buttons: { + Yes : function() { + var selected = $('#datasources option:selected').text(); + var interval = $('#interval').val(); + $.ajax({ + type: 'DELETE', + url:'/info/datasources/' + selected +'?kill=true&interval=' + interval, + contentType:"application/json; charset=utf-8", + dataType:"json", + error: function(xhr, status, error) { + $("#confirm_dialog").dialog("close"); + $("#error_dialog").html(xhr.responseText); + $("#error_dialog").dialog("open"); + }, + success: function(data, status, xhr) { + $("#confirm_dialog").dialog("close"); + } + }); + }, + Cancel: function() { + $(this).dialog("close"); + } + } + }); + + $.getJSON("/info/db/datasources?includeDisabled", function(data) { + $.each(data, function(index, datasource) { + $('#datasources').append($('').attr("value", datasource).text(datasource)); + }); + }); + + $("#confirm").click(function() { + $("#confirm_dialog").dialog("open"); + }); +}); \ No newline at end of file diff --git a/server/src/main/resources/static/kill.html b/server/src/main/resources/static/kill.html new file mode 100644 index 00000000000..8741fd25f41 --- /dev/null +++ b/server/src/main/resources/static/kill.html @@ -0,0 +1,61 @@ + + + + + + Druid Master Console - Enable/Disable Datasources + + + + + + + + + + + + + +
    + +
    Permanently Delete Segments
    + +
    + Select Data Source: + +
    + +

    Interval:

    + + +
    + +
    +

    Are you sure you want delete segments for this datasource and range? There is no going back!

    +
    + +
    +
    +
    + + \ No newline at end of file diff --git a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java index 034bcc8f42e..d10566c5da4 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java @@ -31,7 +31,6 @@ import com.metamx.common.Pair; import com.metamx.common.guava.ConcatSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; -import com.metamx.druid.Capabilities; import com.metamx.druid.Druids; import com.metamx.druid.Query; import com.metamx.druid.QueryGranularity; @@ -39,12 +38,9 @@ import com.metamx.druid.StorageAdapter; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.QueryableIndex; import com.metamx.druid.index.Segment; -import com.metamx.druid.index.brita.Filter; import com.metamx.druid.index.v1.IndexIO; -import com.metamx.druid.index.v1.SegmentIdAttachedStorageAdapter; -import com.metamx.druid.index.v1.processing.Cursor; import com.metamx.druid.loading.SegmentLoader; -import com.metamx.druid.loading.StorageAdapterLoadingException; +import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.metrics.NoopServiceEmitter; import com.metamx.druid.query.CacheStrategy; import com.metamx.druid.query.ConcatQueryRunner; @@ -54,7 +50,6 @@ import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryRunnerFactory; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.druid.query.QueryToolChest; -import com.metamx.druid.query.search.SearchHit; import com.metamx.druid.query.search.SearchQuery; import com.metamx.druid.result.Result; import com.metamx.druid.result.SearchResultValue; @@ -62,7 +57,6 @@ import com.metamx.druid.shard.NoneShardSpec; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceMetricEvent; -import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; @@ -72,7 +66,6 @@ import java.io.IOException; import java.util.Arrays; import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.concurrent.ExecutorService; /** @@ -101,7 +94,7 @@ public class ServerManagerTest } @Override - public void cleanup(DataSegment segment) throws StorageAdapterLoadingException + public void cleanup(DataSegment segment) throws SegmentLoadingException { } @@ -245,7 +238,7 @@ public class ServerManagerTest ) ); } - catch (StorageAdapterLoadingException e) { + catch (SegmentLoadingException e) { throw new RuntimeException(e); } } @@ -267,7 +260,7 @@ public class ServerManagerTest ) ); } - catch (StorageAdapterLoadingException e) { + catch (SegmentLoadingException e) { throw new RuntimeException(e); } } @@ -392,7 +385,7 @@ public class ServerManagerTest } } - public static class NoopQueryToolChest> implements QueryToolChest + public static class NoopQueryToolChest> extends QueryToolChest { @Override public QueryRunner mergeResults(QueryRunner runner) @@ -423,23 +416,5 @@ public class ServerManagerTest { return new TypeReference(){}; } - - @Override - public CacheStrategy getCacheStrategy(QueryType query) - { - return null; - } - - @Override - public QueryRunner preMergeQueryDecoration(QueryRunner runner) - { - return runner; - } - - @Override - public QueryRunner postMergeQueryDecoration(QueryRunner runner) - { - return runner; - } } } diff --git a/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java b/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java index 29d784d3631..ca41c4dfec6 100644 --- a/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java +++ b/server/src/test/java/com/metamx/druid/loading/NoopSegmentLoader.java @@ -30,7 +30,7 @@ import org.joda.time.Interval; public class NoopSegmentLoader implements SegmentLoader { @Override - public Segment getSegment(final DataSegment segment) throws StorageAdapterLoadingException + public Segment getSegment(final DataSegment segment) throws SegmentLoadingException { return new Segment() { @@ -61,7 +61,7 @@ public class NoopSegmentLoader implements SegmentLoader } @Override - public void cleanup(DataSegment loadSpec) throws StorageAdapterLoadingException + public void cleanup(DataSegment loadSpec) throws SegmentLoadingException { } } diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java index 584ae31de47..895659586ec 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java @@ -152,6 +152,8 @@ public class DruidMasterTest new NoopServiceEmitter(), scheduledExecutorFactory, loadManagementPeons, + null, + null, null ); }