diff --git a/client/pom.xml b/client/pom.xml index 207458b37c1..1c1cf4b9a99 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.19-SNAPSHOT + 0.5.33-SNAPSHOT @@ -68,6 +68,10 @@ commons-codec commons-codec + + commons-httpclient + commons-httpclient + org.skife.config config-magic diff --git a/client/src/main/java/com/metamx/druid/client/DruidServer.java b/client/src/main/java/com/metamx/druid/client/DruidServer.java index e86efdad816..a3acc3c00d8 100644 --- a/client/src/main/java/com/metamx/druid/client/DruidServer.java +++ b/client/src/main/java/com/metamx/druid/client/DruidServer.java @@ -34,6 +34,7 @@ import java.util.concurrent.ConcurrentMap; */ public class DruidServer implements Comparable { + public static final String DEFAULT_TIER = "_default_tier"; private static final Logger log = new Logger(DruidServer.class); private final Object lock = new Object(); diff --git a/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java b/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java index 89028021c18..d66ef146db4 100644 --- a/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java +++ b/client/src/main/java/com/metamx/druid/client/DruidServerConfig.java @@ -33,9 +33,10 @@ public abstract class DruidServerConfig public abstract String getHost(); @Config("druid.server.maxSize") + @Default("0") public abstract long getMaxSize(); @Config("druid.server.tier") - @Default("_default_tier") + @Default(DruidServer.DEFAULT_TIER) public abstract String getTier(); } diff --git a/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java b/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java index 4b345dc5a29..781c4ed0c77 100644 --- a/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java +++ b/client/src/main/java/com/metamx/druid/client/SingleServerInventoryView.java @@ -62,9 +62,7 @@ public class SingleServerInventoryView extends ServerInventoryView curator, exec, jsonMapper, - new TypeReference() - { - } + new TypeReference(){} ); } diff --git a/client/src/main/java/com/metamx/druid/curator/cache/SimplePathChildrenCacheFactory.java b/client/src/main/java/com/metamx/druid/curator/cache/SimplePathChildrenCacheFactory.java index 2808ce4d7b8..fb7a3044ee4 100644 --- a/client/src/main/java/com/metamx/druid/curator/cache/SimplePathChildrenCacheFactory.java +++ b/client/src/main/java/com/metamx/druid/curator/cache/SimplePathChildrenCacheFactory.java @@ -21,8 +21,11 @@ package com.metamx.druid.curator.cache; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.PathChildrenCache; +import org.apache.curator.utils.ThreadUtils; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; /** */ @@ -48,4 +51,43 @@ public class SimplePathChildrenCacheFactory implements PathChildrenCacheFactory { return new PathChildrenCache(curator, path, cacheData, compressed, exec); } + + public static class Builder + { + private static final ThreadFactory defaultThreadFactory = ThreadUtils.newThreadFactory("PathChildrenCache"); + + private boolean cacheData; + private boolean compressed; + private ExecutorService exec; + + public Builder() + { + cacheData = true; + compressed = false; + exec = Executors.newSingleThreadExecutor(defaultThreadFactory); + } + + public Builder withCacheData(boolean cacheData) + { + this.cacheData = cacheData; + return this; + } + + public Builder withCompressed(boolean compressed) + { + this.compressed = compressed; + return this; + } + + public Builder withExecutorService(ExecutorService exec) + { + this.exec = exec; + return this; + } + + public SimplePathChildrenCacheFactory build() + { + return new SimplePathChildrenCacheFactory(cacheData, compressed, exec); + } + } } diff --git a/client/src/main/java/com/metamx/druid/initialization/CuratorConfig.java b/client/src/main/java/com/metamx/druid/initialization/CuratorConfig.java index 741bc59d3d9..be46aea41f6 100644 --- a/client/src/main/java/com/metamx/druid/initialization/CuratorConfig.java +++ b/client/src/main/java/com/metamx/druid/initialization/CuratorConfig.java @@ -33,7 +33,7 @@ public abstract class CuratorConfig @Default("30000") public abstract int getZkSessionTimeoutMs(); - @Config("druid.curator.compression.enable") + @Config("druid.curator.compress") @Default("false") public abstract boolean enableCompression(); } diff --git a/client/src/main/java/com/metamx/druid/initialization/ServiceDiscoveryConfig.java b/client/src/main/java/com/metamx/druid/initialization/ServiceDiscoveryConfig.java index 04776d6545a..7d0c20cd7ef 100644 --- a/client/src/main/java/com/metamx/druid/initialization/ServiceDiscoveryConfig.java +++ b/client/src/main/java/com/metamx/druid/initialization/ServiceDiscoveryConfig.java @@ -36,4 +36,11 @@ public abstract class ServiceDiscoveryConfig extends CuratorConfig @Config("druid.zk.paths.discoveryPath") public abstract String getDiscoveryPath(); + + @Override + @Config("druid.curator.discovery.compress") + public boolean enableCompression() + { + return false; + } } diff --git a/common/pom.xml b/common/pom.xml index d9d6249b5f9..fccf2c2eeb7 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.19-SNAPSHOT + 0.5.33-SNAPSHOT diff --git a/common/src/main/java/com/metamx/druid/partition/PartitionHolder.java b/common/src/main/java/com/metamx/druid/partition/PartitionHolder.java index 00dc3e18a43..79d1b5dd96e 100644 --- a/common/src/main/java/com/metamx/druid/partition/PartitionHolder.java +++ b/common/src/main/java/com/metamx/druid/partition/PartitionHolder.java @@ -61,6 +61,11 @@ public class PartitionHolder implements Iterable> holderSet.add(chunk); } + public int size() + { + return holderSet.size(); + } + public PartitionChunk remove(PartitionChunk chunk) { // Somewhat funky implementation in order to return the removed object as it exists in the set diff --git a/examples/pom.xml b/examples/pom.xml index 9669767f81e..968a63867bd 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.5.19-SNAPSHOT + 0.5.33-SNAPSHOT diff --git a/examples/src/test/java/druid/examples/web/InputSupplierUpdateStreamTest.java b/examples/src/test/java/druid/examples/web/InputSupplierUpdateStreamTest.java index df00f598b40..a86d239806f 100644 --- a/examples/src/test/java/druid/examples/web/InputSupplierUpdateStreamTest.java +++ b/examples/src/test/java/druid/examples/web/InputSupplierUpdateStreamTest.java @@ -68,6 +68,7 @@ public class InputSupplierUpdateStreamTest updateStream.start(); Map insertedRow = updateStream.pollFromQueue(waitTime, unit); Assert.assertEquals(expectedAnswer, insertedRow); + updateStream.stop(); } //If a timestamp is missing, we should throw away the event @@ -85,6 +86,7 @@ public class InputSupplierUpdateStreamTest ); updateStream.start(); Assert.assertEquals(updateStream.getQueueSize(), 0); + updateStream.stop(); } //If any other value is missing, we should still add the event and process it properly @@ -105,6 +107,7 @@ public class InputSupplierUpdateStreamTest expectedAnswer.put("item1", "value1"); expectedAnswer.put("time", 1372121562); Assert.assertEquals(expectedAnswer, insertedRow); + updateStream.stop(); } diff --git a/examples/src/test/java/druid/examples/web/RenamingKeysUpdateStreamTest.java b/examples/src/test/java/druid/examples/web/RenamingKeysUpdateStreamTest.java index e31b8a8e346..86ba0476c8e 100644 --- a/examples/src/test/java/druid/examples/web/RenamingKeysUpdateStreamTest.java +++ b/examples/src/test/java/druid/examples/web/RenamingKeysUpdateStreamTest.java @@ -47,7 +47,7 @@ public class RenamingKeysUpdateStreamTest } @Test - public void testPollFromQueue() throws Exception + public void testPolFromQueue() throws Exception { InputSupplierUpdateStream updateStream = new InputSupplierUpdateStream(testCaseSupplier, timeDimension); Map renamedKeys = new HashMap(); @@ -61,8 +61,6 @@ public class RenamingKeysUpdateStreamTest expectedAnswer.put("i1", "value1"); expectedAnswer.put("i2", 2); expectedAnswer.put("t", 1372121562); - - Assert.assertEquals(expectedAnswer, renamer.pollFromQueue(waitTime, unit)); } @@ -88,7 +86,6 @@ public class RenamingKeysUpdateStreamTest renamedKeys.put("item2", "i2"); RenamingKeysUpdateStream renamer = new RenamingKeysUpdateStream(updateStream, renamedKeys); Assert.assertEquals("time", renamer.getTimeDimension()); - } } diff --git a/examples/src/test/java/druid/examples/web/WebJsonSupplierTest.java b/examples/src/test/java/druid/examples/web/WebJsonSupplierTest.java index c1bc5c4dc34..5bb72e2af17 100644 --- a/examples/src/test/java/druid/examples/web/WebJsonSupplierTest.java +++ b/examples/src/test/java/druid/examples/web/WebJsonSupplierTest.java @@ -21,11 +21,11 @@ package druid.examples.web; import org.junit.Test; -import java.net.UnknownHostException; +import java.io.IOException; public class WebJsonSupplierTest { - @Test(expected = UnknownHostException.class) + @Test(expected = IOException.class) public void checkInvalidUrl() throws Exception { diff --git a/indexing-common/pom.xml b/indexing-common/pom.xml index 5c50eac9b54..31dc874c4d2 100644 --- a/indexing-common/pom.xml +++ b/indexing-common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.19-SNAPSHOT + 0.5.33-SNAPSHOT diff --git a/indexing-common/src/main/java/com/metamx/druid/index/QueryableIndex.java b/indexing-common/src/main/java/com/metamx/druid/index/QueryableIndex.java index 82cee9e54dd..8e429d45879 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/QueryableIndex.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/QueryableIndex.java @@ -23,6 +23,9 @@ import com.metamx.druid.index.column.ColumnSelector; import com.metamx.druid.kv.Indexed; import org.joda.time.Interval; +import java.io.Closeable; +import java.io.IOException; + /** */ public interface QueryableIndex extends ColumnSelector @@ -31,4 +34,11 @@ public interface QueryableIndex extends ColumnSelector public int getNumRows(); public Indexed getColumnNames(); public Indexed getAvailableDimensions(); + + /** + * The close method shouldn't actually be here as this is nasty. We will adjust it in the future. + * @throws IOException + */ + @Deprecated + public void close() throws IOException; } diff --git a/indexing-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java b/indexing-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java index 2f60b73adc6..f9898d3a607 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java @@ -19,10 +19,12 @@ package com.metamx.druid.index; +import com.metamx.common.io.smoosh.SmooshedFileMapper; import com.metamx.druid.index.column.Column; import com.metamx.druid.kv.Indexed; import org.joda.time.Interval; +import java.io.IOException; import java.util.Map; /** @@ -34,13 +36,15 @@ public class SimpleQueryableIndex implements QueryableIndex private final Indexed availableDimensions; private final Column timeColumn; private final Map otherColumns; + private final SmooshedFileMapper fileMapper; public SimpleQueryableIndex( Interval dataInterval, Indexed columnNames, Indexed dimNames, Column timeColumn, - Map otherColumns + Map otherColumns, + SmooshedFileMapper fileMapper ) { this.dataInterval = dataInterval; @@ -48,6 +52,7 @@ public class SimpleQueryableIndex implements QueryableIndex this.availableDimensions = dimNames; this.timeColumn = timeColumn; this.otherColumns = otherColumns; + this.fileMapper = fileMapper; } @Override @@ -85,4 +90,10 @@ public class SimpleQueryableIndex implements QueryableIndex { return otherColumns.get(columnName); } + + @Override + public void close() throws IOException + { + fileMapper.close(); + } } diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java index 24f6750785f..07008e0a42b 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java @@ -375,7 +375,8 @@ public class IndexIO dimValueLookups, dimColumns, invertedIndexed, - spatialIndexed + spatialIndexed, + smooshedFiles ); log.debug("Mapped v8 index[%s] in %,d millis", inDir, System.currentTimeMillis() - startTime); @@ -761,7 +762,8 @@ public class IndexIO .setType(ValueType.LONG) .setGenericColumn(new LongGenericColumnSupplier(index.timestamps)) .build(), - columns + columns, + index.getFileMapper() ); } } @@ -795,7 +797,7 @@ public class IndexIO } final QueryableIndex index = new SimpleQueryableIndex( - dataInterval, cols, dims, deserializeColumn(mapper, smooshedFiles.mapFile("__time")), columns + dataInterval, cols, dims, deserializeColumn(mapper, smooshedFiles.mapFile("__time")), columns, smooshedFiles ); log.debug("Mapped v9 index[%s] in %,d millis", inDir, System.currentTimeMillis() - startTime); diff --git a/indexing-common/src/main/java/com/metamx/druid/index/v1/MMappedIndex.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/MMappedIndex.java index fcdcef4eddc..d792d80de01 100644 --- a/indexing-common/src/main/java/com/metamx/druid/index/v1/MMappedIndex.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/MMappedIndex.java @@ -24,18 +24,19 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import com.google.common.primitives.Ints; import com.metamx.collections.spatial.ImmutableRTree; +import com.metamx.common.io.smoosh.SmooshedFileMapper; import com.metamx.common.logger.Logger; import com.metamx.druid.kv.ConciseCompressedIndexedInts; import com.metamx.druid.kv.GenericIndexed; import com.metamx.druid.kv.Indexed; import com.metamx.druid.kv.IndexedList; import com.metamx.druid.kv.IndexedLongs; -import com.metamx.druid.kv.IndexedRTree; import com.metamx.druid.kv.VSizeIndexed; import com.metamx.druid.kv.VSizeIndexedInts; import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet; import org.joda.time.Interval; +import java.io.IOException; import java.nio.ByteOrder; import java.nio.LongBuffer; import java.util.Arrays; @@ -57,6 +58,7 @@ public class MMappedIndex final Map dimColumns; final Map> invertedIndexes; final Map spatialIndexes; + final SmooshedFileMapper fileMapper; private final Map metricIndexes = Maps.newHashMap(); @@ -69,7 +71,8 @@ public class MMappedIndex Map> dimValueLookups, Map dimColumns, Map> invertedIndexes, - Map spatialIndexes + Map spatialIndexes, + SmooshedFileMapper fileMapper ) { this.availableDimensions = availableDimensions; @@ -81,6 +84,7 @@ public class MMappedIndex this.dimColumns = dimColumns; this.invertedIndexes = invertedIndexes; this.spatialIndexes = spatialIndexes; + this.fileMapper = fileMapper; for (int i = 0; i < availableMetrics.size(); i++) { metricIndexes.put(availableMetrics.get(i), i); @@ -169,6 +173,18 @@ public class MMappedIndex return (retVal == null) ? emptySet : retVal; } + public SmooshedFileMapper getFileMapper() + { + return fileMapper; + } + + public void close() throws IOException + { + if (fileMapper != null) { + fileMapper.close(); + } + } + public static MMappedIndex fromIndex(Index index) { log.info("Converting timestamps"); @@ -273,7 +289,8 @@ public class MMappedIndex dimValueLookups, dimColumns, invertedIndexes, - spatialIndexes + spatialIndexes, + null ); } } diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java index 82a3c947bc3..231cbd44102 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java @@ -1,9 +1,10 @@ package com.metamx.druid.indexer.data; +import com.metamx.common.exception.FormattedException; import com.metamx.druid.input.InputRow; public interface InputRowParser { - public InputRow parse(T input); + public InputRow parse(T input) throws FormattedException; public void addDimensionExclusion(String dimension); } diff --git a/indexing-common/src/main/java/com/metamx/druid/indexer/data/MapInputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/MapInputRowParser.java index 60a97c131bf..b2d9586f272 100644 --- a/indexing-common/src/main/java/com/metamx/druid/indexer/data/MapInputRowParser.java +++ b/indexing-common/src/main/java/com/metamx/druid/indexer/data/MapInputRowParser.java @@ -4,6 +4,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Lists; import com.google.common.collect.Sets; +import com.metamx.common.exception.FormattedException; import com.metamx.druid.input.InputRow; import com.metamx.druid.input.MapBasedInputRow; import org.joda.time.DateTime; @@ -37,21 +38,30 @@ public class MapInputRowParser implements InputRowParser> } @Override - public InputRow parse(Map theMap) + public InputRow parse(Map theMap) throws FormattedException { final List dimensions = dataSpec.hasCustomDimensions() ? dataSpec.getDimensions() : Lists.newArrayList(Sets.difference(theMap.keySet(), dimensionExclusions)); - final DateTime timestamp = timestampSpec.extractTimestamp(theMap); - if (timestamp == null) { - final String input = theMap.toString(); - throw new NullPointerException( - String.format( - "Null timestamp in input: %s", - input.length() < 100 ? input : input.substring(0, 100) + "..." - ) - ); + final DateTime timestamp; + try { + timestamp = timestampSpec.extractTimestamp(theMap); + if (timestamp == null) { + final String input = theMap.toString(); + throw new NullPointerException( + String.format( + "Null timestamp in input: %s", + input.length() < 100 ? input : input.substring(0, 100) + "..." + ) + ); + } + } + catch (Exception e) { + throw new FormattedException.Builder() + .withErrorCode(FormattedException.ErrorCode.UNPARSABLE_TIMESTAMP) + .withMessage(e.toString()) + .build(); } return new MapBasedInputRow(timestamp.getMillis(), dimensions, theMap); diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 803c3bbb571..e5b5c3f6d30 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.19-SNAPSHOT + 0.5.33-SNAPSHOT diff --git a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java index 6d0ea322313..1c9e29801ee 100644 --- a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java @@ -57,9 +57,10 @@ import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.shard.ShardSpec; import com.metamx.druid.utils.JodaUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.mapreduce.Job; - import org.joda.time.DateTime; import org.joda.time.Interval; import org.joda.time.format.ISODateTimeFormat; @@ -656,22 +657,34 @@ public class HadoopDruidIndexerConfig return new Path(makeDescriptorInfoDir(), String.format("%s.json", segment.getIdentifier().replace(":", ""))); } - public Path makeSegmentOutputPath(Bucket bucket) - { - final Interval bucketInterval = getGranularitySpec().bucketInterval(bucket.time).get(); - - return new Path( - String.format( - "%s/%s/%s_%s/%s/%s", - getSegmentOutputDir(), - dataSource, - bucketInterval.getStart().toString(), - bucketInterval.getEnd().toString(), - getVersion(), - bucket.partitionNum - ) - ); - } + public Path makeSegmentOutputPath(FileSystem fileSystem, Bucket bucket) + { + final Interval bucketInterval = getGranularitySpec().bucketInterval(bucket.time).get(); + if (fileSystem instanceof DistributedFileSystem) + { + return new Path( + String.format( + "%s/%s/%s_%s/%s/%s", + getSegmentOutputDir(), + dataSource, + bucketInterval.getStart().toString(ISODateTimeFormat.basicDateTime()), + bucketInterval.getEnd().toString(ISODateTimeFormat.basicDateTime()), + getVersion().replace(":", "_"), + bucket.partitionNum + ) + ); + } + return new Path( + String.format( + "%s/%s/%s_%s/%s/%s", + getSegmentOutputDir(), + dataSource, + bucketInterval.getStart().toString(), + bucketInterval.getEnd().toString(), + getVersion(), + bucket.partitionNum + )); + } public Job addInputPaths(Job job) throws IOException { diff --git a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java index 45cbded7f1a..583cafcae4f 100644 --- a/indexing-hadoop/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java @@ -375,12 +375,14 @@ public class IndexGeneratorJob implements Jobby Interval interval = config.getGranularitySpec().bucketInterval(bucket.time).get(); int attemptNumber = context.getTaskAttemptID().getId(); - Path indexBasePath = config.makeSegmentOutputPath(bucket); - Path indexZipFilePath = new Path(indexBasePath, String.format("index.zip.%s", attemptNumber)); - final FileSystem infoFS = config.makeDescriptorInfoDir().getFileSystem(context.getConfiguration()); - final FileSystem outputFS = indexBasePath.getFileSystem(context.getConfiguration()); - outputFS.mkdirs(indexBasePath); + FileSystem fileSystem = FileSystem.get(context.getConfiguration()); + Path indexBasePath = config.makeSegmentOutputPath(fileSystem, bucket); + Path indexZipFilePath = new Path(indexBasePath, String.format("index.zip.%s", attemptNumber)); + final FileSystem infoFS = config.makeDescriptorInfoDir().getFileSystem(context.getConfiguration()); + final FileSystem outputFS = indexBasePath.getFileSystem(context.getConfiguration()); + + outputFS.mkdirs(indexBasePath); Exception caughtException = null; ZipOutputStream out = null; diff --git a/indexing-hadoop/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java index dcabe168e67..687ec6bb084 100644 --- a/indexing-hadoop/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexing-hadoop/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -26,7 +26,10 @@ import com.metamx.druid.indexer.granularity.UniformGranularitySpec; import com.metamx.druid.indexer.partitions.PartitionsSpec; import com.metamx.druid.indexer.updater.DbUpdaterJobSpec; import com.metamx.druid.jackson.DefaultObjectMapper; - +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; @@ -427,6 +430,67 @@ public class HadoopDruidIndexerConfigTest ); } + + @Test + public void shouldMakeHDFSCompliantSegmentOutputPath() { + final HadoopDruidIndexerConfig cfg; + + try { + cfg = jsonReadWriteRead( + "{" + + "\"dataSource\": \"source\"," + + " \"granularitySpec\":{" + + " \"type\":\"uniform\"," + + " \"gran\":\"hour\"," + + " \"intervals\":[\"2012-07-10/P1D\"]" + + " }," + + "\"segmentOutputPath\": \"hdfs://server:9100/tmp/druid/datatest\"" + + "}", + HadoopDruidIndexerConfig.class + ); + } catch(Exception e) { + throw Throwables.propagate(e); + } + + cfg.setVersion("some:brand:new:version"); + + Bucket bucket = new Bucket(4711, new DateTime(2012, 07, 10, 5, 30), 4712); + Path path = cfg.makeSegmentOutputPath(new DistributedFileSystem(), bucket); + Assert.assertEquals( + "hdfs://server:9100/tmp/druid/datatest/source/20120710T050000.000Z_20120710T060000.000Z/some_brand_new_version/4712", + path.toString() + ); + } + + @Test + public void shouldMakeDefaultSegmentOutputPathIfNotHDFS() { + final HadoopDruidIndexerConfig cfg; + + try { + cfg = jsonReadWriteRead( + "{" + + "\"dataSource\": \"the:data:source\"," + + " \"granularitySpec\":{" + + " \"type\":\"uniform\"," + + " \"gran\":\"hour\"," + + " \"intervals\":[\"2012-07-10/P1D\"]" + + " }," + + "\"segmentOutputPath\": \"/tmp/dru:id/data:test\"" + + "}", + HadoopDruidIndexerConfig.class + ); + } catch(Exception e) { + throw Throwables.propagate(e); + } + + cfg.setVersion("some:brand:new:version"); + + Bucket bucket = new Bucket(4711, new DateTime(2012, 07, 10, 5, 30), 4712); + Path path = cfg.makeSegmentOutputPath(new LocalFileSystem(), bucket); + Assert.assertEquals("/tmp/dru:id/data:test/the:data:source/2012-07-10T05:00:00.000Z_2012-07-10T06:00:00.000Z/some:brand:new:version/4712", path.toString()); + + } + private T jsonReadWriteRead(String s, Class klass) { try { diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 64d4a2aa129..7f589f982d4 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.19-SNAPSHOT + 0.5.33-SNAPSHOT diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskStatus.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskStatus.java index 8a798d2a961..d83b7d0de40 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskStatus.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskStatus.java @@ -24,6 +24,7 @@ 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.metamx.druid.indexing.common.task.TaskResource; /** * Represents the status of a task. The task may be ongoing ({@link #isComplete()} false) or it may be @@ -42,33 +43,36 @@ public class TaskStatus public static TaskStatus running(String taskId) { - return new TaskStatus(taskId, Status.RUNNING, -1); + return new TaskStatus(taskId, Status.RUNNING, -1, null); } public static TaskStatus success(String taskId) { - return new TaskStatus(taskId, Status.SUCCESS, -1); + return new TaskStatus(taskId, Status.SUCCESS, -1, null); } public static TaskStatus failure(String taskId) { - return new TaskStatus(taskId, Status.FAILED, -1); + return new TaskStatus(taskId, Status.FAILED, -1, null); } private final String id; private final Status status; private final long duration; + private final TaskResource resource; @JsonCreator private TaskStatus( @JsonProperty("id") String id, @JsonProperty("status") Status status, - @JsonProperty("duration") long duration + @JsonProperty("duration") long duration, + @JsonProperty("resource") TaskResource resource ) { this.id = id; this.status = status; this.duration = duration; + this.resource = resource == null ? new TaskResource(id, 1) : resource; // Check class invariants. Preconditions.checkNotNull(id, "id"); @@ -93,6 +97,12 @@ public class TaskStatus return duration; } + @JsonProperty("resource") + public TaskResource getResource() + { + return resource; + } + /** * Signals that a task is not yet complete, and is still runnable on a worker. Exactly one of isRunnable, * isSuccess, or isFailure will be true at any one time. @@ -134,7 +144,7 @@ public class TaskStatus public TaskStatus withDuration(long _duration) { - return new TaskStatus(id, status, _duration); + return new TaskStatus(id, status, _duration, resource); } @Override @@ -144,6 +154,7 @@ public class TaskStatus .add("id", id) .add("status", status) .add("duration", duration) + .add("resource", resource) .toString(); } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolbox.java index 5bbfd73abbe..050a23cb5e0 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolbox.java @@ -37,6 +37,7 @@ import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.loading.SingleSegmentLoader; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.metrics.MonitorScheduler; import org.jets3t.service.impl.rest.httpclient.RestS3Service; import java.io.File; @@ -58,6 +59,7 @@ public class TaskToolbox private final DataSegmentAnnouncer segmentAnnouncer; private final ServerView newSegmentServerView; private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate; + private final MonitorScheduler monitorScheduler; private final ObjectMapper objectMapper; public TaskToolbox( @@ -71,6 +73,7 @@ public class TaskToolbox DataSegmentAnnouncer segmentAnnouncer, ServerView newSegmentServerView, QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate, + MonitorScheduler monitorScheduler, ObjectMapper objectMapper ) { @@ -84,6 +87,7 @@ public class TaskToolbox this.segmentAnnouncer = segmentAnnouncer; this.newSegmentServerView = newSegmentServerView; this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate; + this.monitorScheduler = monitorScheduler; this.objectMapper = objectMapper; } @@ -127,6 +131,11 @@ public class TaskToolbox return queryRunnerFactoryConglomerate; } + public MonitorScheduler getMonitorScheduler() + { + return monitorScheduler; + } + public ObjectMapper getObjectMapper() { return objectMapper; @@ -156,7 +165,8 @@ public class TaskToolbox return retVal; } - public File getTaskWorkDir() { + public File getTaskWorkDir() + { return new File(new File(config.getBaseTaskDir(), task.getId()), "work"); } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolboxFactory.java index 1d8c244ef7d..91f5c99a333 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolboxFactory.java @@ -29,6 +29,7 @@ import com.metamx.druid.indexing.common.config.TaskConfig; import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.metrics.MonitorScheduler; import org.jets3t.service.impl.rest.httpclient.RestS3Service; /** @@ -45,6 +46,7 @@ public class TaskToolboxFactory private final DataSegmentAnnouncer segmentAnnouncer; private final ServerView newSegmentServerView; private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate; + private final MonitorScheduler monitorScheduler; private final ObjectMapper objectMapper; public TaskToolboxFactory( @@ -57,6 +59,7 @@ public class TaskToolboxFactory DataSegmentAnnouncer segmentAnnouncer, ServerView newSegmentServerView, QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate, + MonitorScheduler monitorScheduler, ObjectMapper objectMapper ) { @@ -69,6 +72,7 @@ public class TaskToolboxFactory this.segmentAnnouncer = segmentAnnouncer; this.newSegmentServerView = newSegmentServerView; this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate; + this.monitorScheduler = monitorScheduler; this.objectMapper = objectMapper; } @@ -85,6 +89,7 @@ public class TaskToolboxFactory segmentAnnouncer, newSegmentServerView, queryRunnerFactoryConglomerate, + monitorScheduler, objectMapper ); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/RemoteTaskActionClient.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/RemoteTaskActionClient.java index b9694ad3fba..2b698f1b234 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/RemoteTaskActionClient.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/RemoteTaskActionClient.java @@ -65,6 +65,8 @@ public class RemoteTaskActionClient implements TaskActionClient final String response; + log.info("Submitting action for task[%s] to coordinator[%s]: %s", task.getId(), serviceUri, taskAction); + try { response = httpClient.post(serviceUri.toURL()) .setContent("application/json", dataToSend) @@ -107,7 +109,7 @@ public class RemoteTaskActionClient implements TaskActionClient final String scheme; final String host; final int port; - final String path = "/mmx/merger/v1/action"; + final String path = "/druid/indexer/v1/action"; if (instance == null) { throw new ISE("Cannot find instance of indexer to talk to!"); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskConfig.java index 9de8bcc9302..5e9789e9660 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskConfig.java @@ -19,6 +19,7 @@ package com.metamx.druid.indexing.common.config; +import com.google.common.base.Joiner; import org.skife.config.Config; import org.skife.config.Default; @@ -26,13 +27,30 @@ import java.io.File; public abstract class TaskConfig { + private static Joiner joiner = Joiner.on("/"); + + @Config("druid.indexer.baseDir") + @Default("/tmp/") + public abstract String getBaseDir(); + @Config("druid.indexer.taskDir") - public abstract File getBaseTaskDir(); + public File getBaseTaskDir() + { + return new File(defaultPath("persistent/task")); + } + + @Config("druid.indexer.hadoopWorkingPath") + public String getHadoopWorkingPath() + { + return defaultPath("druid-indexing"); + } @Config("druid.indexer.rowFlushBoundary") @Default("500000") public abstract int getDefaultRowFlushBoundary(); - @Config("druid.indexer.hadoopWorkingPath") - public abstract String getHadoopWorkingPath(); -} + private String defaultPath(String subPath) + { + return joiner.join(getBaseDir(), subPath); + } +} \ No newline at end of file diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskLogConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskLogConfig.java index a80fdcd76c8..59030c99807 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskLogConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskLogConfig.java @@ -1,10 +1,15 @@ package com.metamx.druid.indexing.common.config; import org.skife.config.Config; +import org.skife.config.Default; import org.skife.config.DefaultNull; public abstract class TaskLogConfig { + @Config("druid.indexer.logs.type") + @Default("noop") + public abstract String getLogType(); + @Config("druid.indexer.logs.s3bucket") @DefaultNull public abstract String getLogStorageBucket(); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandler.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandler.java index 3458c931de4..95c66028e52 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandler.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandler.java @@ -1,7 +1,26 @@ +/* + * 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.indexing.common.index; /** - * Objects that can be registered with a {@link ChatHandlerProvider} and provide http endpoints for indexing-related + * Objects that can be registered with a {@link EventReceivingChatHandlerProvider} and provide http endpoints for indexing-related * objects. This interface is empty because it only exists to signal intent. The actual http endpoints are provided * through JAX-RS annotations on the {@link ChatHandler} objects. */ diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandlerProvider.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandlerProvider.java index 6d67a7a5fb1..1662abc7d9b 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandlerProvider.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandlerProvider.java @@ -1,83 +1,33 @@ +/* + * 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.indexing.common.index; import com.google.common.base.Optional; -import com.google.common.collect.Maps; -import com.metamx.common.ISE; -import com.metamx.common.logger.Logger; -import com.metamx.druid.curator.discovery.ServiceAnnouncer; -import com.metamx.druid.indexing.worker.config.ChatHandlerProviderConfig; - -import java.util.concurrent.ConcurrentMap; /** - * Provides a way for the outside world to talk to objects in the indexing service. The {@link #get(String)} method - * allows anyone with a reference to this object to obtain a particular {@link ChatHandler}. An embedded - * {@link ServiceAnnouncer} will be used to advertise handlers on this host. */ -public class ChatHandlerProvider +public interface ChatHandlerProvider { - private static final Logger log = new Logger(ChatHandlerProvider.class); + public void register(final String key, ChatHandler handler); - private final ChatHandlerProviderConfig config; - private final ServiceAnnouncer serviceAnnouncer; - private final ConcurrentMap handlers; + public void unregister(final String key); - public ChatHandlerProvider( - ChatHandlerProviderConfig config, - ServiceAnnouncer serviceAnnouncer - ) - { - this.config = config; - this.serviceAnnouncer = serviceAnnouncer; - this.handlers = Maps.newConcurrentMap(); - } - - public void register(final String key, ChatHandler handler) - { - final String service = serviceName(key); - log.info("Registering Eventhandler: %s", key); - - if (handlers.putIfAbsent(key, handler) != null) { - throw new ISE("handler already registered for key: %s", key); - } - - try { - serviceAnnouncer.announce(service); - } - catch (Exception e) { - log.warn(e, "Failed to register service: %s", service); - handlers.remove(key, handler); - } - } - - public void unregister(final String key) - { - final String service = serviceName(key); - - log.info("Unregistering chat handler: %s", key); - - final ChatHandler handler = handlers.get(key); - if (handler == null) { - log.warn("handler not currently registered, ignoring: %s", key); - } - - try { - serviceAnnouncer.unannounce(service); - } - catch (Exception e) { - log.warn(e, "Failed to unregister service: %s", service); - } - - handlers.remove(key, handler); - } - - public Optional get(final String key) - { - return Optional.fromNullable(handlers.get(key)); - } - - private String serviceName(String key) - { - return String.format(config.getServiceFormat(), key); - } + public Optional get(final String key); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceiverFirehoseFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceiverFirehoseFactory.java index 5c437eaa356..1c235cf43c9 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceiverFirehoseFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceiverFirehoseFactory.java @@ -1,3 +1,22 @@ +/* + * Druid - a distributed column store. + * Copyright (C) 2012 Metamarkets Group Inc. + * + * This program is free software; you can redistribute it and/or + * modify it under the terms of the GNU General Public License + * as published by the Free Software Foundation; either version 2 + * of the License, or (at your option) any later version. + * + * This program is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * GNU General Public License for more details. + * + * You should have received a copy of the GNU General Public License + * along with this program; if not, write to the Free Software + * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. + */ + package com.metamx.druid.indexing.common.index; import com.fasterxml.jackson.annotation.JacksonInject; @@ -30,7 +49,7 @@ import java.util.concurrent.TimeUnit; /** * Builds firehoses that accept events through the {@link EventReceiver} interface. Can also register these - * firehoses with an {@link ChatHandlerProvider}. + * firehoses with an {@link EventReceivingChatHandlerProvider}. */ @JsonTypeName("receiver") public class EventReceiverFirehoseFactory implements FirehoseFactory @@ -41,14 +60,14 @@ public class EventReceiverFirehoseFactory implements FirehoseFactory private final String firehoseId; private final int bufferSize; private final MapInputRowParser parser; - private final Optional chatHandlerProvider; + private final Optional chatHandlerProvider; @JsonCreator public EventReceiverFirehoseFactory( @JsonProperty("firehoseId") String firehoseId, @JsonProperty("bufferSize") Integer bufferSize, @JsonProperty("parser") MapInputRowParser parser, - @JacksonInject("chatHandlerProvider") ChatHandlerProvider chatHandlerProvider + @JacksonInject("chatHandlerProvider") EventReceivingChatHandlerProvider chatHandlerProvider ) { this.firehoseId = Preconditions.checkNotNull(firehoseId, "firehoseId"); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceivingChatHandlerProvider.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceivingChatHandlerProvider.java new file mode 100644 index 00000000000..f18a79c241c --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceivingChatHandlerProvider.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.indexing.common.index; + +import com.google.common.base.Optional; +import com.google.common.collect.Maps; +import com.metamx.common.ISE; +import com.metamx.common.logger.Logger; +import com.metamx.druid.curator.discovery.ServiceAnnouncer; +import com.metamx.druid.indexing.worker.config.ChatHandlerProviderConfig; + +import java.util.concurrent.ConcurrentMap; + +/** + * Provides a way for the outside world to talk to objects in the indexing service. The {@link #get(String)} method + * allows anyone with a reference to this object to obtain a particular {@link ChatHandler}. An embedded + * {@link ServiceAnnouncer} will be used to advertise handlers on this host. + */ +public class EventReceivingChatHandlerProvider implements ChatHandlerProvider +{ + private static final Logger log = new Logger(EventReceivingChatHandlerProvider.class); + + private final ChatHandlerProviderConfig config; + private final ServiceAnnouncer serviceAnnouncer; + private final ConcurrentMap handlers; + + public EventReceivingChatHandlerProvider( + ChatHandlerProviderConfig config, + ServiceAnnouncer serviceAnnouncer + ) + { + this.config = config; + this.serviceAnnouncer = serviceAnnouncer; + this.handlers = Maps.newConcurrentMap(); + } + + @Override + public void register(final String key, ChatHandler handler) + { + final String service = serviceName(key); + log.info("Registering Eventhandler: %s", key); + + if (handlers.putIfAbsent(key, handler) != null) { + throw new ISE("handler already registered for key: %s", key); + } + + try { + serviceAnnouncer.announce(service); + } + catch (Exception e) { + log.warn(e, "Failed to register service: %s", service); + handlers.remove(key, handler); + } + } + + @Override + public void unregister(final String key) + { + final String service = serviceName(key); + + log.info("Unregistering chat handler: %s", key); + + final ChatHandler handler = handlers.get(key); + if (handler == null) { + log.warn("handler not currently registered, ignoring: %s", key); + } + + try { + serviceAnnouncer.unannounce(service); + } + catch (Exception e) { + log.warn(e, "Failed to unregister service: %s", service); + } + + handlers.remove(key, handler); + } + + @Override + public Optional get(final String key) + { + return Optional.fromNullable(handlers.get(key)); + } + + private String serviceName(String key) + { + return String.format(config.getServiceFormat(), key); + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/NoopChatHandlerProvider.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/NoopChatHandlerProvider.java new file mode 100644 index 00000000000..bf9a36afdb5 --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/NoopChatHandlerProvider.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.indexing.common.index; + +import com.google.common.base.Optional; + +/** + */ +public class NoopChatHandlerProvider implements ChatHandlerProvider +{ + @Override + public void register(String key, ChatHandler handler) + { + // do nothing + } + + @Override + public void unregister(String key) + { + // do nothing + } + + @Override + public Optional get(String key) + { + return Optional.absent(); + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/AbstractTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/AbstractTask.java index 07d11582982..e263c1288b1 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/AbstractTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/AbstractTask.java @@ -43,7 +43,7 @@ public abstract class AbstractTask implements Task private final String groupId; @JsonIgnore - private final String availabilityGroup; + private final TaskResource taskResource; @JsonIgnore private final String dataSource; @@ -53,23 +53,23 @@ public abstract class AbstractTask implements Task protected AbstractTask(String id, String dataSource, Interval interval) { - this(id, id, id, dataSource, interval); + this(id, id, new TaskResource(id, 1), dataSource, interval); } protected AbstractTask(String id, String groupId, String dataSource, Interval interval) { this.id = Preconditions.checkNotNull(id, "id"); this.groupId = Preconditions.checkNotNull(groupId, "groupId"); - this.availabilityGroup = id; + this.taskResource = new TaskResource(id, 1); this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); this.interval = Optional.fromNullable(interval); } - protected AbstractTask(String id, String groupId, String availabilityGroup, String dataSource, Interval interval) + protected AbstractTask(String id, String groupId, TaskResource taskResource, String dataSource, Interval interval) { this.id = Preconditions.checkNotNull(id, "id"); this.groupId = Preconditions.checkNotNull(groupId, "groupId"); - this.availabilityGroup = Preconditions.checkNotNull(availabilityGroup, "availabilityGroup"); + this.taskResource = Preconditions.checkNotNull(taskResource, "taskResource"); this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); this.interval = Optional.fromNullable(interval); } @@ -90,9 +90,9 @@ public abstract class AbstractTask implements Task @JsonProperty @Override - public String getAvailabilityGroup() + public TaskResource getTaskResource() { - return availabilityGroup; + return taskResource; } @Override @@ -127,12 +127,6 @@ public abstract class AbstractTask implements Task return TaskStatus.running(id); } - @Override - public void shutdown() - { - // Do nothing. - } - @Override public String toString() { @@ -172,19 +166,16 @@ public abstract class AbstractTask implements Task AbstractTask that = (AbstractTask) o; - if (dataSource != null ? !dataSource.equals(that.dataSource) : that.dataSource != null) { - return false; - } - if (groupId != null ? !groupId.equals(that.groupId) : that.groupId != null) { - return false; - } - if (id != null ? !id.equals(that.id) : that.id != null) { - return false; - } - if (interval != null ? !interval.equals(that.interval) : that.interval != null) { + if (!id.equals(that.id)) { return false; } return true; } + + @Override + public int hashCode() + { + return id.hashCode(); + } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexDeterminePartitionsTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexDeterminePartitionsTask.java index 7c7d8707a37..8388c9653e3 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexDeterminePartitionsTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexDeterminePartitionsTask.java @@ -88,7 +88,6 @@ public class IndexDeterminePartitionsTask extends AbstractTask super( id != null ? id : makeTaskId(groupId, interval.getStart(), interval.getEnd()), groupId, - makeTaskId(groupId, interval.getStart(), interval.getEnd()), schema.getDataSource(), Preconditions.checkNotNull(interval, "interval") ); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java index d5dd40472f4..dc3ad87a9cb 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.io.Closeables; import com.metamx.common.exception.FormattedException; @@ -41,15 +42,18 @@ import com.metamx.druid.input.InputRow; import com.metamx.druid.query.FinalizeResultsQueryRunner; 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.realtime.FireDepartment; import com.metamx.druid.realtime.FireDepartmentConfig; -import com.metamx.druid.realtime.FireDepartmentMetrics; +import com.metamx.druid.realtime.RealtimeMetricsMonitor; import com.metamx.druid.realtime.Schema; import com.metamx.druid.realtime.SegmentPublisher; +import com.metamx.druid.realtime.firehose.Firehose; import com.metamx.druid.realtime.firehose.FirehoseFactory; -import com.metamx.druid.realtime.firehose.GracefulShutdownFirehose; import com.metamx.druid.realtime.plumber.Plumber; import com.metamx.druid.realtime.plumber.RealtimePlumberSchool; +import com.metamx.druid.realtime.plumber.RejectionPolicyFactory; import com.metamx.druid.realtime.plumber.Sink; import com.metamx.druid.realtime.plumber.VersioningPolicy; import com.metamx.emitter.EmittingLogger; @@ -87,43 +91,44 @@ public class RealtimeIndexTask extends AbstractTask @JsonIgnore private final IndexGranularity segmentGranularity; + @JsonIgnore + private final RejectionPolicyFactory rejectionPolicyFactory; + @JsonIgnore private volatile Plumber plumber = null; @JsonIgnore - private volatile TaskToolbox toolbox = null; - - @JsonIgnore - private volatile GracefulShutdownFirehose firehose = null; - - @JsonIgnore - private final Object lock = new Object(); - - @JsonIgnore - private volatile boolean shutdown = false; + private volatile QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate = null; @JsonCreator public RealtimeIndexTask( @JsonProperty("id") String id, - @JsonProperty("availabilityGroup") String availabilityGroup, + @JsonProperty("resource") TaskResource taskResource, @JsonProperty("schema") Schema schema, @JsonProperty("firehose") FirehoseFactory firehoseFactory, @JsonProperty("fireDepartmentConfig") FireDepartmentConfig fireDepartmentConfig, @JsonProperty("windowPeriod") Period windowPeriod, - @JsonProperty("segmentGranularity") IndexGranularity segmentGranularity + @JsonProperty("segmentGranularity") IndexGranularity segmentGranularity, + @JsonProperty("rejectionPolicy") RejectionPolicyFactory rejectionPolicyFactory ) { super( - id != null - ? id - : makeTaskId(schema.getDataSource(), schema.getShardSpec().getPartitionNum(), new DateTime().toString()), + id == null + ? makeTaskId(schema.getDataSource(), schema.getShardSpec().getPartitionNum(), new DateTime().toString()) + :id, String.format( "index_realtime_%s", schema.getDataSource() ), - availabilityGroup != null - ? availabilityGroup - : makeTaskId(schema.getDataSource(), schema.getShardSpec().getPartitionNum(), new DateTime().toString()), + taskResource == null + ? new TaskResource( + makeTaskId( + schema.getDataSource(), + schema.getShardSpec().getPartitionNum(), + new DateTime().toString() + ), 1 + ) + : taskResource, schema.getDataSource(), null ); @@ -133,6 +138,7 @@ public class RealtimeIndexTask extends AbstractTask this.fireDepartmentConfig = fireDepartmentConfig; this.windowPeriod = windowPeriod; this.segmentGranularity = segmentGranularity; + this.rejectionPolicyFactory = rejectionPolicyFactory; } @Override @@ -151,7 +157,7 @@ public class RealtimeIndexTask extends AbstractTask public QueryRunner getQueryRunner(Query query) { if (plumber != null) { - QueryRunnerFactory> factory = toolbox.getQueryRunnerFactoryConglomerate().findFactory(query); + QueryRunnerFactory> factory = queryRunnerFactoryConglomerate.findFactory(query); QueryToolChest> toolChest = factory.getToolchest(); return new FinalizeResultsQueryRunner(plumber.getQueryRunner(query), toolChest); @@ -175,21 +181,9 @@ public class RealtimeIndexTask extends AbstractTask boolean normalExit = true; - final FireDepartmentMetrics metrics = new FireDepartmentMetrics(); + // Set up firehose final Period intermediatePersistPeriod = fireDepartmentConfig.getIntermediatePersistPeriod(); - - synchronized (lock) { - if (shutdown) { - return TaskStatus.success(getId()); - } - - log.info( - "Wrapping firehose in GracefulShutdownFirehose with segmentGranularity[%s] and windowPeriod[%s]", - segmentGranularity, - windowPeriod - ); - firehose = new GracefulShutdownFirehose(firehoseFactory.connect(), segmentGranularity, windowPeriod); - } + final Firehose firehose = firehoseFactory.connect(); // It would be nice to get the PlumberSchool in the constructor. Although that will need jackson injectables for // stuff like the ServerView, which seems kind of odd? Perhaps revisit this when Guice has been introduced. @@ -286,12 +280,22 @@ public class RealtimeIndexTask extends AbstractTask realtimePlumberSchool.setServerView(toolbox.getNewSegmentServerView()); realtimePlumberSchool.setServiceEmitter(toolbox.getEmitter()); - this.toolbox = toolbox; - this.plumber = realtimePlumberSchool.findPlumber(schema, metrics); + if (this.rejectionPolicyFactory != null) { + realtimePlumberSchool.setRejectionPolicyFactory(rejectionPolicyFactory); + } + + final FireDepartment fireDepartment = new FireDepartment(schema, fireDepartmentConfig, null, null); + final RealtimeMetricsMonitor metricsMonitor = new RealtimeMetricsMonitor(ImmutableList.of(fireDepartment)); + this.queryRunnerFactoryConglomerate = toolbox.getQueryRunnerFactoryConglomerate(); + this.plumber = realtimePlumberSchool.findPlumber(schema, fireDepartment.getMetrics()); try { plumber.startJob(); + // Set up metrics emission + toolbox.getMonitorScheduler().addMonitor(metricsMonitor); + + // Time to read data! long nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis(); while (firehose.hasMore()) { final InputRow inputRow; @@ -303,7 +307,7 @@ public class RealtimeIndexTask extends AbstractTask final Sink sink = plumber.getSink(inputRow.getTimestampFromEpoch()); if (sink == null) { - metrics.incrementThrownAway(); + fireDepartment.getMetrics().incrementThrownAway(); log.debug("Throwing away event[%s]", inputRow); if (System.currentTimeMillis() > nextFlush) { @@ -319,7 +323,7 @@ public class RealtimeIndexTask extends AbstractTask } int currCount = sink.add(inputRow); - metrics.incrementProcessed(); + fireDepartment.getMetrics().incrementProcessed(); if (currCount >= fireDepartmentConfig.getMaxRowsInMemory() || System.currentTimeMillis() > nextFlush) { plumber.persist(firehose.commit()); nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis(); @@ -327,7 +331,7 @@ public class RealtimeIndexTask extends AbstractTask } catch (FormattedException e) { log.warn(e, "unparseable line"); - metrics.incrementUnparseable(); + fireDepartment.getMetrics().incrementUnparseable(); } } } @@ -348,6 +352,7 @@ public class RealtimeIndexTask extends AbstractTask } finally { Closeables.closeQuietly(firehose); + toolbox.getMonitorScheduler().removeMonitor(metricsMonitor); } } } @@ -355,22 +360,6 @@ public class RealtimeIndexTask extends AbstractTask return TaskStatus.success(getId()); } - @Override - public void shutdown() - { - try { - synchronized (lock) { - shutdown = true; - if (firehose != null) { - firehose.shutdown(); - } - } - } - catch (IOException e) { - throw Throwables.propagate(e); - } - } - @JsonProperty public Schema getSchema() { @@ -401,6 +390,12 @@ public class RealtimeIndexTask extends AbstractTask return segmentGranularity; } + @JsonProperty("rejectionPolicy") + public RejectionPolicyFactory getRejectionPolicyFactory() + { + return rejectionPolicyFactory; + } + public static class TaskActionSegmentPublisher implements SegmentPublisher { final Task task; diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java index 6272669cda5..25f8b6425cd 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java @@ -72,11 +72,10 @@ public interface Task public String getGroupId(); /** - * Returns availability group ID of this task. Tasks the same availability group cannot be assigned to the same - * worker. If tasks do not have this restriction, a common convention is to set the availability group ID to the - * task ID. + * Returns a {@link com.metamx.druid.indexing.common.task.TaskResource} for this task. Task resources define specific + * worker requirements a task may require. */ - public String getAvailabilityGroup(); + public TaskResource getTaskResource(); /** * Returns a descriptive label for this task type. Used for metrics emission and logging. @@ -134,10 +133,4 @@ public interface Task * @throws Exception */ public TaskStatus run(TaskToolbox toolbox) throws Exception; - - /** - * Best-effort task cancellation. May or may not do anything. Calling this multiple times may have - * a stronger effect. - */ - public void shutdown(); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/TaskResource.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/TaskResource.java new file mode 100644 index 00000000000..81d23a942b9 --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/TaskResource.java @@ -0,0 +1,52 @@ +package com.metamx.druid.indexing.common.task; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + */ +public class TaskResource +{ + private final String availabilityGroup; + private final int requiredCapacity; + + @JsonCreator + public TaskResource( + @JsonProperty("availabilityGroup") String availabilityGroup, + @JsonProperty("requiredCapacity") int requiredCapacity + ) + { + this.availabilityGroup = availabilityGroup; + this.requiredCapacity = requiredCapacity; + } + + /** + * Returns availability group ID of this task. Tasks the same availability group cannot be assigned to the same + * worker. If tasks do not have this restriction, a common convention is to set the availability group ID to the + * task ID. + */ + @JsonProperty + public String getAvailabilityGroup() + { + return availabilityGroup; + } + + + /** + * Returns the number of worker slots this task will take. + */ + @JsonProperty + public int getRequiredCapacity() + { + return requiredCapacity; + } + + @Override + public String toString() + { + return "TaskResource{" + + "availabilityGroup='" + availabilityGroup + '\'' + + ", requiredCapacity=" + requiredCapacity + + '}'; + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/VersionConverterTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/VersionConverterTask.java index 27e46f9ce42..4722222bf52 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/VersionConverterTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/VersionConverterTask.java @@ -104,7 +104,7 @@ public class VersionConverterTask extends AbstractTask DataSegment segment ) { - super(id, groupId, id, dataSource, interval); + super(id, groupId, dataSource, interval); this.segment = segment; } @@ -205,13 +205,6 @@ public class VersionConverterTask extends AbstractTask segment.getShardSpec().getPartitionNum() ), groupId, - joinId( - groupId, - "sub", - segment.getInterval().getStart(), - segment.getInterval().getEnd(), - segment.getShardSpec().getPartitionNum() - ), segment.getDataSource(), segment.getInterval() ); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java index 753af4b8daa..0eb0094c6c0 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java @@ -32,6 +32,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.io.ByteStreams; import com.google.common.io.Closeables; +import com.google.common.io.Closer; import com.google.common.io.Files; import com.google.common.io.InputSupplier; import com.google.common.util.concurrent.ListenableFuture; @@ -48,6 +49,7 @@ import com.metamx.druid.indexing.worker.executor.ExecutorMain; import com.metamx.emitter.EmittingLogger; import org.apache.commons.io.FileUtils; +import java.io.Closeable; import java.io.File; import java.io.IOException; import java.io.InputStream; @@ -94,6 +96,12 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider this.jsonMapper = jsonMapper; } + @Override + public void bootstrap(List tasks) + { + // do nothing + } + @Override public ListenableFuture run(final Task task) { @@ -115,137 +123,135 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider final ProcessHolder processHolder; try { - if (!attemptDir.mkdirs()) { - throw new IOException(String.format("Could not create directories: %s", attemptDir)); - } - - final File taskFile = new File(attemptDir, "task.json"); - final File statusFile = new File(attemptDir, "status.json"); - final File logFile = new File(attemptDir, "log"); - - // time to adjust process holders - synchronized (tasks) { - final TaskInfo taskInfo = tasks.get(task.getId()); - - if (taskInfo.shutdown) { - throw new IllegalStateException("Task has been shut down!"); + final Closer closer = Closer.create(); + try { + if (!attemptDir.mkdirs()) { + throw new IOException(String.format("Could not create directories: %s", attemptDir)); } - if (taskInfo == null) { - throw new ISE("WTF?! TaskInfo disappeared for task: %s", task.getId()); - } + final File taskFile = new File(attemptDir, "task.json"); + final File statusFile = new File(attemptDir, "status.json"); + final File logFile = new File(attemptDir, "log"); - if (taskInfo.processHolder != null) { - throw new ISE("WTF?! TaskInfo already has a process holder for task: %s", task.getId()); - } + // time to adjust process holders + synchronized (tasks) { + final TaskInfo taskInfo = tasks.get(task.getId()); - final List command = Lists.newArrayList(); - final int childPort = findUnusedPort(); - final String childHost = String.format(config.getHostPattern(), childPort); + if (taskInfo.shutdown) { + throw new IllegalStateException("Task has been shut down!"); + } - command.add(config.getJavaCommand()); - command.add("-cp"); - command.add(config.getJavaClasspath()); + if (taskInfo == null) { + throw new ISE("WTF?! TaskInfo disappeared for task: %s", task.getId()); + } - Iterables.addAll( - command, - Splitter.on(CharMatcher.WHITESPACE) - .omitEmptyStrings() - .split(config.getJavaOptions()) - ); + if (taskInfo.processHolder != null) { + throw new ISE("WTF?! TaskInfo already has a process holder for task: %s", task.getId()); + } - for (String propName : props.stringPropertyNames()) { - for (String allowedPrefix : config.getAllowedPrefixes()) { - if (propName.startsWith(allowedPrefix)) { + final List command = Lists.newArrayList(); + final int childPort = findUnusedPort(); + final String childHost = String.format(config.getHostPattern(), childPort); + + command.add(config.getJavaCommand()); + command.add("-cp"); + command.add(config.getJavaClasspath()); + + Iterables.addAll( + command, + Splitter.on(CharMatcher.WHITESPACE) + .omitEmptyStrings() + .split(config.getJavaOptions()) + ); + + for (String propName : props.stringPropertyNames()) { + for (String allowedPrefix : config.getAllowedPrefixes()) { + if (propName.startsWith(allowedPrefix)) { + command.add( + String.format( + "-D%s=%s", + propName, + props.getProperty(propName) + ) + ); + } + } + } + + // Override child JVM specific properties + for (String propName : props.stringPropertyNames()) { + if (propName.startsWith(CHILD_PROPERTY_PREFIX)) { command.add( String.format( "-D%s=%s", - propName, + propName.substring(CHILD_PROPERTY_PREFIX.length()), props.getProperty(propName) ) ); } } - } - // Override child JVM specific properties - for (String propName : props.stringPropertyNames()) { - if (propName.startsWith(CHILD_PROPERTY_PREFIX)) { - command.add( - String.format( - "-D%s=%s", - propName.substring(CHILD_PROPERTY_PREFIX.length()), - props.getProperty(propName) - ) - ); + String nodeType = task.getNodeType(); + if (nodeType != null) { + command.add(String.format("-Ddruid.executor.nodeType=%s", nodeType)); } + + command.add(String.format("-Ddruid.host=%s", childHost)); + command.add(String.format("-Ddruid.port=%d", childPort)); + + command.add(config.getMainClass()); + command.add(taskFile.toString()); + command.add(statusFile.toString()); + + jsonMapper.writeValue(taskFile, task); + + log.info("Running command: %s", Joiner.on(" ").join(command)); + taskInfo.processHolder = new ProcessHolder( + new ProcessBuilder(ImmutableList.copyOf(command)).redirectErrorStream(true).start(), + logFile, + childPort + ); + + processHolder = taskInfo.processHolder; + processHolder.registerWithCloser(closer); } - String nodeType = task.getNodeType(); - if (nodeType != null) { - command.add(String.format("-Ddruid.executor.nodeType=%s", nodeType)); - } + log.info("Logging task %s output to: %s", task.getId(), logFile); - command.add(String.format("-Ddruid.host=%s", childHost)); - command.add(String.format("-Ddruid.port=%d", childPort)); - - command.add(config.getMainClass()); - command.add(taskFile.toString()); - command.add(statusFile.toString()); - - jsonMapper.writeValue(taskFile, task); - - log.info("Running command: %s", Joiner.on(" ").join(command)); - taskInfo.processHolder = new ProcessHolder( - new ProcessBuilder(ImmutableList.copyOf(command)).redirectErrorStream(true).start(), - logFile, - childPort + final InputStream fromProc = processHolder.process.getInputStream(); + final OutputStream toLogfile = closer.register( + Files.newOutputStreamSupplier(logFile).getOutput() ); - processHolder = taskInfo.processHolder; - } + boolean runFailed = true; - log.info("Logging task %s output to: %s", task.getId(), logFile); - - final OutputStream toProc = processHolder.process.getOutputStream(); - final InputStream fromProc = processHolder.process.getInputStream(); - final OutputStream toLogfile = Files.newOutputStreamSupplier(logFile).getOutput(); - - boolean runFailed = false; - - try { ByteStreams.copy(fromProc, toLogfile); final int statusCode = processHolder.process.waitFor(); log.info("Process exited with status[%d] for task: %s", statusCode, task.getId()); - if (statusCode != 0) { - runFailed = true; + if (statusCode == 0) { + runFailed = false; } - } - catch (Exception e) { - log.warn(e, "Failed to read from process for task: %s", task.getId()); - runFailed = true; - } - finally { - Closeables.closeQuietly(fromProc); - Closeables.closeQuietly(toLogfile); - Closeables.closeQuietly(toProc); - } - // Upload task logs + // Upload task logs - // XXX: Consider uploading periodically for very long-lived tasks to prevent - // XXX: bottlenecks at the end or the possibility of losing a lot of logs all - // XXX: at once. + // XXX: Consider uploading periodically for very long-lived tasks to prevent + // XXX: bottlenecks at the end or the possibility of losing a lot of logs all + // XXX: at once. - taskLogPusher.pushTaskLog(task.getId(), logFile); + taskLogPusher.pushTaskLog(task.getId(), logFile); - if (!runFailed) { - // Process exited successfully - return jsonMapper.readValue(statusFile, TaskStatus.class); - } else { - // Process exited unsuccessfully - return TaskStatus.failure(task.getId()); + if (!runFailed) { + // Process exited successfully + return jsonMapper.readValue(statusFile, TaskStatus.class); + } else { + // Process exited unsuccessfully + return TaskStatus.failure(task.getId()); + } + } catch (Throwable t) { + throw closer.rethrow(t); + } finally { + closer.close(); } } catch (Exception e) { @@ -311,31 +317,9 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider } if (taskInfo.processHolder != null) { - final int shutdowns = taskInfo.processHolder.shutdowns.getAndIncrement(); - if (shutdowns == 0) { - log.info("Attempting to gracefully shutdown task: %s", taskid); - try { - // This is gross, but it may still be nicer than talking to the forked JVM via HTTP. - final OutputStream out = taskInfo.processHolder.process.getOutputStream(); - out.write( - jsonMapper.writeValueAsBytes( - ImmutableMap.of( - "shutdown", - "now" - ) - ) - ); - out.write('\n'); - out.flush(); - } - catch (IOException e) { - throw Throwables.propagate(e); - } - } else { - // Will trigger normal failure mechanisms due to process exit - log.info("Killing process for task: %s", taskid); - taskInfo.processHolder.process.destroy(); - } + // Will trigger normal failure mechanisms due to process exit + log.info("Killing process for task: %s", taskid); + taskInfo.processHolder.process.destroy(); } } @@ -429,7 +413,6 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider private final Process process; private final File logFile; private final int port; - private final AtomicInteger shutdowns = new AtomicInteger(0); private ProcessHolder(Process process, File logFile, int port) { @@ -437,5 +420,11 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider this.logFile = logFile; this.port = port; } + + private void registerWithCloser(Closer closer) + { + closer.register(process.getInputStream()); + closer.register(process.getOutputStream()); + } } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java index 08157d2ab43..32c5bed1dcd 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java @@ -24,22 +24,18 @@ import com.google.common.base.Charsets; import com.google.common.base.Joiner; import com.google.common.base.Optional; import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; import com.google.common.base.Stopwatch; import com.google.common.base.Throwables; import com.google.common.collect.Lists; -import com.google.common.collect.MinMaxPriorityQueue; import com.google.common.collect.Sets; import com.google.common.io.InputSupplier; import com.google.common.primitives.Ints; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import com.metamx.common.ISE; -import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.druid.indexing.common.RetryPolicy; -import com.metamx.druid.indexing.common.RetryPolicyFactory; +import com.metamx.druid.curator.cache.PathChildrenCacheFactory; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.indexing.common.tasklogs.TaskLogProvider; @@ -49,15 +45,17 @@ import com.metamx.druid.indexing.worker.Worker; import com.metamx.emitter.EmittingLogger; import com.metamx.http.client.HttpClient; import com.metamx.http.client.response.InputStreamResponseHandler; -import com.metamx.http.client.response.ToStringResponseHandler; +import com.metamx.http.client.response.StatusResponseHandler; +import com.metamx.http.client.response.StatusResponseHolder; import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.recipes.cache.ChildData; import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; import org.apache.curator.utils.ZKPaths; import org.apache.zookeeper.CreateMode; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.joda.time.DateTime; -import org.joda.time.Duration; import java.io.IOException; import java.io.InputStream; @@ -68,54 +66,50 @@ import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.TreeSet; import java.util.concurrent.Callable; import java.util.concurrent.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.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; /** - * The RemoteTaskRunner's primary responsibility is to assign tasks to worker nodes and manage retries in failure - * 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's primary responsibility is to assign tasks to worker nodes. + * The RemoteTaskRunner uses Zookeeper to keep track of which workers are running which tasks. Tasks are assigned by + * creating ephemeral nodes in ZK that workers must remove. Workers announce the statuses of the tasks they are running. + * Once a task completes, it is up to the RTR to remove the task status and run any necessary cleanup. + * 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. At that point, task assignment will * fail. The RemoteTaskRunner depends on another component to create additional worker resources. * For example, {@link com.metamx.druid.indexing.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. + * If a worker node becomes inexplicably disconnected from Zk, the RemoteTaskRunner will fail any tasks associated with the worker. *

* The RemoteTaskRunner uses ZK for job management and assignment and http for IPC messages. */ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider { private static final EmittingLogger log = new EmittingLogger(RemoteTaskRunner.class); - private static final ToStringResponseHandler STRING_RESPONSE_HANDLER = new ToStringResponseHandler(Charsets.UTF_8); + private static final StatusResponseHandler RESPONSE_HANDLER = new StatusResponseHandler(Charsets.UTF_8); private static final Joiner JOINER = Joiner.on("/"); private final ObjectMapper jsonMapper; private final RemoteTaskRunnerConfig config; private final CuratorFramework cf; + private final PathChildrenCacheFactory pathChildrenCacheFactory; private final PathChildrenCache workerPathCache; - private final ScheduledExecutorService scheduledExec; - private final RetryPolicyFactory retryPolicyFactory; private final AtomicReference workerSetupData; private final HttpClient httpClient; // all workers that exist in ZK private final Map zkWorkers = new ConcurrentHashMap(); // all tasks that have been assigned to a worker - private final TaskRunnerWorkQueue runningTasks = new TaskRunnerWorkQueue(); + private final RemoteTaskRunnerWorkQueue runningTasks = new RemoteTaskRunnerWorkQueue(); // tasks that have not yet run - private final TaskRunnerWorkQueue pendingTasks = new TaskRunnerWorkQueue(); - // idempotent task retry - private final Set tasksToRetry = new ConcurrentSkipListSet(); + private final RemoteTaskRunnerWorkQueue pendingTasks = new RemoteTaskRunnerWorkQueue(); private final ExecutorService runPendingTasksExec = Executors.newSingleThreadExecutor(); @@ -127,9 +121,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider ObjectMapper jsonMapper, RemoteTaskRunnerConfig config, CuratorFramework cf, - PathChildrenCache workerPathCache, - ScheduledExecutorService scheduledExec, - RetryPolicyFactory retryPolicyFactory, + PathChildrenCacheFactory pathChildrenCacheFactory, AtomicReference workerSetupData, HttpClient httpClient ) @@ -137,9 +129,8 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider this.jsonMapper = jsonMapper; this.config = config; this.cf = cf; - this.workerPathCache = workerPathCache; - this.scheduledExec = scheduledExec; - this.retryPolicyFactory = retryPolicyFactory; + this.pathChildrenCacheFactory = pathChildrenCacheFactory; + this.workerPathCache = pathChildrenCacheFactory.make(cf, config.getIndexerAnnouncementPath()); this.workerSetupData = workerSetupData; this.httpClient = httpClient; } @@ -159,25 +150,37 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider @Override public void childEvent(CuratorFramework client, final PathChildrenCacheEvent event) throws Exception { - if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) { - final Worker worker = jsonMapper.readValue( - event.getData().getData(), - Worker.class - ); - log.info("Worker[%s] reportin' for duty!", worker.getHost()); - addWorker(worker); - } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED)) { - final Worker worker = jsonMapper.readValue( - event.getData().getData(), - Worker.class - ); - log.info("Kaboom! Worker[%s] removed!", worker.getHost()); - removeWorker(worker); + Worker worker; + switch (event.getType()) { + case CHILD_ADDED: + worker = jsonMapper.readValue( + event.getData().getData(), + Worker.class + ); + addWorker(worker, PathChildrenCache.StartMode.NORMAL); + break; + case CHILD_REMOVED: + worker = jsonMapper.readValue( + event.getData().getData(), + Worker.class + ); + removeWorker(worker); + break; + default: + break; } } } ); - workerPathCache.start(); + workerPathCache.start(PathChildrenCache.StartMode.BUILD_INITIAL_CACHE); + + for (ChildData childData : workerPathCache.getCurrentData()) { + final Worker worker = jsonMapper.readValue( + childData.getData(), + Worker.class + ); + addWorker(worker, PathChildrenCache.StartMode.BUILD_INITIAL_CACHE); + } started = true; } @@ -197,6 +200,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider for (ZkWorker zkWorker : zkWorkers.values()) { zkWorker.close(); } + workerPathCache.close(); } catch (Exception e) { throw Throwables.propagate(e); @@ -213,13 +217,13 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider } @Override - public Collection getRunningTasks() + public Collection getRunningTasks() { return runningTasks.values(); } @Override - public Collection getPendingTasks() + public Collection getPendingTasks() { return pendingTasks.values(); } @@ -227,18 +231,46 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider public ZkWorker findWorkerRunningTask(String taskId) { for (ZkWorker zkWorker : zkWorkers.values()) { - if (zkWorker.getRunningTasks().contains(taskId)) { + if (zkWorker.isRunningTask(taskId)) { return zkWorker; } } return null; } - public boolean isWorkerRunningTask(String workerHost, String taskId) + public boolean isWorkerRunningTask(Worker worker, Task task) { - ZkWorker zkWorker = zkWorkers.get(workerHost); + ZkWorker zkWorker = zkWorkers.get(worker.getHost()); - return (zkWorker != null && zkWorker.getRunningTasks().contains(taskId)); + return (zkWorker != null && zkWorker.isRunningTask(task.getId())); + } + + @Override + public void bootstrap(List tasks) + { + try { + if (!started) { + throw new ISE("Must start RTR first before calling bootstrap!"); + } + + Set existingTasks = Sets.newHashSet(); + for (ZkWorker zkWorker : zkWorkers.values()) { + existingTasks.addAll(zkWorker.getRunningTasks().keySet()); + } + + for (Task task : tasks) { + if (existingTasks.contains(task.getId())) { + log.info("Bootstrap found %s running.", task.getId()); + runningTasks.put( + task.getId(), + new RemoteTaskRunnerWorkItem(task, SettableFuture.create()) + ); + } + } + } + catch (Exception e) { + throw Throwables.propagate(e); + } } /** @@ -249,11 +281,19 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider @Override public ListenableFuture run(final Task task) { - 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()); + RemoteTaskRunnerWorkItem runningTask = runningTasks.get(task.getId()); + if (runningTask != null) { + log.info("Assigned a task[%s] that is already running, not doing anything", task.getId()); + return runningTask.getResult(); } - TaskRunnerWorkItem taskRunnerWorkItem = new TaskRunnerWorkItem( - task, SettableFuture.create(), retryPolicyFactory.makeRetryPolicy(), new DateTime() + RemoteTaskRunnerWorkItem pendingTask = pendingTasks.get(task.getId()); + if (pendingTask != null) { + log.info("Assigned a task[%s] that is already pending, not doing anything", task.getId()); + return pendingTask.getResult(); + } + RemoteTaskRunnerWorkItem taskRunnerWorkItem = new RemoteTaskRunnerWorkItem( + task, + SettableFuture.create() ); addPendingTask(taskRunnerWorkItem); return taskRunnerWorkItem.getResult(); @@ -262,7 +302,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider /** * Finds the worker running the task and forwards the shutdown signal to the worker. * - * @param taskId + * @param taskId - task id to shutdown */ @Override public void shutdown(String taskId) @@ -275,40 +315,30 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider final ZkWorker zkWorker = findWorkerRunningTask(taskId); if (zkWorker == null) { - // Would be nice to have an ability to shut down pending tasks log.info("Can't shutdown! No worker running task %s", taskId); return; } - final RetryPolicy shutdownRetryPolicy = retryPolicyFactory.makeRetryPolicy(); - final URL url = workerURL(zkWorker.getWorker(), String.format("/task/%s/shutdown", taskId)); + try { + final URL url = makeWorkerURL(zkWorker.getWorker(), String.format("/task/%s/shutdown", taskId)); + final StatusResponseHolder response = httpClient.post(url) + .go(RESPONSE_HANDLER) + .get(); - while (!shutdownRetryPolicy.hasExceededRetryThreshold()) { - try { - final String response = httpClient.post(url) - .go(STRING_RESPONSE_HANDLER) - .get(); - log.info("Sent shutdown message to worker: %s, response: %s", zkWorker.getWorker().getHost(), response); + log.info( + "Sent shutdown message to worker: %s, status %s, response: %s", + zkWorker.getWorker().getHost(), + response.getStatus(), + response.getContent() + ); - return; - } - catch (Exception e) { - log.error(e, "Exception shutting down taskId: %s", taskId); - - if (shutdownRetryPolicy.hasExceededRetryThreshold()) { - throw Throwables.propagate(e); - } else { - try { - final long sleepTime = shutdownRetryPolicy.getAndIncrementRetryDelay().getMillis(); - log.info("Will try again in %s.", new Duration(sleepTime).toString()); - Thread.sleep(sleepTime); - } - catch (InterruptedException e2) { - throw Throwables.propagate(e2); - } - } + if (!response.getStatus().equals(HttpResponseStatus.ACCEPTED)) { + log.error("Shutdown failed for %s! Are you sure the task was running?", taskId); } } + catch (Exception e) { + throw Throwables.propagate(e); + } } @Override @@ -321,7 +351,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider return Optional.absent(); } else { // Worker is still running this task - final URL url = workerURL(zkWorker.getWorker(), String.format("/task/%s/log?offset=%d", taskId, offset)); + final URL url = makeWorkerURL(zkWorker.getWorker(), String.format("/task/%s/log?offset=%d", taskId, offset)); return Optional.>of( new InputSupplier() { @@ -347,7 +377,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider } } - private URL workerURL(Worker worker, String path) + private URL makeWorkerURL(Worker worker, String path) { Preconditions.checkArgument(path.startsWith("/"), "path must start with '/': %s", path); @@ -361,10 +391,8 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider /** * Adds a task to the pending queue - * - * @param taskRunnerWorkItem */ - private void addPendingTask(final TaskRunnerWorkItem taskRunnerWorkItem) + private void addPendingTask(final RemoteTaskRunnerWorkItem taskRunnerWorkItem) { log.info("Added pending task %s", taskRunnerWorkItem.getTask().getId()); @@ -388,8 +416,8 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider try { // 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) { + List copy = Lists.newArrayList(pendingTasks.values()); + for (RemoteTaskRunnerWorkItem taskWrapper : copy) { assignTask(taskWrapper); } } @@ -403,42 +431,6 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider ); } - /** - * Retries a task by inserting it back into the pending queue after a given delay. - * - * @param taskRunnerWorkItem - the task to retry - */ - private void retryTask(final TaskRunnerWorkItem taskRunnerWorkItem) - { - final String taskId = taskRunnerWorkItem.getTask().getId(); - - if (tasksToRetry.contains(taskId)) { - return; - } - - tasksToRetry.add(taskId); - - if (!taskRunnerWorkItem.getRetryPolicy().hasExceededRetryThreshold()) { - log.info("Retry scheduled in %s for %s", taskRunnerWorkItem.getRetryPolicy().getRetryDelay(), taskId); - scheduledExec.schedule( - new Runnable() - { - @Override - public void run() - { - runningTasks.remove(taskId); - tasksToRetry.remove(taskId); - addPendingTask(taskRunnerWorkItem); - } - }, - taskRunnerWorkItem.getRetryPolicy().getAndIncrementRetryDelay().getMillis(), - TimeUnit.MILLISECONDS - ); - } else { - log.makeAlert("Task exceeded retry threshold").addData("task", taskId).emit(); - } - } - /** * Removes a task from the running queue and clears out the ZK status path of the task. * @@ -464,28 +456,25 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider * * @param taskRunnerWorkItem - the task to assign */ - private void assignTask(TaskRunnerWorkItem taskRunnerWorkItem) + private void assignTask(RemoteTaskRunnerWorkItem 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); + if (runningTasks.containsKey(taskId) || findWorkerRunningTask(taskId) != null) { + log.info("Task[%s] already running.", taskId); } else { // Nothing running this task, announce it in ZK for a worker to run it - zkWorker = findWorkerForTask(taskRunnerWorkItem.getTask()); + ZkWorker zkWorker = findWorkerForTask(taskRunnerWorkItem.getTask()); if (zkWorker != null) { announceTask(zkWorker.getWorker(), taskRunnerWorkItem); } } } catch (Exception e) { - throw Throwables.propagate(e); + log.makeAlert("Exception while trying to run task") + .addData("taskId", taskRunnerWorkItem.getTask().getId()) + .emit(); } } @@ -496,7 +485,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider * @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 + private void announceTask(Worker theWorker, RemoteTaskRunnerWorkItem taskRunnerWorkItem) throws Exception { final Task task = taskRunnerWorkItem.getTask(); @@ -525,7 +514,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider Stopwatch timeoutStopwatch = new Stopwatch(); timeoutStopwatch.start(); synchronized (statusLock) { - while (!isWorkerRunningTask(theWorker.getHost(), task.getId())) { + while (!isWorkerRunningTask(theWorker, task)) { statusLock.wait(config.getTaskAssignmentTimeoutDuration().getMillis()); if (timeoutStopwatch.elapsed(TimeUnit.MILLISECONDS) >= config.getTaskAssignmentTimeoutDuration().getMillis()) { log.error( @@ -534,7 +523,8 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider task.getId(), config.getTaskAssignmentTimeoutDuration() ); - retryTask(runningTasks.get(task.getId())); + + taskRunnerWorkItem.setResult(TaskStatus.failure(taskRunnerWorkItem.getTask().getId())); break; } } @@ -548,11 +538,13 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider * * @param worker - contains metadata for a worker that has appeared in ZK */ - private void addWorker(final Worker worker) + private ZkWorker addWorker(final Worker worker, PathChildrenCache.StartMode startMode) { + log.info("Worker[%s] reportin' for duty!", worker.getHost()); + try { final String workerStatusPath = JOINER.join(config.getIndexerStatusPath(), worker.getHost()); - final PathChildrenCache statusCache = new PathChildrenCache(cf, workerStatusPath, true); + final PathChildrenCache statusCache = pathChildrenCacheFactory.make(cf, workerStatusPath); final ZkWorker zkWorker = new ZkWorker( worker, statusCache, @@ -560,72 +552,70 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider ); // Add status listener to the watcher for status changes - statusCache.getListenable().addListener( + zkWorker.addListener( new PathChildrenCacheListener() { @Override public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception { + String taskId; + RemoteTaskRunnerWorkItem taskRunnerWorkItem; 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 = jsonMapper.readValue( - event.getData().getData(), TaskStatus.class - ); + switch (event.getType()) { + case CHILD_ADDED: + case CHILD_UPDATED: + taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); + final TaskStatus taskStatus = jsonMapper.readValue( + event.getData().getData(), TaskStatus.class + ); - // This can fail if a worker writes a bogus status. Retry if so. - if (!taskStatus.getId().equals(taskId)) { - retryTask(runningTasks.get(taskId)); - return; - } - - log.info( - "Worker[%s] wrote %s status for task: %s", - worker.getHost(), - taskStatus.getStatusCode(), - taskId - ); - - - // Synchronizing state with ZK - 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(), + log.info( + "Worker[%s] wrote %s status for task: %s", + zkWorker.getWorker().getHost(), + taskStatus.getStatusCode(), taskId ); - } - if (taskStatus.isComplete()) { - if (taskRunnerWorkItem != null) { - final ListenableFuture result = taskRunnerWorkItem.getResult(); - if (result != null) { - ((SettableFuture) result).set(taskStatus); - } + // Synchronizing state with ZK + statusLock.notify(); + + taskRunnerWorkItem = runningTasks.get(taskId); + if (taskRunnerWorkItem == null) { + log.warn( + "WTF?! Worker[%s] announcing a status for a task I didn't know about: %s", + zkWorker.getWorker().getHost(), + taskId + ); } - // 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()); - TaskRunnerWorkItem taskRunnerWorkItem = runningTasks.get(taskId); - if (taskRunnerWorkItem != null) { - log.info("Task %s just disappeared!", taskId); - retryTask(taskRunnerWorkItem); - } + if (taskStatus.isComplete()) { + if (taskRunnerWorkItem != null) { + final ListenableFuture result = taskRunnerWorkItem.getResult(); + if (result != null) { + ((SettableFuture) result).set(taskStatus); + } + } + + // Worker is done with this task + zkWorker.setLastCompletedTaskTime(new DateTime()); + cleanup(zkWorker.getWorker().getHost(), taskId); + runPendingTasks(); + } + break; + case CHILD_REMOVED: + taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); + taskRunnerWorkItem = runningTasks.get(taskId); + if (taskRunnerWorkItem != null) { + log.info("Task %s just disappeared!", taskId); + taskRunnerWorkItem.setResult(TaskStatus.failure(taskRunnerWorkItem.getTask().getId())); + } + break; } } catch (Exception e) { log.makeAlert(e, "Failed to handle new worker status") - .addData("worker", worker.getHost()) + .addData("worker", zkWorker.getWorker().getHost()) .addData("znode", event.getData().getPath()) .emit(); } @@ -633,10 +623,13 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider } } ); + + zkWorker.start(startMode); zkWorkers.put(worker.getHost(), zkWorker); - statusCache.start(); runPendingTasks(); + + return zkWorker; } catch (Exception e) { throw Throwables.propagate(e); @@ -652,38 +645,35 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider */ private void removeWorker(final Worker worker) { + log.info("Kaboom! Worker[%s] removed!", worker.getHost()); + ZkWorker zkWorker = zkWorkers.get(worker.getHost()); if (zkWorker != null) { try { - Set tasksToRetry = Sets.newHashSet( - cf.getChildren() - .forPath(JOINER.join(config.getIndexerTaskPath(), worker.getHost())) - ); - tasksToRetry.addAll( - cf.getChildren() - .forPath(JOINER.join(config.getIndexerStatusPath(), worker.getHost())) - ); - log.info("%s has %d tasks to retry", worker.getHost(), tasksToRetry.size()); - - for (String taskId : tasksToRetry) { - TaskRunnerWorkItem taskRunnerWorkItem = runningTasks.get(taskId); + for (String assignedTask : cf.getChildren() + .forPath(JOINER.join(config.getIndexerTaskPath(), worker.getHost()))) { + RemoteTaskRunnerWorkItem taskRunnerWorkItem = runningTasks.get(assignedTask); if (taskRunnerWorkItem != null) { - String taskPath = JOINER.join(config.getIndexerTaskPath(), worker.getHost(), taskId); + String taskPath = JOINER.join(config.getIndexerTaskPath(), worker.getHost(), assignedTask); if (cf.checkExists().forPath(taskPath) != null) { cf.delete().guaranteed().forPath(taskPath); } - retryTask(taskRunnerWorkItem); + taskRunnerWorkItem.setResult(TaskStatus.failure(taskRunnerWorkItem.getTask().getId())); } else { - log.warn("RemoteTaskRunner has no knowledge of task %s", taskId); + log.warn("RemoteTaskRunner has no knowledge of task %s", assignedTask); } } - - zkWorker.close(); } catch (Exception e) { throw Throwables.propagate(e); } finally { + try { + zkWorker.close(); + } + catch (Exception e) { + log.error(e, "Exception closing worker %s!", worker.getHost()); + } zkWorkers.remove(worker.getHost()); } } @@ -691,48 +681,33 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider private ZkWorker findWorkerForTask(final Task task) { - try { - final MinMaxPriorityQueue workerQueue = MinMaxPriorityQueue.orderedBy( - new Comparator() - { - @Override - public int compare(ZkWorker w1, ZkWorker w2) - { - return -Ints.compare(w1.getRunningTasks().size(), w2.getRunningTasks().size()); - } - } - ).create( - FunctionalIterable.create(zkWorkers.values()).filter( - new Predicate() - { - @Override - public boolean apply(ZkWorker input) - { - for (String taskId : input.getRunningTasks()) { - TaskRunnerWorkItem workerTask = runningTasks.get(taskId); - if (workerTask != null && task.getAvailabilityGroup() - .equalsIgnoreCase(workerTask.getTask().getAvailabilityGroup())) { - return false; - } - } - return (!input.isAtCapacity() && - input.getWorker() - .getVersion() - .compareTo(workerSetupData.get().getMinVersion()) >= 0); - } - } + TreeSet sortedWorkers = Sets.newTreeSet( + new Comparator() + { + @Override + public int compare( + ZkWorker zkWorker, ZkWorker zkWorker2 ) - ); + { + int retVal = -Ints.compare(zkWorker.getCurrCapacityUsed(), zkWorker2.getCurrCapacityUsed()); + if (retVal == 0) { + retVal = zkWorker.getWorker().getHost().compareTo(zkWorker2.getWorker().getHost()); + } - if (workerQueue.isEmpty()) { - log.debug("Worker nodes %s do not have capacity to run any more tasks!", zkWorkers.values()); - return null; + return retVal; + } + } + ); + sortedWorkers.addAll(zkWorkers.values()); + final String configMinWorkerVer = workerSetupData.get().getMinVersion(); + final String minWorkerVer = configMinWorkerVer == null ? config.getWorkerVersion() : configMinWorkerVer; + + for (ZkWorker zkWorker : sortedWorkers) { + if (zkWorker.canRunTask(task) && zkWorker.isValidVersion(minWorkerVer)) { + return zkWorker; } - - return workerQueue.peek(); - } - catch (Exception e) { - throw Throwables.propagate(e); } + log.debug("Worker nodes %s do not have capacity to run any more tasks!", zkWorkers.values()); + return null; } -} +} \ No newline at end of file diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerWorkItem.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerWorkItem.java new file mode 100644 index 00000000000..72cb7155af8 --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerWorkItem.java @@ -0,0 +1,63 @@ +/* + * 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.indexing.coordinator; + +import com.google.common.util.concurrent.SettableFuture; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.task.Task; +import org.joda.time.DateTime; + +/** + */ +public class RemoteTaskRunnerWorkItem extends TaskRunnerWorkItem +{ + private final SettableFuture result; + + public RemoteTaskRunnerWorkItem( + Task task, + SettableFuture result + ) + { + super(task, result); + this.result = result; + } + + public RemoteTaskRunnerWorkItem( + Task task, + SettableFuture result, + DateTime createdTime, + DateTime queueInsertionTime + ) + { + super(task, result, createdTime, queueInsertionTime); + this.result = result; + } + + public void setResult(TaskStatus status) + { + result.set(status); + } + + @Override + public RemoteTaskRunnerWorkItem withQueueInsertionTime(DateTime time) + { + return new RemoteTaskRunnerWorkItem(getTask(), result, getCreatedTime(), time); + } +} diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkQueue.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerWorkQueue.java similarity index 84% rename from indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkQueue.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerWorkQueue.java index 0562040c35b..338233c5dca 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkQueue.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerWorkQueue.java @@ -25,10 +25,10 @@ import java.util.concurrent.ConcurrentSkipListMap; /** */ -public class TaskRunnerWorkQueue extends ConcurrentSkipListMap +public class RemoteTaskRunnerWorkQueue extends ConcurrentSkipListMap { @Override - public TaskRunnerWorkItem put(String s, TaskRunnerWorkItem taskRunnerWorkItem) + public RemoteTaskRunnerWorkItem put(String s, RemoteTaskRunnerWorkItem taskRunnerWorkItem) { return super.put(s, taskRunnerWorkItem.withQueueInsertionTime(new DateTime())); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java index 8b4e3fba6e1..f0332aa3cdf 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java @@ -21,12 +21,11 @@ package com.metamx.druid.indexing.coordinator; import com.google.common.base.Optional; import com.google.common.base.Throwables; +import com.metamx.common.ISE; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.curator.discovery.ServiceAnnouncer; -import com.metamx.druid.initialization.Initialization; -import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.indexing.common.actions.TaskActionClient; import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; import com.metamx.druid.indexing.common.task.Task; @@ -34,6 +33,8 @@ import com.metamx.druid.indexing.coordinator.config.IndexerCoordinatorConfig; import com.metamx.druid.indexing.coordinator.exec.TaskConsumer; import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementScheduler; import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementSchedulerFactory; +import com.metamx.druid.initialization.Initialization; +import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import org.apache.curator.framework.CuratorFramework; @@ -88,7 +89,6 @@ public class TaskMasterLifecycle log.info("By the power of Grayskull, I have the power!"); taskRunner = runnerFactory.build(); - resourceManagementScheduler = managementSchedulerFactory.build(taskRunner); final TaskConsumer taskConsumer = new TaskConsumer( taskQueue, taskRunner, @@ -101,12 +101,34 @@ public class TaskMasterLifecycle // Sensible order to start stuff: final Lifecycle leaderLifecycle = new Lifecycle(); - leaderLifecycle.addManagedInstance(taskQueue); leaderLifecycle.addManagedInstance(taskRunner); + leaderLifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() throws Exception + { + taskRunner.bootstrap(taskQueue.snapshot()); + } + + @Override + public void stop() + { + + } + } + ); + leaderLifecycle.addManagedInstance(taskQueue); Initialization.announceDefaultService(serviceDiscoveryConfig, serviceAnnouncer, leaderLifecycle); leaderLifecycle.addManagedInstance(taskConsumer); - leaderLifecycle.addManagedInstance(resourceManagementScheduler); + if ("remote".equalsIgnoreCase(indexerCoordinatorConfig.getRunnerImpl())) { + if (!(taskRunner instanceof RemoteTaskRunner)) { + throw new ISE("WTF?! We configured a remote runner and got %s", taskRunner.getClass()); + } + resourceManagementScheduler = managementSchedulerFactory.build((RemoteTaskRunner) taskRunner); + leaderLifecycle.addManagedInstance(resourceManagementScheduler); + } try { leaderLifecycle.start(); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskQueue.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskQueue.java index 630934c6d17..252728b32dc 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskQueue.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskQueue.java @@ -23,6 +23,7 @@ import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Multimap; import com.google.common.collect.Ordering; @@ -165,6 +166,20 @@ public class TaskQueue } } + /** + * Returns an immutable snapshot of the current status of this queue. + */ + public List snapshot() + { + giant.lock(); + + try { + return ImmutableList.copyOf(queue); + } 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. diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java index 6e8fcfd6a0f..38e4874afb8 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java @@ -24,6 +24,7 @@ import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.task.Task; import java.util.Collection; +import java.util.List; /** * Interface for handing off tasks. Used by a {@link com.metamx.druid.indexing.coordinator.exec.TaskConsumer} to @@ -31,6 +32,16 @@ import java.util.Collection; */ public interface TaskRunner { + /** + * Provide a new task runner with a list of tasks that may already be running. Will be called once shortly + * after instantiation and before any calls to {@link #run}. Bootstrapping should not be construed as a command + * to run the tasks; they will be passed to {@link #run} one-by-one when this is desired. Some bootstrapped tasks + * may not actually be running (for example, if they are currently held back due to not having a lock). + * + * @param tasks the tasks + */ + public void bootstrap(List tasks); + /** * Run a task. The returned status should be some kind of completed status. * @@ -40,14 +51,13 @@ public interface TaskRunner public ListenableFuture run(Task task); /** - * Best-effort task cancellation. May or may not do anything. Calling this multiple times may have - * a stronger effect. + * Best-effort task shutdown. May or may not do anything. */ public void shutdown(String taskid); - public Collection getRunningTasks(); + public Collection getRunningTasks(); - public Collection getPendingTasks(); + public Collection getPendingTasks(); public Collection getWorkers(); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkItem.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkItem.java index 74321f5b1b6..10a4ff5d1a3 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkItem.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkItem.java @@ -22,7 +22,6 @@ package com.metamx.druid.indexing.coordinator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ComparisonChain; import com.google.common.util.concurrent.ListenableFuture; -import com.metamx.druid.indexing.common.RetryPolicy; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.task.Task; import org.joda.time.DateTime; @@ -35,22 +34,29 @@ public class TaskRunnerWorkItem implements Comparable { private final Task task; private final ListenableFuture result; - private final RetryPolicy retryPolicy; private final DateTime createdTime; private volatile DateTime queueInsertionTime; + public TaskRunnerWorkItem( + Task task, + ListenableFuture result + ) + { + this(task, result, new DateTime(), new DateTime()); + } + public TaskRunnerWorkItem( Task task, ListenableFuture result, - RetryPolicy retryPolicy, - DateTime createdTime + DateTime createdTime, + DateTime queueInsertionTime ) { this.task = task; this.result = result; - this.retryPolicy = retryPolicy; this.createdTime = createdTime; + this.queueInsertionTime = queueInsertionTime; } @JsonProperty @@ -64,11 +70,6 @@ public class TaskRunnerWorkItem implements Comparable return result; } - public RetryPolicy getRetryPolicy() - { - return retryPolicy; - } - @JsonProperty public DateTime getCreatedTime() { @@ -83,8 +84,7 @@ public class TaskRunnerWorkItem implements Comparable public TaskRunnerWorkItem withQueueInsertionTime(DateTime time) { - this.queueInsertionTime = time; - return this; + return new TaskRunnerWorkItem(task, result, createdTime, time); } @Override @@ -102,7 +102,6 @@ public class TaskRunnerWorkItem implements Comparable return "TaskRunnerWorkItem{" + "task=" + task + ", result=" + result + - ", retryPolicy=" + retryPolicy + ", createdTime=" + createdTime + '}'; } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java index 6ed4d3abf82..ac2d680ebc0 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java @@ -77,13 +77,19 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker exec.shutdownNow(); } + @Override + public void bootstrap(List tasks) + { + // do nothing + } + @Override public ListenableFuture run(final Task task) { final TaskToolbox toolbox = toolboxFactory.build(task); final ListenableFuture statusFuture = exec.submit(new ExecutorServiceTaskRunnerCallable(task, toolbox)); - final TaskRunnerWorkItem taskRunnerWorkItem = new TaskRunnerWorkItem(task, statusFuture, null, new DateTime()); + final TaskRunnerWorkItem taskRunnerWorkItem = new TaskRunnerWorkItem(task, statusFuture); runningItems.add(taskRunnerWorkItem); Futures.addCallback( statusFuture, new FutureCallback() @@ -110,7 +116,7 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker { for (final TaskRunnerWorkItem runningItem : runningItems) { if (runningItem.getTask().getId().equals(taskid)) { - runningItem.getTask().shutdown(); + runningItem.getResult().cancel(true); } } } @@ -184,14 +190,10 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker private final Task task; private final TaskToolbox toolbox; - private final DateTime createdTime; - public ExecutorServiceTaskRunnerCallable(Task task, TaskToolbox toolbox) { this.task = task; this.toolbox = toolbox; - - this.createdTime = new DateTime(); } @Override @@ -243,12 +245,7 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker public TaskRunnerWorkItem getTaskRunnerWorkItem() { - return new TaskRunnerWorkItem( - task, - null, - null, - createdTime - ); + return new TaskRunnerWorkItem(task, null); } } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java index 16b0f57f5c5..c322548ffea 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java @@ -24,17 +24,20 @@ 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.Maps; import com.google.common.collect.Sets; import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.indexing.worker.Worker; import org.apache.curator.framework.recipes.cache.ChildData; import org.apache.curator.framework.recipes.cache.PathChildrenCache; - +import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; import org.joda.time.DateTime; -import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; +import java.util.Collection; +import java.util.Map; import java.util.Set; /** @@ -44,7 +47,7 @@ public class ZkWorker implements Closeable { private final Worker worker; private final PathChildrenCache statusCache; - private final Function cacheConverter; + private final Function cacheConverter; private volatile DateTime lastCompletedTaskTime = new DateTime(); @@ -52,13 +55,13 @@ public class ZkWorker implements Closeable { this.worker = worker; this.statusCache = statusCache; - this.cacheConverter = new Function() + this.cacheConverter = new Function() { @Override - public String apply(@Nullable ChildData input) + public TaskStatus apply(ChildData input) { try { - return jsonMapper.readValue(input.getData(), TaskStatus.class).getId(); + return jsonMapper.readValue(input.getData(), TaskStatus.class); } catch (Exception e) { throw Throwables.propagate(e); @@ -67,21 +70,59 @@ public class ZkWorker implements Closeable }; } - @JsonProperty + public void start(PathChildrenCache.StartMode startMode) throws Exception + { + statusCache.start(startMode); + } + + public void addListener(PathChildrenCacheListener listener) + { + statusCache.getListenable().addListener(listener); + } + + @JsonProperty("worker") public Worker getWorker() { return worker; } - @JsonProperty - public Set getRunningTasks() + @JsonProperty("runningTasks") + public Collection getRunningTaskIds() { - return Sets.newHashSet( - Lists.transform( - statusCache.getCurrentData(), - cacheConverter - ) - ); + return getRunningTasks().keySet(); + } + + public Map getRunningTasks() + { + Map retVal = Maps.newHashMap(); + for (TaskStatus taskStatus : Lists.transform( + statusCache.getCurrentData(), + cacheConverter + )) { + retVal.put(taskStatus.getId(), taskStatus); + } + + return retVal; + } + + @JsonProperty("currCapacityUsed") + public int getCurrCapacityUsed() + { + int currCapacity = 0; + for (TaskStatus taskStatus : getRunningTasks().values()) { + currCapacity += taskStatus.getResource().getRequiredCapacity(); + } + return currCapacity; + } + + @JsonProperty("availabilityGroups") + public Set getAvailabilityGroups() + { + Set retVal = Sets.newHashSet(); + for (TaskStatus taskStatus : getRunningTasks().values()) { + retVal.add(taskStatus.getResource().getAvailabilityGroup()); + } + return retVal; } @JsonProperty @@ -90,10 +131,25 @@ public class ZkWorker implements Closeable return lastCompletedTaskTime; } - @JsonProperty + public boolean isRunningTask(String taskId) + { + return getRunningTasks().containsKey(taskId); + } + public boolean isAtCapacity() { - return statusCache.getCurrentData().size() >= worker.getCapacity(); + return getCurrCapacityUsed() >= worker.getCapacity(); + } + + public boolean isValidVersion(String minVersion) + { + return worker.getVersion().compareTo(minVersion) >= 0; + } + + public boolean canRunTask(Task task) + { + return (worker.getCapacity() - getCurrCapacityUsed() >= task.getTaskResource().getRequiredCapacity() + && !getAvailabilityGroups().contains(task.getTaskResource().getAvailabilityGroup())); } public void setLastCompletedTaskTime(DateTime completedTaskTime) diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/EC2AutoScalingStrategyConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/EC2AutoScalingStrategyConfig.java index 77575b37cfd..da67e88c64e 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/EC2AutoScalingStrategyConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/EC2AutoScalingStrategyConfig.java @@ -21,6 +21,7 @@ package com.metamx.druid.indexing.coordinator.config; import org.skife.config.Config; import org.skife.config.Default; +import org.skife.config.DefaultNull; /** */ @@ -29,4 +30,8 @@ public abstract class EC2AutoScalingStrategyConfig @Config("druid.indexer.worker.port") @Default("8080") public abstract String getWorkerPort(); + + @Config("druid.indexer.worker.version") + @DefaultNull + public abstract String getWorkerVersion(); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java index 4ff499dbd92..25233ed3300 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java @@ -13,6 +13,7 @@ import java.util.Set; public abstract class ForkingTaskRunnerConfig { @Config("druid.indexer.taskDir") + @Default("/tmp/persistent") public abstract File getBaseTaskDir(); @Config("druid.indexer.fork.java") diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/IndexerCoordinatorConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/IndexerCoordinatorConfig.java index 9c3baf06c24..f8d2cf8c3dc 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/IndexerCoordinatorConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/IndexerCoordinatorConfig.java @@ -41,7 +41,7 @@ public abstract class IndexerCoordinatorConfig extends ZkPathsConfig public abstract int getNumLocalThreads(); @Config("druid.indexer.runner") - @Default("remote") + @Default("local") public abstract String getRunnerImpl(); @Config("druid.indexer.storage") diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java index 15970fc37b3..2acadd3ae83 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java @@ -23,6 +23,7 @@ import com.metamx.druid.indexing.common.config.IndexerZkConfig; import org.joda.time.Duration; import org.skife.config.Config; import org.skife.config.Default; +import org.skife.config.DefaultNull; /** */ @@ -31,4 +32,12 @@ public abstract class RemoteTaskRunnerConfig extends IndexerZkConfig @Config("druid.indexer.taskAssignmentTimeoutDuration") @Default("PT5M") public abstract Duration getTaskAssignmentTimeoutDuration(); + + @Config("druid.curator.compress") + @Default("false") + public abstract boolean enableCompression(); + + @Config("druid.indexer.worker.version") + @DefaultNull + public abstract String getWorkerVersion(); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java index b07ae5f1792..42a82fc8ada 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java @@ -27,13 +27,13 @@ import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Charsets; import com.google.common.base.Optional; import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.io.InputSupplier; import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.inject.Guice; import com.google.inject.Injector; import com.google.inject.servlet.GuiceFilter; +import com.metamx.common.IAE; import com.metamx.common.ISE; import com.metamx.common.concurrent.ScheduledExecutorFactory; import com.metamx.common.concurrent.ScheduledExecutors; @@ -46,6 +46,7 @@ import com.metamx.druid.QueryableNode; import com.metamx.druid.config.ConfigManager; import com.metamx.druid.config.ConfigManagerConfig; import com.metamx.druid.config.JacksonConfigManager; +import com.metamx.druid.curator.cache.SimplePathChildrenCacheFactory; import com.metamx.druid.curator.discovery.CuratorServiceAnnouncer; import com.metamx.druid.curator.discovery.ServiceAnnouncer; import com.metamx.druid.curator.discovery.ServiceInstanceFactory; @@ -55,12 +56,10 @@ import com.metamx.druid.http.GuiceServletConfig; import com.metamx.druid.http.RedirectFilter; import com.metamx.druid.http.RedirectInfo; import com.metamx.druid.http.StatusServlet; -import com.metamx.druid.indexing.common.RetryPolicyFactory; import com.metamx.druid.indexing.common.actions.LocalTaskActionClientFactory; import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; import com.metamx.druid.indexing.common.actions.TaskActionToolbox; import com.metamx.druid.indexing.common.config.IndexerZkConfig; -import com.metamx.druid.indexing.common.config.RetryPolicyConfig; import com.metamx.druid.indexing.common.config.TaskLogConfig; import com.metamx.druid.indexing.common.index.EventReceiverFirehoseFactory; import com.metamx.druid.indexing.common.index.StaticS3FirehoseFactory; @@ -114,7 +113,6 @@ import com.metamx.metrics.MonitorScheduler; import com.metamx.metrics.MonitorSchedulerConfig; import com.metamx.metrics.SysMonitor; import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.x.discovery.ServiceDiscovery; import org.jets3t.service.S3ServiceException; import org.jets3t.service.impl.rest.httpclient.RestS3Service; @@ -386,15 +384,17 @@ public class IndexerCoordinatorNode extends QueryableNode WorkerSetupData setupData = workerSetupDataRef.get(); EC2NodeData workerConfig = setupData.getNodeData(); + GalaxyUserData userData = setupData.getUserData(); + if (config.getWorkerVersion() != null) { + userData = userData.withVersion(config.getWorkerVersion()); + } + RunInstancesResult result = amazonEC2Client.runInstances( new RunInstancesRequest( workerConfig.getAmiId(), @@ -84,7 +90,7 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy .withUserData( Base64.encodeBase64String( jsonMapper.writeValueAsBytes( - setupData.getUserData() + userData ) ) ) @@ -212,7 +218,7 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy } ); - log.info("Performing lookup: %s --> %s", ips, retVal); + log.debug("Performing lookup: %s --> %s", ips, retVal); return retVal; } @@ -244,7 +250,7 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy } ); - log.info("Performing lookup: %s --> %s", nodeIds, retVal); + log.debug("Performing lookup: %s --> %s", nodeIds, retVal); return retVal; } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementScheduler.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementScheduler.java index 5ac9d4384bc..0cfdc94c76a 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementScheduler.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementScheduler.java @@ -24,6 +24,7 @@ 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.indexing.coordinator.RemoteTaskRunner; import com.metamx.druid.indexing.coordinator.TaskRunner; import org.joda.time.DateTime; import org.joda.time.Duration; @@ -42,7 +43,7 @@ public class ResourceManagementScheduler { private static final Logger log = new Logger(ResourceManagementScheduler.class); - private final TaskRunner taskRunner; + private final RemoteTaskRunner taskRunner; private final ResourceManagementStrategy resourceManagementStrategy; private final ResourceManagementSchedulerConfig config; private final ScheduledExecutorService exec; @@ -51,7 +52,7 @@ public class ResourceManagementScheduler private volatile boolean started = false; public ResourceManagementScheduler( - TaskRunner taskRunner, + RemoteTaskRunner taskRunner, ResourceManagementStrategy resourceManagementStrategy, ResourceManagementSchedulerConfig config, ScheduledExecutorService exec diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementSchedulerFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementSchedulerFactory.java index 51c7833db9a..16ae50ccc2b 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementSchedulerFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementSchedulerFactory.java @@ -19,11 +19,11 @@ package com.metamx.druid.indexing.coordinator.scaling; -import com.metamx.druid.indexing.coordinator.TaskRunner; +import com.metamx.druid.indexing.coordinator.RemoteTaskRunner; /** */ public interface ResourceManagementSchedulerFactory { - public ResourceManagementScheduler build(TaskRunner runner); + public ResourceManagementScheduler build(RemoteTaskRunner runner); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementStrategy.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementStrategy.java index b658b812ac8..2c61dc79285 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementStrategy.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementStrategy.java @@ -19,7 +19,7 @@ package com.metamx.druid.indexing.coordinator.scaling; -import com.metamx.druid.indexing.coordinator.TaskRunnerWorkItem; +import com.metamx.druid.indexing.coordinator.RemoteTaskRunnerWorkItem; import com.metamx.druid.indexing.coordinator.ZkWorker; import java.util.Collection; @@ -30,9 +30,9 @@ import java.util.Collection; */ public interface ResourceManagementStrategy { - public boolean doProvision(Collection runningTasks, Collection zkWorkers); + public boolean doProvision(Collection runningTasks, Collection zkWorkers); - public boolean doTerminate(Collection runningTasks, Collection zkWorkers); + public boolean doTerminate(Collection runningTasks, Collection zkWorkers); public ScalingStats getStats(); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java index 023091935f8..0f9fd927245 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java @@ -25,6 +25,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.common.guava.FunctionalIterable; +import com.metamx.druid.indexing.coordinator.RemoteTaskRunnerWorkItem; import com.metamx.druid.indexing.coordinator.TaskRunnerWorkItem; import com.metamx.druid.indexing.coordinator.ZkWorker; import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; @@ -68,10 +69,24 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat } @Override - public boolean doProvision(Collection pendingTasks, Collection zkWorkers) + public boolean doProvision(Collection pendingTasks, Collection zkWorkers) { - if (zkWorkers.size() >= workerSetupdDataRef.get().getMaxNumWorkers()) { - log.info( + final WorkerSetupData workerSetupData = workerSetupdDataRef.get(); + + final String minVersion = workerSetupData.getMinVersion() == null + ? config.getWorkerVersion() + : workerSetupData.getMinVersion(); + int maxNumWorkers = workerSetupData.getMaxNumWorkers(); + + int currValidWorkers = 0; + for (ZkWorker zkWorker : zkWorkers) { + if (zkWorker.isValidVersion(minVersion)) { + currValidWorkers++; + } + } + + if (currValidWorkers >= maxNumWorkers) { + log.debug( "Cannot scale anymore. Num workers = %d, Max num workers = %d", zkWorkers.size(), workerSetupdDataRef.get().getMaxNumWorkers() @@ -135,7 +150,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat } @Override - public boolean doTerminate(Collection pendingTasks, Collection zkWorkers) + public boolean doTerminate(Collection pendingTasks, Collection zkWorkers) { Set workerNodeIds = Sets.newHashSet( autoScalingStrategy.ipToIdLookup( @@ -244,7 +259,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat return scalingStats; } - private boolean hasTaskPendingBeyondThreshold(Collection pendingTasks) + private boolean hasTaskPendingBeyondThreshold(Collection pendingTasks) { long now = System.currentTimeMillis(); for (TaskRunnerWorkItem pendingTask : pendingTasks) { diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagmentConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagmentConfig.java index 1584e4b5dfc..184e1aba7ed 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagmentConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagmentConfig.java @@ -22,6 +22,7 @@ package com.metamx.druid.indexing.coordinator.scaling; import org.joda.time.Duration; import org.skife.config.Config; import org.skife.config.Default; +import org.skife.config.DefaultNull; /** */ @@ -42,4 +43,8 @@ public abstract class SimpleResourceManagmentConfig @Config("druid.indexer.maxPendingTaskDuration") @Default("PT30S") public abstract Duration getMaxPendingTaskDuration(); + + @Config("druid.indexer.worker.version") + @DefaultNull + public abstract String getWorkerVersion(); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/setup/GalaxyUserData.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/setup/GalaxyUserData.java index 60241f472ff..4a7490660cc 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/setup/GalaxyUserData.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/setup/GalaxyUserData.java @@ -60,6 +60,11 @@ public class GalaxyUserData return type; } + public GalaxyUserData withVersion(String ver) + { + return new GalaxyUserData(env, ver, type); + } + @Override public String toString() { diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerCuratorCoordinator.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerCuratorCoordinator.java index 77e70f52a5f..89d6c313487 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerCuratorCoordinator.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerCuratorCoordinator.java @@ -130,6 +130,7 @@ public class WorkerCuratorCoordinator } curatorFramework.create() + .creatingParentsIfNeeded() .withMode(mode) .forPath(path, rawBytes); } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/config/WorkerConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/config/WorkerConfig.java index c6b3bdcdc74..deca87c56f5 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/config/WorkerConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/config/WorkerConfig.java @@ -29,10 +29,6 @@ public abstract class WorkerConfig @Config("druid.host") public abstract String getHost(); - @Config("druid.worker.threads") - @Default("1") - public abstract int getNumThreads(); - @Config("druid.worker.ip") public abstract String getIp(); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ChatHandlerResource.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ChatHandlerResource.java index 05d459d1d3a..cf5a89ce219 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ChatHandlerResource.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ChatHandlerResource.java @@ -1,6 +1,5 @@ package com.metamx.druid.indexing.worker.executor; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.inject.Inject; import com.metamx.druid.indexing.common.index.ChatHandler; @@ -13,13 +12,11 @@ import javax.ws.rs.core.Response; @Path("/druid/worker/v1") public class ChatHandlerResource { - private final ObjectMapper jsonMapper; private final ChatHandlerProvider handlers; @Inject - public ChatHandlerResource(ObjectMapper jsonMapper, ChatHandlerProvider handlers) + public ChatHandlerResource(ChatHandlerProvider handlers) { - this.jsonMapper = jsonMapper; this.handlers = handlers; } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycle.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycle.java index 2872da7ed58..d4f47a19771 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycle.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycle.java @@ -78,7 +78,6 @@ public class ExecutorLifecycle } // Spawn monitor thread to keep a watch on parent's stdin - // If a message comes over stdin, we want to handle it // If stdin reaches eof, the parent is gone, and we should shut down parentMonitorExec.submit( new Runnable() @@ -87,25 +86,8 @@ public class ExecutorLifecycle public void run() { try { - final BufferedReader parentReader = new BufferedReader(new InputStreamReader(parentStream)); - String messageString; - while ((messageString = parentReader.readLine()) != null) { - final Map message = jsonMapper - .readValue( - messageString, - new TypeReference>() - { - } - ); - - if (message == null) { - break; - } else if (message.get("shutdown") != null && message.get("shutdown").equals("now")) { - log.info("Shutting down!"); - task.shutdown(); - } else { - throw new ISE("Unrecognized message from parent: %s", message); - } + while (parentStream.read() != -1) { + // Toss the byte } } catch (Exception e) { diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java index 06c160106d5..341a578d6de 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java @@ -22,8 +22,8 @@ package com.metamx.druid.indexing.worker.executor; import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.smile.SmileFactory; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.inject.Guice; import com.google.inject.Injector; @@ -37,12 +37,24 @@ import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.BaseServerNode; import com.metamx.druid.curator.discovery.CuratorServiceAnnouncer; -import com.metamx.druid.curator.discovery.NoopServiceAnnouncer; import com.metamx.druid.curator.discovery.ServiceAnnouncer; import com.metamx.druid.curator.discovery.ServiceInstanceFactory; import com.metamx.druid.http.GuiceServletConfig; import com.metamx.druid.http.QueryServlet; import com.metamx.druid.http.StatusServlet; +import com.metamx.druid.indexing.common.RetryPolicyFactory; +import com.metamx.druid.indexing.common.TaskToolboxFactory; +import com.metamx.druid.indexing.common.actions.RemoteTaskActionClientFactory; +import com.metamx.druid.indexing.common.config.RetryPolicyConfig; +import com.metamx.druid.indexing.common.config.TaskConfig; +import com.metamx.druid.indexing.common.index.ChatHandlerProvider; +import com.metamx.druid.indexing.common.index.EventReceiverFirehoseFactory; +import com.metamx.druid.indexing.common.index.EventReceivingChatHandlerProvider; +import com.metamx.druid.indexing.common.index.NoopChatHandlerProvider; +import com.metamx.druid.indexing.common.index.StaticS3FirehoseFactory; +import com.metamx.druid.indexing.coordinator.ThreadPoolTaskRunner; +import com.metamx.druid.indexing.worker.config.ChatHandlerProviderConfig; +import com.metamx.druid.indexing.worker.config.WorkerConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ServerInit; @@ -51,17 +63,6 @@ import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentKiller; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.S3DataSegmentKiller; -import com.metamx.druid.indexing.common.RetryPolicyFactory; -import com.metamx.druid.indexing.common.TaskToolboxFactory; -import com.metamx.druid.indexing.common.actions.RemoteTaskActionClientFactory; -import com.metamx.druid.indexing.common.config.RetryPolicyConfig; -import com.metamx.druid.indexing.common.config.TaskConfig; -import com.metamx.druid.indexing.common.index.EventReceiverFirehoseFactory; -import com.metamx.druid.indexing.common.index.ChatHandlerProvider; -import com.metamx.druid.indexing.common.index.StaticS3FirehoseFactory; -import com.metamx.druid.indexing.coordinator.ThreadPoolTaskRunner; -import com.metamx.druid.indexing.worker.config.ChatHandlerProviderConfig; -import com.metamx.druid.indexing.worker.config.WorkerConfig; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitters; @@ -69,11 +70,10 @@ import com.metamx.emitter.service.ServiceEmitter; import com.metamx.http.client.HttpClient; import com.metamx.http.client.HttpClientConfig; import com.metamx.http.client.HttpClientInit; -import com.metamx.metrics.JvmMonitor; import com.metamx.metrics.Monitor; import com.metamx.metrics.MonitorScheduler; import com.metamx.metrics.MonitorSchedulerConfig; -import com.metamx.metrics.SysMonitor; +import org.apache.curator.framework.CuratorFramework; import org.apache.curator.x.discovery.ServiceDiscovery; import org.apache.curator.x.discovery.ServiceProvider; import org.jets3t.service.S3ServiceException; @@ -85,7 +85,6 @@ import org.mortbay.jetty.servlet.DefaultServlet; import org.mortbay.jetty.servlet.ServletHolder; import org.skife.config.ConfigurationObjectFactory; -import java.util.List; import java.util.Properties; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -107,7 +106,7 @@ public class ExecutorNode extends BaseServerNode private final ExecutorLifecycleFactory executorLifecycleFactory; private RestS3Service s3Service = null; - private List monitors = null; + private MonitorScheduler monitorScheduler = null; private HttpClient httpClient = null; private ServiceEmitter emitter = null; private TaskConfig taskConfig = null; @@ -140,58 +139,16 @@ public class ExecutorNode extends BaseServerNode this.executorLifecycleFactory = executorLifecycleFactory; } - public ExecutorNode setHttpClient(HttpClient httpClient) - { - this.httpClient = httpClient; - return this; - } - - public ExecutorNode setEmitter(ServiceEmitter emitter) - { - this.emitter = emitter; - return this; - } - - public ExecutorNode setS3Service(RestS3Service s3Service) - { - this.s3Service = s3Service; - return this; - } - - public ExecutorNode setSegmentPusher(DataSegmentPusher segmentPusher) - { - this.segmentPusher = segmentPusher; - return this; - } - - public ExecutorNode setTaskToolboxFactory(TaskToolboxFactory taskToolboxFactory) - { - this.taskToolboxFactory = taskToolboxFactory; - return this; - } - - public ExecutorNode setCoordinatorServiceProvider(ServiceProvider coordinatorServiceProvider) - { - this.coordinatorServiceProvider = coordinatorServiceProvider; - return this; - } - - public ExecutorNode setServiceDiscovery(ServiceDiscovery serviceDiscovery) - { - this.serviceDiscovery = serviceDiscovery; - return this; - } - @Override public void doInit() throws Exception { initializeHttpClient(); initializeEmitter(); initializeS3Service(); - initializeMonitors(); initializeMergerConfig(); initializeServiceDiscovery(); initializeDataSegmentPusher(); + initializeMonitorScheduler(); initializeTaskToolbox(); initializeTaskRunner(); initializeChatHandlerProvider(); @@ -199,13 +156,6 @@ public class ExecutorNode extends BaseServerNode initializeJacksonSubtypes(); initializeServer(); - final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle); - final ScheduledExecutorService globalScheduledExec = scheduledExecutorFactory.create(1, "Global--%d"); - final MonitorScheduler monitorScheduler = new MonitorScheduler( - configFactory.build(MonitorSchedulerConfig.class), globalScheduledExec, emitter, monitors - ); - lifecycle.addManagedInstance(monitorScheduler); - executorLifecycle = executorLifecycleFactory.build(taskRunner, getJsonMapper()); lifecycle.addManagedInstance(executorLifecycle); @@ -229,6 +179,19 @@ public class ExecutorNode extends BaseServerNode ); } + private void initializeMonitorScheduler() + { + if (monitorScheduler == null) + { + final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle); + final ScheduledExecutorService globalScheduledExec = scheduledExecutorFactory.create(1, "Global--%d"); + this.monitorScheduler = new MonitorScheduler( + configFactory.build(MonitorSchedulerConfig.class), globalScheduledExec, emitter, ImmutableList.of() + ); + lifecycle.addManagedInstance(monitorScheduler); + } + } + @LifecycleStart public synchronized void start() throws Exception { @@ -323,7 +286,7 @@ public class ExecutorNode extends BaseServerNode private void initializeS3Service() throws S3ServiceException { - if(s3Service == null) { + if (s3Service == null) { s3Service = new RestS3Service( new AWSCredentials( PropUtils.getProperty(props, "com.metamx.aws.accessKey"), @@ -333,15 +296,6 @@ public class ExecutorNode extends BaseServerNode } } - private void initializeMonitors() - { - if (monitors == null) { - monitors = Lists.newArrayList(); - monitors.add(new JvmMonitor()); - monitors.add(new SysMonitor()); - } - } - private void initializeMergerConfig() { if (taskConfig == null) { @@ -384,6 +338,7 @@ public class ExecutorNode extends BaseServerNode getAnnouncer(), getServerView(), getConglomerate(), + monitorScheduler, getJsonMapper() ); } @@ -393,8 +348,9 @@ public class ExecutorNode extends BaseServerNode { final ServiceDiscoveryConfig config = configFactory.build(ServiceDiscoveryConfig.class); if (serviceDiscovery == null) { + final CuratorFramework serviceDiscoveryCuratorFramework = Initialization.makeCuratorFramework(config, lifecycle); this.serviceDiscovery = Initialization.makeServiceDiscoveryClient( - getCuratorFramework(), config, lifecycle + serviceDiscoveryCuratorFramework, config, lifecycle ); } if (serviceAnnouncer == null) { @@ -430,17 +386,15 @@ public class ExecutorNode extends BaseServerNode { if (chatHandlerProvider == null) { final ChatHandlerProviderConfig config = configFactory.build(ChatHandlerProviderConfig.class); - final ServiceAnnouncer myServiceAnnouncer; if (config.getServiceFormat() == null) { - log.info("ChatHandlerProvider: Using NoopServiceAnnouncer. Good luck finding your firehoses!"); - myServiceAnnouncer = new NoopServiceAnnouncer(); + log.info("ChatHandlerProvider: Using NoopChatHandlerProvider. Good luck finding your firehoses!"); + this.chatHandlerProvider = new NoopChatHandlerProvider(); } else { - myServiceAnnouncer = serviceAnnouncer; + this.chatHandlerProvider = new EventReceivingChatHandlerProvider( + config, + serviceAnnouncer + ); } - this.chatHandlerProvider = new ChatHandlerProvider( - config, - myServiceAnnouncer - ); } } @@ -482,9 +436,12 @@ public class ExecutorNode extends BaseServerNode jsonMapper = new DefaultObjectMapper(); smileMapper = new DefaultObjectMapper(new SmileFactory()); smileMapper.getJsonFactory().setCodec(smileMapper); - } - else if (jsonMapper == null || smileMapper == null) { - throw new ISE("Only jsonMapper[%s] or smileMapper[%s] was set, must set neither or both.", jsonMapper, smileMapper); + } else if (jsonMapper == null || smileMapper == null) { + throw new ISE( + "Only jsonMapper[%s] or smileMapper[%s] was set, must set neither or both.", + jsonMapper, + smileMapper + ); } if (lifecycle == null) { @@ -499,7 +456,15 @@ public class ExecutorNode extends BaseServerNode configFactory = Config.createFactory(props); } - return new ExecutorNode(nodeType, props, lifecycle, jsonMapper, smileMapper, configFactory, executorLifecycleFactory); + return new ExecutorNode( + nodeType, + props, + lifecycle, + jsonMapper, + smileMapper, + configFactory, + executorLifecycleFactory + ); } } } diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java index 1ccf32e0795..5c38440a9a2 100644 --- a/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java @@ -331,11 +331,13 @@ public class WorkerNode extends QueryableNode { if (serviceDiscovery == null) { final ServiceDiscoveryConfig config = getConfigFactory().build(ServiceDiscoveryConfig.class); - this.serviceDiscovery = Initialization.makeServiceDiscoveryClient( - getCuratorFramework(), + final CuratorFramework serviceDiscoveryCuratorFramework = Initialization.makeCuratorFramework( config, getLifecycle() ); + this.serviceDiscovery = Initialization.makeServiceDiscoveryClient( + serviceDiscoveryCuratorFramework, config, getLifecycle() + ); } if (coordinatorServiceProvider == null) { this.coordinatorServiceProvider = Initialization.makeServiceProvider( @@ -392,7 +394,7 @@ public class WorkerNode extends QueryableNode public void initializeWorkerTaskMonitor() { if (workerTaskMonitor == null) { - final ExecutorService workerExec = Executors.newFixedThreadPool(workerConfig.getNumThreads()); + final ExecutorService workerExec = Executors.newFixedThreadPool(workerConfig.getCapacity()); final CuratorFramework curatorFramework = getCuratorFramework(); final PathChildrenCache pathChildrenCache = new PathChildrenCache( diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/common/task/TaskSerdeTest.java index dd819056ac9..d4d2e99d4f2 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/common/task/TaskSerdeTest.java @@ -207,7 +207,8 @@ public class TaskSerdeTest null, null, new Period("PT10M"), - IndexGranularity.HOUR + IndexGranularity.HOUR, + null ); final ObjectMapper jsonMapper = new DefaultObjectMapper(); diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RealtimeishTask.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RealtimeishTask.java index a43ec4fb2fc..4e1dc18ae5f 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RealtimeishTask.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RealtimeishTask.java @@ -30,6 +30,7 @@ import com.metamx.druid.indexing.common.actions.LockListAction; import com.metamx.druid.indexing.common.actions.LockReleaseAction; import com.metamx.druid.indexing.common.actions.SegmentInsertAction; import com.metamx.druid.indexing.common.task.AbstractTask; +import com.metamx.druid.indexing.common.task.TaskResource; import org.joda.time.Interval; import org.junit.Assert; @@ -41,12 +42,12 @@ public class RealtimeishTask extends AbstractTask { public RealtimeishTask() { - super("rt1", "rt", "rt1", "foo", null); + super("rt1", "rt", new TaskResource("rt1", 1), "foo", null); } - public RealtimeishTask(String id, String groupId, String availGroup, String dataSource, Interval interval) + public RealtimeishTask(String id, String groupId, TaskResource taskResource, String dataSource, Interval interval) { - super(id, groupId, availGroup, dataSource, interval); + super(id, groupId, taskResource, dataSource, interval); } @Override diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java index 0a11fcb49fb..6c6da7ccb89 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java @@ -2,22 +2,26 @@ package com.metamx.druid.indexing.coordinator; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.google.common.base.Function; +import com.google.common.base.Joiner; import com.google.common.base.Stopwatch; -import com.google.common.base.Throwables; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.FutureCallback; -import com.google.common.util.concurrent.Futures; +import com.google.common.collect.Sets; +import com.google.common.io.Files; +import com.google.common.util.concurrent.ListenableFuture; import com.metamx.common.ISE; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.curator.PotentiallyGzippedCompressionProvider; +import com.metamx.druid.curator.cache.SimplePathChildrenCacheFactory; import com.metamx.druid.indexing.TestTask; -import com.metamx.druid.indexing.common.RetryPolicyFactory; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.TaskToolboxFactory; import com.metamx.druid.indexing.common.config.IndexerZkConfig; -import com.metamx.druid.indexing.common.config.RetryPolicyConfig; import com.metamx.druid.indexing.common.config.TaskConfig; +import com.metamx.druid.indexing.common.task.Task; +import com.metamx.druid.indexing.common.task.TaskResource; import com.metamx.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; import com.metamx.druid.indexing.worker.Worker; @@ -26,7 +30,6 @@ import com.metamx.druid.indexing.worker.WorkerTaskMonitor; import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; -import org.apache.commons.lang.mutable.MutableBoolean; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.framework.recipes.cache.PathChildrenCache; @@ -42,35 +45,34 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import javax.annotation.Nullable; import java.io.File; +import java.util.Arrays; +import java.util.Set; import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; -import static junit.framework.Assert.fail; - /** */ public class RemoteTaskRunnerTest { private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); - private static final String basePath = "/test/druid/indexer"; - private static final String announcementsPath = String.format("%s/announcements", basePath); - private static final String tasksPath = String.format("%s/tasks", basePath); - private static final String statusPath = String.format("%s/status", basePath); + private static final Joiner joiner = Joiner.on("/"); + private static final String basePath = "/test/druid"; + private static final String announcementsPath = String.format("%s/indexer/announcements/worker", basePath); + private static final String tasksPath = String.format("%s/indexer/tasks/worker", basePath); + private static final String statusPath = String.format("%s/indexer/status/worker", basePath); private TestingCluster testingCluster; private CuratorFramework cf; - private PathChildrenCache pathChildrenCache; private RemoteTaskRunner remoteTaskRunner; + private WorkerCuratorCoordinator workerCuratorCoordinator; private WorkerTaskMonitor workerTaskMonitor; - private ScheduledExecutorService scheduledExec; + private TestTask task; - private TestTask task1; - - private Worker worker1; + private Worker worker; @Before public void setUp() throws Exception @@ -84,24 +86,189 @@ public class RemoteTaskRunnerTest .compressionProvider(new PotentiallyGzippedCompressionProvider(false)) .build(); cf.start(); + cf.create().creatingParentsIfNeeded().forPath(basePath); - cf.create().creatingParentsIfNeeded().forPath(announcementsPath); - cf.create().forPath(tasksPath); - cf.create().forPath(String.format("%s/worker1", tasksPath)); - cf.create().forPath(statusPath); - cf.create().forPath(String.format("%s/worker1", statusPath)); + task = makeTask(TaskStatus.success("task")); + } - pathChildrenCache = new PathChildrenCache(cf, announcementsPath, true); + @After + public void tearDown() throws Exception + { + remoteTaskRunner.stop(); + workerCuratorCoordinator.stop(); + workerTaskMonitor.stop(); + cf.close(); + testingCluster.stop(); + } - worker1 = new Worker( - "worker1", - "localhost", - 3, - "0" + @Test + public void testRunNoExistingTask() throws Exception + { + doSetup(); + + remoteTaskRunner.run(task); + } + + @Test + public void testRunTooMuchZKData() throws Exception + { + ServiceEmitter emitter = EasyMock.createMock(ServiceEmitter.class); + EmittingLogger.registerEmitter(emitter); + EasyMock.replay(emitter); + + doSetup(); + + remoteTaskRunner.run(makeTask(TaskStatus.success(new String(new char[5000])))); + + EasyMock.verify(emitter); + } + + @Test + public void testRunSameAvailabilityGroup() throws Exception + { + doSetup(); + + TestRealtimeTask theTask = new TestRealtimeTask( + "rt1", + new TaskResource("rt1", 1), + "foo", + TaskStatus.running("rt1") + ); + remoteTaskRunner.run(theTask); + remoteTaskRunner.run( + new TestRealtimeTask("rt2", new TaskResource("rt1", 1), "foo", TaskStatus.running("rt2")) + ); + remoteTaskRunner.run( + new TestRealtimeTask("rt3", new TaskResource("rt2", 1), "foo", TaskStatus.running("rt3")) ); - task1 = new TestTask( - "task1", + Stopwatch stopwatch = new Stopwatch(); + stopwatch.start(); + while (remoteTaskRunner.getRunningTasks().size() < 2) { + Thread.sleep(100); + if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > 1000) { + throw new ISE("Cannot find running task"); + } + } + + Assert.assertTrue(remoteTaskRunner.getRunningTasks().size() == 2); + Assert.assertTrue(remoteTaskRunner.getPendingTasks().size() == 1); + Assert.assertTrue(remoteTaskRunner.getPendingTasks().iterator().next().getTask().getId().equals("rt2")); + } + + @Test + public void testRunWithCapacity() throws Exception + { + doSetup(); + + TestRealtimeTask theTask = new TestRealtimeTask( + "rt1", + new TaskResource("rt1", 1), + "foo", + TaskStatus.running("rt1") + ); + remoteTaskRunner.run(theTask); + remoteTaskRunner.run( + new TestRealtimeTask("rt2", new TaskResource("rt2", 3), "foo", TaskStatus.running("rt2")) + ); + remoteTaskRunner.run( + new TestRealtimeTask("rt3", new TaskResource("rt3", 2), "foo", TaskStatus.running("rt3")) + ); + + Stopwatch stopwatch = new Stopwatch(); + stopwatch.start(); + while (remoteTaskRunner.getRunningTasks().size() < 2) { + Thread.sleep(100); + if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > 1000) { + throw new ISE("Cannot find running task"); + } + } + + Assert.assertTrue(remoteTaskRunner.getRunningTasks().size() == 2); + Assert.assertTrue(remoteTaskRunner.getPendingTasks().size() == 1); + Assert.assertTrue(remoteTaskRunner.getPendingTasks().iterator().next().getTask().getId().equals("rt2")); + } + + @Test + public void testFailure() throws Exception + { + doSetup(); + + ListenableFuture future = remoteTaskRunner.run(makeTask(TaskStatus.running("task"))); + final String taskStatus = joiner.join(statusPath, "task"); + + Stopwatch stopwatch = new Stopwatch(); + stopwatch.start(); + while (cf.checkExists().forPath(taskStatus) == null) { + Thread.sleep(100); + if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > 1000) { + throw new ISE("Cannot find running task"); + } + } + Assert.assertTrue(remoteTaskRunner.getRunningTasks().iterator().next().getTask().getId().equals("task")); + + cf.delete().forPath(taskStatus); + + TaskStatus status = future.get(); + + Assert.assertEquals(status.getStatusCode(), TaskStatus.Status.FAILED); + } + + @Test + public void testBootstrap() throws Exception + { + cf.create() + .creatingParentsIfNeeded() + .withMode(CreateMode.EPHEMERAL) + .forPath(joiner.join(statusPath, "first"), jsonMapper.writeValueAsBytes(TaskStatus.running("first"))); + cf.create() + .creatingParentsIfNeeded() + .withMode(CreateMode.EPHEMERAL) + .forPath(joiner.join(statusPath, "second"), jsonMapper.writeValueAsBytes(TaskStatus.running("second"))); + + doSetup(); + + Set existingTasks = Sets.newHashSet(); + for (ZkWorker zkWorker : remoteTaskRunner.getWorkers()) { + existingTasks.addAll(zkWorker.getRunningTasks().keySet()); + } + + Assert.assertTrue(existingTasks.size() == 2); + Assert.assertTrue(existingTasks.contains("first")); + Assert.assertTrue(existingTasks.contains("second")); + + remoteTaskRunner.bootstrap(Arrays.asList(makeTask(TaskStatus.running("second")))); + + Set runningTasks = Sets.newHashSet( + Iterables.transform( + remoteTaskRunner.getRunningTasks(), + new Function() + { + @Override + public String apply(RemoteTaskRunnerWorkItem input) + { + return input.getTask().getId(); + } + } + ) + ); + + Assert.assertTrue(runningTasks.size() == 1); + Assert.assertTrue(runningTasks.contains("second")); + Assert.assertFalse(runningTasks.contains("first")); + } + + private void doSetup() throws Exception + { + makeWorker(); + makeRemoteTaskRunner(); + makeTaskMonitor(); + } + + private TestTask makeTask(TaskStatus status) + { + return new TestTask( + status.getId(), "dummyDs", Lists.newArrayList( new DataSegment( @@ -117,194 +284,20 @@ public class RemoteTaskRunnerTest ) ), Lists.newArrayList(), - TaskStatus.success("task1") + status ); - - makeRemoteTaskRunner(); - makeTaskMonitor(); } - @After - public void tearDown() throws Exception - { - testingCluster.stop(); - remoteTaskRunner.stop(); - workerTaskMonitor.stop(); - } - - @Test - public void testRunNoExistingTask() throws Exception - { - remoteTaskRunner.run(task1); - } - - @Test - public void testExceptionThrownWithExistingTask() throws Exception - { - remoteTaskRunner.run( - new TestTask( - task1.getId(), - task1.getDataSource(), - task1.getSegments(), - Lists.newArrayList(), - TaskStatus.running(task1.getId()) - ) - ); - try { - remoteTaskRunner.run(task1); - fail("ISE expected"); - } - catch (ISE expected) { - } - } - - @Test - public void testRunTooMuchZKData() throws Exception - { - ServiceEmitter emitter = EasyMock.createMock(ServiceEmitter.class); - EmittingLogger.registerEmitter(emitter); - EasyMock.replay(emitter); - remoteTaskRunner.run( - new TestTask( - new String(new char[5000]), - "dummyDs", - Lists.newArrayList( - new DataSegment( - "dummyDs", - new Interval(new DateTime(), new DateTime()), - new DateTime().toString(), - null, - null, - null, - null, - 0, - 0 - ) - ), - Lists.newArrayList(), - TaskStatus.success("foo") - ) - ); - EasyMock.verify(emitter); - } - - @Test - public void testRunWithCallback() throws Exception - { - final MutableBoolean callbackCalled = new MutableBoolean(false); - - Futures.addCallback( - remoteTaskRunner.run( - new TestTask( - task1.getId(), - task1.getDataSource(), - task1.getSegments(), - Lists.newArrayList(), - TaskStatus.running(task1.getId()) - ) - ), new FutureCallback() - { - @Override - public void onSuccess(TaskStatus taskStatus) - { - callbackCalled.setValue(true); - } - - @Override - public void onFailure(Throwable throwable) - { - // neg - } - } - ); - - // 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()); - } - - - @Test - public void testRunSameAvailabilityGroup() throws Exception - { - TestRealtimeTask theTask = new TestRealtimeTask("rt1", "rt1", "foo", TaskStatus.running("rt1")); - remoteTaskRunner.run(theTask); - remoteTaskRunner.run( - new TestRealtimeTask("rt2", "rt1", "foo", TaskStatus.running("rt2")) - ); - remoteTaskRunner.run( - new TestRealtimeTask("rt3", "rt2", "foo", TaskStatus.running("rt3")) - ); - - Stopwatch stopwatch = new Stopwatch(); - stopwatch.start(); - while (remoteTaskRunner.getRunningTasks().isEmpty()) { - Thread.sleep(100); - if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > 1000) { - throw new ISE("Cannot find running task"); - } - } - - Assert.assertTrue(remoteTaskRunner.getRunningTasks().size() == 2); - Assert.assertTrue(remoteTaskRunner.getPendingTasks().size() == 1); - Assert.assertTrue(remoteTaskRunner.getPendingTasks().iterator().next().getTask().getId().equals("rt2")); - } - - private void makeTaskMonitor() throws Exception { - WorkerCuratorCoordinator workerCuratorCoordinator = new WorkerCuratorCoordinator( + workerCuratorCoordinator = new WorkerCuratorCoordinator( jsonMapper, new IndexerZkConfig() { - @Override - public String getIndexerAnnouncementPath() - { - return announcementsPath; - } - - @Override - public String getIndexerTaskPath() - { - return tasksPath; - } - - @Override - public String getIndexerStatusPath() - { - return statusPath; - } - @Override public String getZkBasePath() { - throw new UnsupportedOperationException(); + return basePath; } @Override @@ -314,13 +307,14 @@ public class RemoteTaskRunnerTest } }, cf, - worker1 + worker ); workerCuratorCoordinator.start(); + // Start a task monitor workerTaskMonitor = new WorkerTaskMonitor( jsonMapper, - new PathChildrenCache(cf, String.format("%s/worker1", tasksPath), true), + new PathChildrenCache(cf, tasksPath, true), cf, workerCuratorCoordinator, new ThreadPoolTaskRunner( @@ -328,14 +322,11 @@ public class RemoteTaskRunnerTest new TaskConfig() { @Override - public File getBaseTaskDir() + public String getBaseDir() { - try { - return File.createTempFile("billy", "yay"); - } - catch (Exception e) { - throw Throwables.propagate(e); - } + File tmp = Files.createTempDir(); + tmp.deleteOnExit(); + return tmp.toString(); } @Override @@ -349,7 +340,7 @@ public class RemoteTaskRunnerTest { return null; } - }, null, null, null, null, null, null, null, null, jsonMapper + }, null, null, null, null, null, null, null, null, null, jsonMapper ), Executors.newSingleThreadExecutor() ), Executors.newSingleThreadExecutor() @@ -361,80 +352,45 @@ public class RemoteTaskRunnerTest private void makeRemoteTaskRunner() throws Exception { - scheduledExec = EasyMock.createMock(ScheduledExecutorService.class); - remoteTaskRunner = new RemoteTaskRunner( jsonMapper, new TestRemoteTaskRunnerConfig(), cf, - pathChildrenCache, - scheduledExec, - new RetryPolicyFactory(new TestRetryPolicyConfig()), + new SimplePathChildrenCacheFactory.Builder().build(), new AtomicReference(new WorkerSetupData("0", 0, 1, null, null)), null ); - // Create a single worker and wait for things for be ready remoteTaskRunner.start(); - cf.create().creatingParentsIfNeeded().withMode(CreateMode.EPHEMERAL).forPath( - String.format("%s/worker1", announcementsPath), - jsonMapper.writeValueAsBytes(worker1) - ); - int count = 0; - while (remoteTaskRunner.getWorkers().size() == 0) { - Thread.sleep(500); - if (count > 10) { - throw new ISE("WTF?! Still can't find worker!"); - } - count++; - } } - private static class TestRetryPolicyConfig extends RetryPolicyConfig + private void makeWorker() throws Exception { - @Override - public Duration getRetryMinDuration() - { - return null; - } + worker = new Worker( + "worker", + "localhost", + 3, + "0" + ); - @Override - public Duration getRetryMaxDuration() - { - return null; - } - - @Override - public long getMaxRetryCount() - { - return 0; - } + cf.create().creatingParentsIfNeeded().withMode(CreateMode.EPHEMERAL).forPath( + announcementsPath, + jsonMapper.writeValueAsBytes(worker) + ); } private static class TestRemoteTaskRunnerConfig extends RemoteTaskRunnerConfig { @Override - public String getIndexerAnnouncementPath() + public boolean enableCompression() { - return announcementsPath; - } - - @Override - public String getIndexerTaskPath() - { - return tasksPath; - } - - @Override - public String getIndexerStatusPath() - { - return statusPath; + return false; } @Override public String getZkBasePath() { - throw new UnsupportedOperationException(); + return basePath; } @Override @@ -448,5 +404,11 @@ public class RemoteTaskRunnerTest { return 1000; } + + @Override + public String getWorkerVersion() + { + return ""; + } } } diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java index ef3c6412c6f..17e718a8983 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java @@ -36,6 +36,7 @@ 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.indexing.common.task.TaskResource; import com.metamx.druid.input.InputRow; import com.metamx.druid.input.MapBasedInputRow; import com.metamx.druid.jackson.DefaultObjectMapper; @@ -120,9 +121,9 @@ public class TaskLifecycleTest new TaskConfig() { @Override - public File getBaseTaskDir() + public String getBaseDir() { - return tmp; + return tmp.toString(); } @Override @@ -159,6 +160,7 @@ public class TaskLifecycleTest null, // segment announcer null, // new segment server view null, // query runner factory conglomerate corporation unionized collective + null, // monitor scheduler new DefaultObjectMapper() ); @@ -284,7 +286,7 @@ public class TaskLifecycleTest @Test public void testSimple() throws Exception { - final Task task = new AbstractTask("id1", "id1", "id1", "ds", new Interval("2012-01-01/P1D")) + final Task task = new AbstractTask("id1", "id1", new TaskResource("id1", 1), "ds", new Interval("2012-01-01/P1D")) { @Override public String getType() @@ -321,7 +323,7 @@ public class TaskLifecycleTest @Test public void testBadInterval() throws Exception { - final Task task = new AbstractTask("id1", "id1", "id1", "ds", new Interval("2012-01-01/P1D")) + final Task task = new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D")) { @Override public String getType() @@ -355,7 +357,7 @@ public class TaskLifecycleTest @Test public void testBadVersion() throws Exception { - final Task task = new AbstractTask("id1", "id1", "id1", "ds", new Interval("2012-01-01/P1D")) + final Task task = new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D")) { @Override public String getType() diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskQueueTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskQueueTest.java index 4e023b736dd..f421bee7f3c 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskQueueTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskQueueTest.java @@ -168,6 +168,7 @@ public class TaskQueueTest null, null, null, + null, null ); @@ -228,6 +229,7 @@ public class TaskQueueTest null, null, null, + null, null ); @@ -346,7 +348,7 @@ public class TaskQueueTest private static Task newTask(final String id, final String groupId, final String dataSource, final Interval interval) { - return new AbstractTask(id, groupId, id, dataSource, interval) + return new AbstractTask(id, groupId, dataSource, interval) { @Override public TaskStatus run(TaskToolbox toolbox) throws Exception @@ -370,7 +372,7 @@ public class TaskQueueTest final List nextTasks ) { - return new AbstractTask(id, groupId, id, dataSource, interval) + return new AbstractTask(id, groupId, dataSource, interval) { @Override public String getType() diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TestRealtimeTask.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TestRealtimeTask.java index 515b75e3e1f..9c30cf690d0 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TestRealtimeTask.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TestRealtimeTask.java @@ -27,6 +27,7 @@ import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.TaskToolbox; import com.metamx.druid.indexing.common.task.RealtimeIndexTask; +import com.metamx.druid.indexing.common.task.TaskResource; import com.metamx.druid.realtime.Schema; import com.metamx.druid.shard.NoneShardSpec; @@ -40,18 +41,19 @@ public class TestRealtimeTask extends RealtimeIndexTask @JsonCreator public TestRealtimeTask( @JsonProperty("id") String id, - @JsonProperty("availabilityGroup") String availGroup, + @JsonProperty("resource") TaskResource taskResource, @JsonProperty("dataSource") String dataSource, @JsonProperty("taskStatus") TaskStatus status ) { super( id, - availGroup, + taskResource, new Schema(dataSource, null, new AggregatorFactory[]{}, QueryGranularity.NONE, new NoneShardSpec()), null, null, null, + null, null ); this.status = status; diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategyTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategyTest.java index eba026ea620..8f512390f14 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategyTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategyTest.java @@ -84,6 +84,12 @@ public class EC2AutoScalingStrategyTest { return "8080"; } + + @Override + public String getWorkerVersion() + { + return ""; + } }, workerSetupData ); diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java index c44d555f798..f73d289526d 100644 --- a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java @@ -19,17 +19,20 @@ package com.metamx.druid.indexing.coordinator.scaling; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; import com.metamx.druid.indexing.TestTask; import com.metamx.druid.indexing.common.TaskStatus; import com.metamx.druid.indexing.common.task.Task; -import com.metamx.druid.indexing.coordinator.TaskRunnerWorkItem; +import com.metamx.druid.indexing.coordinator.RemoteTaskRunnerWorkItem; import com.metamx.druid.indexing.coordinator.ZkWorker; import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; import com.metamx.druid.indexing.worker.Worker; +import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEventBuilder; @@ -42,7 +45,9 @@ import org.junit.Before; import org.junit.Test; import java.util.Arrays; +import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; @@ -111,6 +116,12 @@ public class SimpleResourceManagementStrategyTest { return new Duration(0); } + + @Override + public String getWorkerVersion() + { + return ""; + } }, workerSetupData ); @@ -127,8 +138,8 @@ public class SimpleResourceManagementStrategyTest EasyMock.replay(autoScalingStrategy); boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( - Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime()) + Arrays.asList( + new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -155,8 +166,8 @@ public class SimpleResourceManagementStrategyTest EasyMock.replay(autoScalingStrategy); boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( - Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime()) + Arrays.asList( + new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -171,8 +182,8 @@ public class SimpleResourceManagementStrategyTest ); provisionedSomething = simpleResourceManagementStrategy.doProvision( - Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime()) + Arrays.asList( + new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -212,8 +223,8 @@ public class SimpleResourceManagementStrategyTest EasyMock.replay(autoScalingStrategy); boolean provisionedSomething = simpleResourceManagementStrategy.doProvision( - Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime()) + Arrays.asList( + new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -230,8 +241,8 @@ public class SimpleResourceManagementStrategyTest Thread.sleep(2000); provisionedSomething = simpleResourceManagementStrategy.doProvision( - Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime()) + Arrays.asList( + new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(testTask) @@ -264,8 +275,8 @@ public class SimpleResourceManagementStrategyTest EasyMock.replay(autoScalingStrategy); boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate( - Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime()) + Arrays.asList( + new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(null) @@ -294,8 +305,8 @@ public class SimpleResourceManagementStrategyTest EasyMock.replay(autoScalingStrategy); boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate( - Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime()) + Arrays.asList( + new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(null) @@ -309,8 +320,8 @@ public class SimpleResourceManagementStrategyTest ); terminatedSomething = simpleResourceManagementStrategy.doTerminate( - Arrays.asList( - new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime()) + Arrays.asList( + new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime()) ), Arrays.asList( new TestZkWorker(null) @@ -334,18 +345,18 @@ public class SimpleResourceManagementStrategyTest Task testTask ) { - super(new Worker("host", "ip", 3, "version"), null, null); + super(new Worker("host", "ip", 3, "version"), null, new DefaultObjectMapper()); this.testTask = testTask; } @Override - public Set getRunningTasks() + public Map getRunningTasks() { if (testTask == null) { - return Sets.newHashSet(); + return Maps.newHashMap(); } - return Sets.newHashSet(testTask.getId()); + return ImmutableMap.of(testTask.getId(), TaskStatus.running(testTask.getId())); } } } diff --git a/pom.xml b/pom.xml index 0ba44f14093..adae7dd623c 100644 --- a/pom.xml +++ b/pom.xml @@ -23,7 +23,7 @@ com.metamx druid pom - 0.5.19-SNAPSHOT + 0.5.33-SNAPSHOT druid druid @@ -38,7 +38,7 @@ UTF-8 - 0.22.3 + 0.22.6 2.1.0-incubating @@ -80,7 +80,7 @@ com.metamx server-metrics - 0.0.2 + 0.0.3 diff --git a/realtime/pom.xml b/realtime/pom.xml index aac1b971aa5..d7df1b1747d 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.19-SNAPSHOT + 0.5.33-SNAPSHOT diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeManager.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeManager.java index 0874a900f42..f40fd75998c 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeManager.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeManager.java @@ -164,32 +164,37 @@ public class RealtimeManager implements QuerySegmentWalker final InputRow inputRow; try { inputRow = firehose.nextRow(); - - final Sink sink = plumber.getSink(inputRow.getTimestampFromEpoch()); - if (sink == null) { - metrics.incrementThrownAway(); - log.debug("Throwing away event[%s]", inputRow); - - if (System.currentTimeMillis() > nextFlush) { - plumber.persist(firehose.commit()); - nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis(); - } - - continue; - } - - int currCount = sink.add(inputRow); - metrics.incrementProcessed(); - if (currCount >= config.getMaxRowsInMemory() || System.currentTimeMillis() > nextFlush) { - plumber.persist(firehose.commit()); - nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis(); - } } catch (FormattedException e) { log.info(e, "unparseable line: %s", e.getDetails()); metrics.incrementUnparseable(); continue; } + catch (Exception e) { + log.info(e, "thrown away line due to exception"); + metrics.incrementThrownAway(); + continue; + } + + final Sink sink = plumber.getSink(inputRow.getTimestampFromEpoch()); + if (sink == null) { + metrics.incrementThrownAway(); + log.debug("Throwing away event[%s]", inputRow); + + if (System.currentTimeMillis() > nextFlush) { + plumber.persist(firehose.commit()); + nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis(); + } + + continue; + } + + int currCount = sink.add(inputRow); + metrics.incrementProcessed(); + if (currCount >= config.getMaxRowsInMemory() || System.currentTimeMillis() > nextFlush) { + plumber.persist(firehose.commit()); + nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis(); + } } } catch (RuntimeException e) { log.makeAlert(e, "RuntimeException aborted realtime processing[%s]", fireDepartment.getSchema().getDataSource()) diff --git a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeMetricsMonitor.java b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeMetricsMonitor.java index 3d04392aab0..69d1e0169fa 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/RealtimeMetricsMonitor.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/RealtimeMetricsMonitor.java @@ -31,7 +31,7 @@ import java.util.Map; */ public class RealtimeMetricsMonitor extends AbstractMonitor { - Map previousValues; + private final Map previousValues; private final List fireDepartments; public RealtimeMetricsMonitor(List fireDepartments) diff --git a/realtime/src/main/java/com/metamx/druid/realtime/firehose/GracefulShutdownFirehose.java b/realtime/src/main/java/com/metamx/druid/realtime/firehose/GracefulShutdownFirehose.java deleted file mode 100644 index 4cb95384661..00000000000 --- a/realtime/src/main/java/com/metamx/druid/realtime/firehose/GracefulShutdownFirehose.java +++ /dev/null @@ -1,142 +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.realtime.firehose; - -import com.google.common.base.Throwables; -import com.google.common.util.concurrent.ThreadFactoryBuilder; -import com.metamx.common.concurrent.ScheduledExecutors; -import com.metamx.common.logger.Logger; -import com.metamx.druid.index.v1.IndexGranularity; -import com.metamx.druid.input.InputRow; -import com.metamx.druid.realtime.plumber.IntervalRejectionPolicyFactory; -import com.metamx.druid.realtime.plumber.RejectionPolicy; -import org.joda.time.DateTime; -import org.joda.time.Duration; -import org.joda.time.Interval; -import org.joda.time.Period; - -import java.io.IOException; -import java.util.concurrent.Callable; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.atomic.AtomicBoolean; - -/** - */ -public class GracefulShutdownFirehose implements Firehose -{ - private static final Logger log = new Logger(GracefulShutdownFirehose.class); - - private final Firehose firehose; - private final IndexGranularity segmentGranularity; - private final long windowMillis; - private final ScheduledExecutorService scheduledExecutor; - private final RejectionPolicy rejectionPolicy; - - // when this is set to false, the firehose will have no more rows - private final AtomicBoolean valveOn = new AtomicBoolean(true); - - // when this is set to true, the firehose will begin rejecting events - private volatile boolean beginRejectionPolicy = false; - - public GracefulShutdownFirehose( - Firehose firehose, - IndexGranularity segmentGranularity, - Period windowPeriod - ) - { - this.firehose = firehose; - this.segmentGranularity = segmentGranularity; - this.windowMillis = windowPeriod.toStandardDuration().getMillis() * 2; - this.scheduledExecutor = Executors.newScheduledThreadPool( - 1, - new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat("firehose_scheduled_%d") - .build() - ); - - final long truncatedNow = segmentGranularity.truncate(new DateTime()).getMillis(); - final long end = segmentGranularity.increment(truncatedNow); - - this.rejectionPolicy = new IntervalRejectionPolicyFactory(new Interval(truncatedNow, end)).create(windowPeriod); - } - - public void shutdown() throws IOException - { - final long truncatedNow = segmentGranularity.truncate(new DateTime()).getMillis(); - final long end = segmentGranularity.increment(truncatedNow) + windowMillis; - final Duration timeUntilShutdown = new Duration(System.currentTimeMillis(), end); - - log.info("Shutdown at approx. %s (in %s)", new DateTime(end), timeUntilShutdown); - - ScheduledExecutors.scheduleWithFixedDelay( - scheduledExecutor, - timeUntilShutdown, - new Callable() - { - @Override - public ScheduledExecutors.Signal call() throws Exception - { - try { - valveOn.set(false); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - - return ScheduledExecutors.Signal.STOP; - } - } - ); - - beginRejectionPolicy = true; - } - - @Override - public boolean hasMore() - { - return valveOn.get() && firehose.hasMore(); - } - - @Override - public InputRow nextRow() - { - InputRow next = firehose.nextRow(); - - if (!beginRejectionPolicy || rejectionPolicy.accept(next.getTimestampFromEpoch())) { - return next; - } - - return null; - } - - @Override - public Runnable commit() - { - return firehose.commit(); - } - - @Override - public void close() throws IOException - { - firehose.close(); - } -} diff --git a/realtime/src/main/java/com/metamx/druid/realtime/plumber/IntervalRejectionPolicyFactory.java b/realtime/src/main/java/com/metamx/druid/realtime/plumber/IntervalRejectionPolicyFactory.java deleted file mode 100644 index c13a03dafb1..00000000000 --- a/realtime/src/main/java/com/metamx/druid/realtime/plumber/IntervalRejectionPolicyFactory.java +++ /dev/null @@ -1,42 +0,0 @@ -package com.metamx.druid.realtime.plumber; - -import org.joda.time.DateTime; -import org.joda.time.Interval; -import org.joda.time.Period; - -/** - */ -public class IntervalRejectionPolicyFactory implements RejectionPolicyFactory -{ - private final Interval interval; - - public IntervalRejectionPolicyFactory(Interval interval) - { - this.interval = interval; - } - - @Override - public RejectionPolicy create(Period windowPeriod) - { - return new RejectionPolicy() - { - @Override - public DateTime getCurrMaxTime() - { - return new DateTime(); - } - - @Override - public boolean accept(long timestamp) - { - return interval.contains(timestamp); - } - - @Override - public String toString() - { - return String.format("interval-%s", interval); - } - }; - } -} diff --git a/realtime/src/main/java/com/metamx/druid/realtime/plumber/NoopRejectionPolicyFactory.java b/realtime/src/main/java/com/metamx/druid/realtime/plumber/NoopRejectionPolicyFactory.java new file mode 100644 index 00000000000..59a3e24cb21 --- /dev/null +++ b/realtime/src/main/java/com/metamx/druid/realtime/plumber/NoopRejectionPolicyFactory.java @@ -0,0 +1,26 @@ +package com.metamx.druid.realtime.plumber; + +import org.joda.time.DateTime; +import org.joda.time.Period; + +public class NoopRejectionPolicyFactory implements RejectionPolicyFactory +{ + @Override + public RejectionPolicy create(Period windowPeriod) + { + return new RejectionPolicy() + { + @Override + public DateTime getCurrMaxTime() + { + return new DateTime(0); + } + + @Override + public boolean accept(long timestamp) + { + return true; + } + }; + } +} diff --git a/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java b/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java index a429fbef9d5..08383bc6456 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/plumber/RealtimePlumberSchool.java @@ -25,6 +25,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; +import com.google.common.base.Predicates; import com.google.common.base.Throwables; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -37,6 +38,8 @@ import com.metamx.common.Pair; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.guava.FunctionalIterable; import com.metamx.druid.Query; +import com.metamx.druid.TimelineObjectHolder; +import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.DruidServer; import com.metamx.druid.client.ServerView; @@ -50,6 +53,7 @@ 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.DataSegmentPusher; +import com.metamx.druid.partition.SingleElementPartitionChunk; import com.metamx.druid.query.MetricsEmittingQueryRunner; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryRunnerFactory; @@ -186,6 +190,9 @@ public class RealtimePlumberSchool implements PlumberSchool private volatile ScheduledExecutorService scheduledExecutor = null; private final Map sinks = Maps.newConcurrentMap(); + private final VersionedIntervalTimeline sinkTimeline = new VersionedIntervalTimeline( + String.CASE_INSENSITIVE_ORDER + ); @Override public void startJob() @@ -219,6 +226,7 @@ public class RealtimePlumberSchool implements PlumberSchool try { segmentAnnouncer.announceSegment(retVal.getSegment()); sinks.put(truncatedTime, retVal); + sinkTimeline.add(retVal.getInterval(), retVal.getVersion(), new SingleElementPartitionChunk(retVal)); } catch (IOException e) { log.makeAlert(e, "Failed to announce new segment[%s]", schema.getDataSource()) @@ -247,17 +255,23 @@ public class RealtimePlumberSchool implements PlumberSchool } }; + List> querySinks = Lists.newArrayList(); + for (Interval interval : query.getIntervals()) { + querySinks.addAll(sinkTimeline.lookup(interval)); + } + return toolchest.mergeResults( factory.mergeRunners( EXEC, FunctionalIterable - .create(sinks.values()) + .create(querySinks) .transform( - new Function>() + new Function, QueryRunner>() { @Override - public QueryRunner apply(Sink input) + public QueryRunner apply(TimelineObjectHolder holder) { + final Sink theSink = holder.getObject().getChunk(0).getObject(); return new SpecificSegmentQueryRunner( new MetricsEmittingQueryRunner( emitter, @@ -265,7 +279,7 @@ public class RealtimePlumberSchool implements PlumberSchool factory.mergeRunners( EXEC, Iterables.transform( - input, + theSink, new Function>() { @Override @@ -279,9 +293,9 @@ public class RealtimePlumberSchool implements PlumberSchool ), new SpecificSegmentSpec( new SegmentDescriptor( - input.getInterval(), - input.getSegment().getVersion(), - input.getSegment().getShardSpec().getPartitionNum() + holder.getInterval(), + theSink.getSegment().getVersion(), + theSink.getSegment().getShardSpec().getPartitionNum() ) ) ); @@ -319,11 +333,90 @@ public class RealtimePlumberSchool implements PlumberSchool ); } + // Submits persist-n-merge task for a Sink to the persistExecutor + private void persistAndMerge(final long truncatedTime, final Sink sink) + { + final String threadName = String.format( + "%s-%s-persist-n-merge", schema.getDataSource(), new DateTime(truncatedTime) + ); + persistExecutor.execute( + new ThreadRenamingRunnable(threadName) + { + @Override + public void doRun() + { + final Interval interval = sink.getInterval(); + + for (FireHydrant hydrant : sink) { + if (!hydrant.hasSwapped()) { + log.info("Hydrant[%s] hasn't swapped yet, swapping. Sink[%s]", hydrant, sink); + final int rowCount = persistHydrant(hydrant, schema, interval); + metrics.incrementRowOutputCount(rowCount); + } + } + + final File mergedTarget = new File(computePersistDir(schema, interval), "merged"); + if (mergedTarget.exists()) { + log.info("Skipping already-merged sink: %s", sink); + return; + } + + File mergedFile = null; + try { + List indexes = Lists.newArrayList(); + for (FireHydrant fireHydrant : sink) { + Segment segment = fireHydrant.getSegment(); + final QueryableIndex queryableIndex = segment.asQueryableIndex(); + log.info("Adding hydrant[%s]", fireHydrant); + indexes.add(queryableIndex); + } + + mergedFile = IndexMerger.mergeQueryableIndex( + indexes, + schema.getAggregators(), + mergedTarget + ); + + QueryableIndex index = IndexIO.loadIndex(mergedFile); + + DataSegment segment = dataSegmentPusher.push( + mergedFile, + sink.getSegment().withDimensions(Lists.newArrayList(index.getAvailableDimensions())) + ); + + segmentPublisher.publishSegment(segment); + } + catch (IOException e) { + log.makeAlert(e, "Failed to persist merged index[%s]", schema.getDataSource()) + .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); + } + } + } + } + ); + } + @Override public void finishJob() { log.info("Shutting down..."); + for (final Map.Entry entry : sinks.entrySet()) { + persistAndMerge(entry.getKey(), entry.getValue()); + } + while (!sinks.isEmpty()) { try { log.info( @@ -442,6 +535,11 @@ public class RealtimePlumberSchool implements PlumberSchool Sink currSink = new Sink(sinkInterval, schema, versioningPolicy.getVersion(sinkInterval), hydrants); sinks.put(sinkInterval.getStartMillis(), currSink); + sinkTimeline.add( + currSink.getInterval(), + currSink.getVersion(), + new SingleElementPartitionChunk(currSink) + ); segmentAnnouncer.announceSegment(currSink.getSegment()); } @@ -490,6 +588,11 @@ public class RealtimePlumberSchool implements PlumberSchool FileUtils.deleteDirectory(computePersistDir(schema, sink.getInterval())); log.info("Removing sinkKey %d for segment %s", sinkKey, sink.getSegment().getIdentifier()); sinks.remove(sinkKey); + sinkTimeline.remove( + sink.getInterval(), + sink.getVersion(), + new SingleElementPartitionChunk(sink) + ); synchronized (handoffCondition) { handoffCondition.notifyAll(); @@ -560,72 +663,7 @@ public class RealtimePlumberSchool implements PlumberSchool } for (final Map.Entry entry : sinksToPush) { - final Sink sink = entry.getValue(); - - final String threadName = String.format( - "%s-%s-persist-n-merge", schema.getDataSource(), new DateTime(entry.getKey()) - ); - persistExecutor.execute( - new ThreadRenamingRunnable(threadName) - { - @Override - public void doRun() - { - final Interval interval = sink.getInterval(); - - for (FireHydrant hydrant : sink) { - if (!hydrant.hasSwapped()) { - log.info("Hydrant[%s] hasn't swapped yet, swapping. Sink[%s]", hydrant, sink); - final int rowCount = persistHydrant(hydrant, schema, interval); - metrics.incrementRowOutputCount(rowCount); - } - } - - File mergedFile = null; - try { - List indexes = Lists.newArrayList(); - for (FireHydrant fireHydrant : sink) { - Segment segment = fireHydrant.getSegment(); - final QueryableIndex queryableIndex = segment.asQueryableIndex(); - log.info("Adding hydrant[%s]", fireHydrant); - indexes.add(queryableIndex); - } - - mergedFile = IndexMerger.mergeQueryableIndex( - indexes, - schema.getAggregators(), - new File(computePersistDir(schema, interval), "merged") - ); - - QueryableIndex index = IndexIO.loadIndex(mergedFile); - - DataSegment segment = dataSegmentPusher.push( - mergedFile, - sink.getSegment().withDimensions(Lists.newArrayList(index.getAvailableDimensions())) - ); - - segmentPublisher.publishSegment(segment); - } - catch (IOException e) { - log.makeAlert(e, "Failed to persist merged index[%s]", schema.getDataSource()) - .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); - } - } - } - } - ); + persistAndMerge(entry.getKey(), entry.getValue()); } if (stopped) { diff --git a/realtime/src/main/java/com/metamx/druid/realtime/plumber/RejectionPolicyFactory.java b/realtime/src/main/java/com/metamx/druid/realtime/plumber/RejectionPolicyFactory.java index 40e8e496bf6..b47b5c11eac 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/plumber/RejectionPolicyFactory.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/plumber/RejectionPolicyFactory.java @@ -7,7 +7,8 @@ import org.joda.time.Period; @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @JsonSubTypes.Type(name = "serverTime", value = ServerTimeRejectionPolicyFactory.class), - @JsonSubTypes.Type(name = "messageTime", value = MessageTimeRejectionPolicyFactory.class) + @JsonSubTypes.Type(name = "messageTime", value = MessageTimeRejectionPolicyFactory.class), + @JsonSubTypes.Type(name = "none", value = NoopRejectionPolicyFactory.class) }) public interface RejectionPolicyFactory { diff --git a/realtime/src/main/java/com/metamx/druid/realtime/plumber/Sink.java b/realtime/src/main/java/com/metamx/druid/realtime/plumber/Sink.java index a5dd4ae38a1..a1823b6c09a 100644 --- a/realtime/src/main/java/com/metamx/druid/realtime/plumber/Sink.java +++ b/realtime/src/main/java/com/metamx/druid/realtime/plumber/Sink.java @@ -90,6 +90,11 @@ public class Sink implements Iterable makeNewCurrIndex(interval.getStartMillis(), schema); } + public String getVersion() + { + return version; + } + public Interval getInterval() { return interval; diff --git a/server/pom.xml b/server/pom.xml index 3d611f96f03..0da15b0ba7b 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.5.19-SNAPSHOT + 0.5.33-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 7803475d091..8740349c779 100644 --- a/server/src/main/java/com/metamx/druid/coordination/ServerManager.java +++ b/server/src/main/java/com/metamx/druid/coordination/ServerManager.java @@ -24,11 +24,14 @@ import com.google.common.base.Predicates; import com.google.common.collect.Ordering; import com.metamx.common.ISE; import com.metamx.common.guava.FunctionalIterable; +import com.metamx.common.guava.Sequence; import com.metamx.druid.Query; import com.metamx.druid.TimelineObjectHolder; import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.DataSegment; import com.metamx.druid.collect.CountingMap; +import com.metamx.druid.index.ReferenceCountingSegment; +import com.metamx.druid.index.ReferenceCountingSequence; import com.metamx.druid.index.Segment; import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.loading.SegmentLoadingException; @@ -53,6 +56,7 @@ import com.metamx.emitter.service.ServiceMetricEvent; import org.joda.time.Interval; import javax.annotation.Nullable; +import java.io.IOException; import java.util.Arrays; import java.util.HashMap; import java.util.Map; @@ -71,7 +75,7 @@ public class ServerManager implements QuerySegmentWalker private final ServiceEmitter emitter; private final ExecutorService exec; - private final Map> dataSources; + private final Map> dataSources; private final CountingMap dataSourceSizes = new CountingMap(); private final CountingMap dataSourceCounts = new CountingMap(); @@ -88,7 +92,7 @@ public class ServerManager implements QuerySegmentWalker this.exec = exec; - this.dataSources = new HashMap>(); + this.dataSources = new HashMap>(); } public Map getDataSourceSizes() @@ -132,14 +136,14 @@ public class ServerManager implements QuerySegmentWalker synchronized (lock) { String dataSource = segment.getDataSource(); - VersionedIntervalTimeline loadedIntervals = dataSources.get(dataSource); + VersionedIntervalTimeline loadedIntervals = dataSources.get(dataSource); if (loadedIntervals == null) { - loadedIntervals = new VersionedIntervalTimeline(Ordering.natural()); + loadedIntervals = new VersionedIntervalTimeline(Ordering.natural()); dataSources.put(dataSource, loadedIntervals); } - PartitionHolder entry = loadedIntervals.findEntry( + PartitionHolder entry = loadedIntervals.findEntry( segment.getInterval(), segment.getVersion() ); @@ -149,7 +153,9 @@ public class ServerManager implements QuerySegmentWalker } loadedIntervals.add( - segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(adapter) + segment.getInterval(), + segment.getVersion(), + segment.getShardSpec().createChunk(new ReferenceCountingSegment(adapter)) ); synchronized (dataSourceSizes) { dataSourceSizes.add(dataSource, segment.getSize()); @@ -164,17 +170,19 @@ public class ServerManager implements QuerySegmentWalker { String dataSource = segment.getDataSource(); synchronized (lock) { - VersionedIntervalTimeline loadedIntervals = dataSources.get(dataSource); + VersionedIntervalTimeline loadedIntervals = dataSources.get(dataSource); if (loadedIntervals == null) { log.info("Told to delete a queryable for a dataSource[%s] that doesn't exist.", dataSource); return; } - PartitionChunk removed = loadedIntervals.remove( - segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk((Segment) null) + PartitionChunk removed = loadedIntervals.remove( + segment.getInterval(), + segment.getVersion(), + segment.getShardSpec().createChunk((ReferenceCountingSegment) null) ); - Segment oldQueryable = (removed == null) ? null : removed.getObject(); + ReferenceCountingSegment oldQueryable = (removed == null) ? null : removed.getObject(); if (oldQueryable != null) { synchronized (dataSourceSizes) { @@ -183,6 +191,16 @@ public class ServerManager implements QuerySegmentWalker synchronized (dataSourceCounts) { dataSourceCounts.add(dataSource, -1L); } + + try { + oldQueryable.close(); + } + catch (IOException e) { + log.makeAlert(e, "Exception closing segment") + .addData("dataSource", dataSource) + .addData("segmentId", segment.getIdentifier()) + .emit(); + } } else { log.info( "Told to delete a queryable on dataSource[%s] for interval[%s] and version [%s] that I don't have.", @@ -205,7 +223,7 @@ public class ServerManager implements QuerySegmentWalker final QueryToolChest> toolChest = factory.getToolchest(); - final VersionedIntervalTimeline timeline = dataSources.get(query.getDataSource()); + final VersionedIntervalTimeline timeline = dataSources.get(query.getDataSource()); if (timeline == null) { return new NoopQueryRunner(); @@ -214,20 +232,22 @@ public class ServerManager implements QuerySegmentWalker FunctionalIterable> adapters = FunctionalIterable .create(intervals) .transformCat( - new Function>>() + new Function>>() { @Override - public Iterable> apply(Interval input) + public Iterable> apply(Interval input) { return timeline.lookup(input); } } ) .transformCat( - new Function, Iterable>>() + new Function, Iterable>>() { @Override - public Iterable> apply(@Nullable final TimelineObjectHolder holder) + public Iterable> apply( + @Nullable final TimelineObjectHolder holder + ) { if (holder == null) { return null; @@ -236,10 +256,10 @@ public class ServerManager implements QuerySegmentWalker return FunctionalIterable .create(holder.getObject()) .transform( - new Function, QueryRunner>() + new Function, QueryRunner>() { @Override - public QueryRunner apply(PartitionChunk input) + public QueryRunner apply(PartitionChunk input) { return buildAndDecorateQueryRunner( factory, @@ -280,7 +300,7 @@ public class ServerManager implements QuerySegmentWalker final QueryToolChest> toolChest = factory.getToolchest(); - final VersionedIntervalTimeline timeline = dataSources.get(query.getDataSource()); + final VersionedIntervalTimeline timeline = dataSources.get(query.getDataSource()); if (timeline == null) { return new NoopQueryRunner(); @@ -293,9 +313,9 @@ public class ServerManager implements QuerySegmentWalker { @Override @SuppressWarnings("unchecked") - public Iterable> apply(@Nullable SegmentDescriptor input) + public Iterable> apply(SegmentDescriptor input) { - final PartitionHolder entry = timeline.findEntry( + final PartitionHolder entry = timeline.findEntry( input.getInterval(), input.getVersion() ); @@ -303,12 +323,12 @@ public class ServerManager implements QuerySegmentWalker return null; } - final PartitionChunk chunk = entry.getChunk(input.getPartitionNumber()); + final PartitionChunk chunk = entry.getChunk(input.getPartitionNumber()); if (chunk == null) { return null; } - final Segment adapter = chunk.getObject(); + final ReferenceCountingSegment adapter = chunk.getObject(); return Arrays.asList( buildAndDecorateQueryRunner(factory, toolChest, adapter, new SpecificSegmentSpec(input)) ); @@ -323,10 +343,10 @@ public class ServerManager implements QuerySegmentWalker } private QueryRunner buildAndDecorateQueryRunner( - QueryRunnerFactory> factory, + final QueryRunnerFactory> factory, final QueryToolChest> toolChest, - Segment adapter, - QuerySegmentSpec segmentSpec + final ReferenceCountingSegment adapter, + final QuerySegmentSpec segmentSpec ) { return new SpecificSegmentQueryRunner( @@ -335,7 +355,7 @@ public class ServerManager implements QuerySegmentWalker new Function, ServiceMetricEvent.Builder>() { @Override - public ServiceMetricEvent.Builder apply(@Nullable Query input) + public ServiceMetricEvent.Builder apply(@Nullable final Query input) { return toolChest.makeMetricBuilder(input); } @@ -343,10 +363,17 @@ public class ServerManager implements QuerySegmentWalker new BySegmentQueryRunner( adapter.getIdentifier(), adapter.getDataInterval().getStart(), - factory.createRunner(adapter) + new QueryRunner() + { + @Override + public Sequence run(final Query query) + { + return new ReferenceCountingSequence(factory.createRunner(adapter).run(query), adapter); + } + } ) ).withWaitMeasuredFromNow(), segmentSpec ); } -} +} \ No newline at end of file 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 63521717e79..a1db1bf0888 100644 --- a/server/src/main/java/com/metamx/druid/http/MasterMain.java +++ b/server/src/main/java/com/metamx/druid/http/MasterMain.java @@ -49,6 +49,7 @@ import com.metamx.druid.db.DatabaseSegmentManager; import com.metamx.druid.db.DatabaseSegmentManagerConfig; import com.metamx.druid.db.DbConnector; import com.metamx.druid.db.DbConnectorConfig; +import com.metamx.druid.initialization.CuratorConfig; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ServiceDiscoveryConfig; @@ -124,10 +125,15 @@ public class MasterMain final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle); final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class); - CuratorFramework curatorFramework = Initialization.makeCuratorFramework( + CuratorFramework serviceDiscoveryCuratorFramework = Initialization.makeCuratorFramework( serviceDiscoveryConfig, lifecycle ); + final CuratorConfig curatorConfig = configFactory.build(CuratorConfig.class); + CuratorFramework curatorFramework = Initialization.makeCuratorFramework( + curatorConfig, + lifecycle + ); final ZkPathsConfig zkPaths = configFactory.build(ZkPathsConfig.class); @@ -201,7 +207,7 @@ public class MasterMain final DruidMasterConfig druidMasterConfig = configFactory.build(DruidMasterConfig.class); final ServiceDiscovery serviceDiscovery = Initialization.makeServiceDiscoveryClient( - curatorFramework, + serviceDiscoveryCuratorFramework, serviceDiscoveryConfig, lifecycle ); @@ -227,7 +233,10 @@ public class MasterMain ); final LoadQueueTaskMaster taskMaster = new LoadQueueTaskMaster( - curatorFramework, jsonMapper, Execs.singleThreaded("Master-PeonExec--%d") + curatorFramework, + jsonMapper, + scheduledExecutorFactory.create(1, "Master-PeonExec--%d"), + druidMasterConfig ); final DruidMaster master = new DruidMaster( diff --git a/server/src/main/java/com/metamx/druid/index/IncrementalIndexSegment.java b/server/src/main/java/com/metamx/druid/index/IncrementalIndexSegment.java index 79c7dc069b6..6e2d312aa75 100644 --- a/server/src/main/java/com/metamx/druid/index/IncrementalIndexSegment.java +++ b/server/src/main/java/com/metamx/druid/index/IncrementalIndexSegment.java @@ -24,6 +24,8 @@ import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.index.v1.IncrementalIndexStorageAdapter; import org.joda.time.Interval; +import java.io.IOException; + /** */ public class IncrementalIndexSegment implements Segment @@ -60,4 +62,10 @@ public class IncrementalIndexSegment implements Segment { return new IncrementalIndexStorageAdapter(index); } + + @Override + public void close() throws IOException + { + // do nothing + } } diff --git a/server/src/main/java/com/metamx/druid/index/QueryableIndexSegment.java b/server/src/main/java/com/metamx/druid/index/QueryableIndexSegment.java index 770eb783d3b..118dd9ab285 100644 --- a/server/src/main/java/com/metamx/druid/index/QueryableIndexSegment.java +++ b/server/src/main/java/com/metamx/druid/index/QueryableIndexSegment.java @@ -23,6 +23,8 @@ import com.metamx.druid.StorageAdapter; import com.metamx.druid.index.v1.QueryableIndexStorageAdapter; import org.joda.time.Interval; +import java.io.IOException; + /** */ public class QueryableIndexSegment implements Segment @@ -59,4 +61,11 @@ public class QueryableIndexSegment implements Segment { return new QueryableIndexStorageAdapter(index); } + + @Override + public void close() throws IOException + { + // this is kinda nasty + index.close(); + } } diff --git a/server/src/main/java/com/metamx/druid/index/ReferenceCountingSegment.java b/server/src/main/java/com/metamx/druid/index/ReferenceCountingSegment.java new file mode 100644 index 00000000000..74fe1ebd180 --- /dev/null +++ b/server/src/main/java/com/metamx/druid/index/ReferenceCountingSegment.java @@ -0,0 +1,167 @@ +/* + * 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.index; + +import com.metamx.druid.StorageAdapter; +import com.metamx.emitter.EmittingLogger; +import org.joda.time.Interval; + +import java.io.Closeable; +import java.io.IOException; +import java.util.concurrent.atomic.AtomicBoolean; + +public class ReferenceCountingSegment implements Segment +{ + private static final EmittingLogger log = new EmittingLogger(ReferenceCountingSegment.class); + + private final Segment baseSegment; + + private final Object lock = new Object(); + + private volatile int numReferences = 0; + private volatile boolean isClosed = false; + + public ReferenceCountingSegment(Segment baseSegment) + { + this.baseSegment = baseSegment; + } + + public Segment getBaseSegment() + { + synchronized (lock) { + if (!isClosed) { + return baseSegment; + } + + return null; + } + } + + public int getNumReferences() + { + return numReferences; + } + + public boolean isClosed() + { + return isClosed; + } + + @Override + public String getIdentifier() + { + synchronized (lock) { + if (!isClosed) { + return baseSegment.getIdentifier(); + } + + return null; + } + } + + @Override + public Interval getDataInterval() + { + synchronized (lock) { + if (!isClosed) { + return baseSegment.getDataInterval(); + } + + return null; + } + } + + @Override + public QueryableIndex asQueryableIndex() + { + synchronized (lock) { + if (!isClosed) { + return baseSegment.asQueryableIndex(); + } + + return null; + } + } + + @Override + public StorageAdapter asStorageAdapter() + { + synchronized (lock) { + if (!isClosed) { + return baseSegment.asStorageAdapter(); + } + + return null; + } + } + + @Override + public void close() throws IOException + { + synchronized (lock) { + if (!isClosed) { + if (numReferences > 0) { + decrement(); + } else { + baseSegment.close(); + isClosed = true; + } + } + } + } + + public Closeable increment() + { + synchronized (lock) { + if (!isClosed) { + numReferences++; + final AtomicBoolean decrementOnce = new AtomicBoolean(false); + return new Closeable() + { + @Override + public void close() throws IOException + { + if (decrementOnce.compareAndSet(false, true)) { + decrement(); + } + } + }; + } + + return null; + } + } + + private void decrement() + { + synchronized (lock) { + if (!isClosed) { + if (--numReferences < 0) { + try { + close(); + } + catch (Exception e) { + log.error("Unable to close queryable index %s", getIdentifier()); + } + } + } + } + } +} \ No newline at end of file diff --git a/server/src/main/java/com/metamx/druid/index/ReferenceCountingSequence.java b/server/src/main/java/com/metamx/druid/index/ReferenceCountingSequence.java new file mode 100644 index 00000000000..ca7066ececb --- /dev/null +++ b/server/src/main/java/com/metamx/druid/index/ReferenceCountingSequence.java @@ -0,0 +1,48 @@ +/* + * 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.index; + +import com.metamx.common.guava.ResourceClosingYielder; +import com.metamx.common.guava.Sequence; +import com.metamx.common.guava.Yielder; +import com.metamx.common.guava.YieldingAccumulator; +import com.metamx.common.guava.YieldingSequenceBase; + +/** + */ +public class ReferenceCountingSequence extends YieldingSequenceBase +{ + private final Sequence baseSequence; + private final ReferenceCountingSegment segment; + + public ReferenceCountingSequence(Sequence baseSequence, ReferenceCountingSegment segment) + { + this.baseSequence = baseSequence; + this.segment = segment; + } + + @Override + public Yielder toYielder( + OutType initValue, YieldingAccumulator accumulator + ) + { + return new ResourceClosingYielder(baseSequence.toYielder(initValue, accumulator), segment.increment()); + } +} \ No newline at end of file diff --git a/server/src/main/java/com/metamx/druid/index/Segment.java b/server/src/main/java/com/metamx/druid/index/Segment.java index b2edda90f8d..9b9e6c94cd1 100644 --- a/server/src/main/java/com/metamx/druid/index/Segment.java +++ b/server/src/main/java/com/metamx/druid/index/Segment.java @@ -22,9 +22,11 @@ package com.metamx.druid.index; import com.metamx.druid.StorageAdapter; import org.joda.time.Interval; +import java.io.Closeable; + /** */ -public interface Segment +public interface Segment extends Closeable { public String getIdentifier(); public Interval getDataInterval(); diff --git a/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java index e72bd787bb3..349194836d9 100644 --- a/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java +++ b/server/src/main/java/com/metamx/druid/loading/DataSegmentPusherUtil.java @@ -19,6 +19,8 @@ package com.metamx.druid.loading; +import org.joda.time.format.ISODateTimeFormat; + import com.google.common.base.Joiner; import com.metamx.druid.client.DataSegment; @@ -26,19 +28,37 @@ import com.metamx.druid.client.DataSegment; */ public class DataSegmentPusherUtil { - private static final Joiner JOINER = Joiner.on("/").skipNulls(); + 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() - ); - } + 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() + ); + } + + /** + * Due to https://issues.apache.org/jira/browse/HDFS-13 ":" are not allowed in + * path names. So we format paths differently for HDFS. + */ + public static String getHdfsStorageDir(DataSegment segment) + { + return JOINER.join( + segment.getDataSource(), + String.format( + "%s_%s", + segment.getInterval().getStart().toString(ISODateTimeFormat.basicDateTime()), + segment.getInterval().getEnd().toString(ISODateTimeFormat.basicDateTime()) + ), + segment.getVersion().replaceAll(":", "_"), + segment.getShardSpec().getPartitionNum() + ); + } } diff --git a/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPusher.java index 52ac15129d4..e232bbb8f08 100644 --- a/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/HdfsDataSegmentPusher.java @@ -42,7 +42,7 @@ public class HdfsDataSegmentPusher implements DataSegmentPusher @Override public DataSegment push(File inDir, DataSegment segment) throws IOException { - final String storageDir = DataSegmentPusherUtil.getStorageDir(segment); + final String storageDir = DataSegmentPusherUtil.getHdfsStorageDir(segment); Path outFile = new Path(String.format("%s/%s/index.zip", config.getStorageDirectory(), storageDir)); FileSystem fs = outFile.getFileSystem(hadoopConfig); 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 276bbc2028a..535649baea4 100644 --- a/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java +++ b/server/src/main/java/com/metamx/druid/loading/QueryableIndexFactory.java @@ -22,6 +22,7 @@ package com.metamx.druid.loading; import com.metamx.druid.index.QueryableIndex; import java.io.File; +import java.io.IOException; /** */ diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java index 9dacbe8b546..d9ac69e5c5b 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusher.java @@ -80,7 +80,9 @@ public class S3DataSegmentPusher implements DataSegmentPusher final String outputBucket = config.getBucket(); toPush.setBucketName(outputBucket); toPush.setKey(outputKey + "/index.zip"); - toPush.setAcl(GSAccessControlList.REST_CANNED_BUCKET_OWNER_FULL_CONTROL); + if (!config.getDisableAcl()) { + toPush.setAcl(GSAccessControlList.REST_CANNED_BUCKET_OWNER_FULL_CONTROL); + } log.info("Pushing %s.", toPush); s3Client.putObject(outputBucket, toPush); @@ -96,7 +98,9 @@ public class S3DataSegmentPusher implements DataSegmentPusher S3Object descriptorObject = new S3Object(descriptorFile); descriptorObject.setBucketName(outputBucket); descriptorObject.setKey(outputKey + "/descriptor.json"); - descriptorObject.setAcl(GSAccessControlList.REST_CANNED_BUCKET_OWNER_FULL_CONTROL); + if (!config.getDisableAcl()) { + descriptorObject.setAcl(GSAccessControlList.REST_CANNED_BUCKET_OWNER_FULL_CONTROL); + } log.info("Pushing %s", descriptorObject); s3Client.putObject(outputBucket, descriptorObject); diff --git a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java index a2cada422fb..3fbbe2d311f 100644 --- a/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java +++ b/server/src/main/java/com/metamx/druid/loading/S3DataSegmentPusherConfig.java @@ -32,4 +32,8 @@ public abstract class S3DataSegmentPusherConfig @Config("druid.pusher.s3.baseKey") @Default("") public abstract String getBaseKey(); + + @Config("druid.pusher.s3.disableAcl") + @Default("false") + public abstract boolean getDisableAcl(); } diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterConfig.java b/server/src/main/java/com/metamx/druid/master/DruidMasterConfig.java index b4f3db315a4..e6605e096b0 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterConfig.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterConfig.java @@ -94,4 +94,10 @@ public abstract class DruidMasterConfig @Config("druid.master.replicant.throttleLimit") @Default("10") public abstract int getReplicantThrottleLimit(); + + @Config("druid.master.load.timeout") + public Duration getLoadTimeoutDelay() + { + return new Duration(15 * 60 * 1000); + } } diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterLogger.java b/server/src/main/java/com/metamx/druid/master/DruidMasterLogger.java index 382d03966b6..1b226933a52 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterLogger.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterLogger.java @@ -185,6 +185,12 @@ public class DruidMasterLogger implements DruidMasterHelper "master/loadQueue/size", queuePeon.getLoadQueueSize() ) ); + emitter.emit( + new ServiceMetricEvent.Builder() + .setUser1(serverName).build( + "master/loadQueue/failed", queuePeon.getAndResetFailedAssignCount() + ) + ); emitter.emit( new ServiceMetricEvent.Builder() .setUser1(serverName).build( diff --git a/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java b/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java index 8be132b9400..e46c2a06f51 100644 --- a/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java +++ b/server/src/main/java/com/metamx/druid/master/DruidMasterSegmentMerger.java @@ -38,6 +38,7 @@ import com.metamx.druid.VersionedIntervalTimeline; import com.metamx.druid.client.DataSegment; import com.metamx.druid.client.indexing.IndexingServiceClient; import com.metamx.druid.partition.PartitionChunk; +import com.metamx.druid.shard.NoneShardSpec; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -98,16 +99,8 @@ public class DruidMasterSegmentMerger implements DruidMasterHelper SegmentsToMerge segmentsToMerge = new SegmentsToMerge(); for (int i = 0; i < timelineObjects.size(); i++) { - - try { - segmentsToMerge.add(timelineObjects.get(i)); - } - catch (Exception e) { - log.error("Unable to merge segments for %s", entry.getKey()); - throw Throwables.propagate(e); - } - - if (segmentsToMerge.getByteCount() > params.getMergeBytesLimit() + if (!segmentsToMerge.add(timelineObjects.get(i)) + || segmentsToMerge.getByteCount() > params.getMergeBytesLimit() || segmentsToMerge.getSegmentCount() >= params.getMergeSegmentsLimit()) { i -= segmentsToMerge.backtrack(params.getMergeBytesLimit()); @@ -216,7 +209,7 @@ public class DruidMasterSegmentMerger implements DruidMasterHelper ).asList(); } - public void add(TimelineObjectHolder timelineObject) + public boolean add(TimelineObjectHolder timelineObject) { final Interval timelineObjectInterval = timelineObject.getInterval(); @@ -235,6 +228,10 @@ public class DruidMasterSegmentMerger implements DruidMasterHelper Interval underlyingInterval = firstChunk.getObject().getInterval(); for (final PartitionChunk segment : timelineObject.getObject()) { + if (!(segment.getObject().getShardSpec() instanceof NoneShardSpec)) { + return false; + } + segments.add(segment.getObject()); if (segments.count(segment.getObject()) == 1) { byteCount += segment.getObject().getSize(); @@ -256,6 +253,8 @@ public class DruidMasterSegmentMerger implements DruidMasterHelper timelineObjects.add(Pair.of(timelineObject, new Interval(start, end))); } + + return true; } public Interval getMergedTimelineInterval() diff --git a/server/src/main/java/com/metamx/druid/master/LoadQueuePeon.java b/server/src/main/java/com/metamx/druid/master/LoadQueuePeon.java index c4ce6ac5a8c..2b5c96cb187 100644 --- a/server/src/main/java/com/metamx/druid/master/LoadQueuePeon.java +++ b/server/src/main/java/com/metamx/druid/master/LoadQueuePeon.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.collect.Collections2; import com.google.common.collect.Lists; +import com.metamx.common.ISE; import com.metamx.common.guava.Comparators; import com.metamx.druid.client.DataSegment; import com.metamx.druid.coordination.DataSegmentChangeRequest; @@ -43,7 +44,9 @@ import java.util.Comparator; import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentSkipListSet; -import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; /** @@ -54,15 +57,6 @@ public class LoadQueuePeon private static final int DROP = 0; private static final int LOAD = 1; - private final Object lock = new Object(); - - private final CuratorFramework curator; - private final String basePath; - private final ObjectMapper jsonMapper; - private final ExecutorService zkWritingExecutor; - - private final AtomicLong queuedSize = new AtomicLong(0); - private static Comparator segmentHolderComparator = new Comparator() { private Comparator comparator = Comparators.inverse(DataSegment.bucketMonthComparator()); @@ -74,6 +68,15 @@ public class LoadQueuePeon } }; + private final CuratorFramework curator; + private final String basePath; + private final ObjectMapper jsonMapper; + private final ScheduledExecutorService zkWritingExecutor; + private final DruidMasterConfig config; + + private final AtomicLong queuedSize = new AtomicLong(0); + private final AtomicInteger failedAssignCount = new AtomicInteger(0); + private final ConcurrentSkipListSet segmentsToLoad = new ConcurrentSkipListSet( segmentHolderComparator ); @@ -81,19 +84,23 @@ public class LoadQueuePeon segmentHolderComparator ); + private final Object lock = new Object(); + private volatile SegmentHolder currentlyLoading = null; LoadQueuePeon( CuratorFramework curator, String basePath, ObjectMapper jsonMapper, - ExecutorService zkWritingExecutor + ScheduledExecutorService zkWritingExecutor, + DruidMasterConfig config ) { this.curator = curator; this.basePath = basePath; this.jsonMapper = jsonMapper; this.zkWritingExecutor = zkWritingExecutor; + this.config = config; } public Set getSegmentsToLoad() @@ -135,6 +142,11 @@ public class LoadQueuePeon return queuedSize.get(); } + public int getAndResetFailedAssignCount() + { + return failedAssignCount.getAndSet(0); + } + public void loadSegment( DataSegment segment, LoadPeonCallback callback @@ -232,6 +244,26 @@ public class LoadQueuePeon final byte[] payload = jsonMapper.writeValueAsBytes(currentlyLoading.getChangeRequest()); curator.create().withMode(CreateMode.EPHEMERAL).forPath(path, payload); + zkWritingExecutor.schedule( + new Runnable() + { + @Override + public void run() + { + try { + if (curator.checkExists().forPath(path) != null) { + failAssign(new ISE("%s was never removed! Failing this assign!", path)); + } + } + catch (Exception e) { + failAssign(e); + } + } + }, + config.getLoadTimeoutDelay().getMillis(), + TimeUnit.MILLISECONDS + ); + final Stat stat = curator.checkExists().usingWatcher( new CuratorWatcher() { @@ -268,10 +300,7 @@ public class LoadQueuePeon } } catch (Exception e) { - log.error(e, "Server[%s], throwable caught when submitting [%s].", basePath, currentlyLoading); - // Act like it was completed so that the master gives it to someone else - actionCompleted(); - doNext(); + failAssign(e); } } } @@ -327,6 +356,7 @@ public class LoadQueuePeon segmentsToLoad.clear(); queuedSize.set(0L); + failedAssignCount.set(0); } } @@ -351,6 +381,17 @@ public class LoadQueuePeon doNext(); } + private void failAssign(Exception e) + { + synchronized (lock) { + log.error(e, "Server[%s], throwable caught when submitting [%s].", basePath, currentlyLoading); + failedAssignCount.getAndIncrement(); + // Act like it was completed so that the master gives it to someone else + actionCompleted(); + doNext(); + } + } + private class SegmentHolder { private final DataSegment segment; diff --git a/server/src/main/java/com/metamx/druid/master/LoadQueueTaskMaster.java b/server/src/main/java/com/metamx/druid/master/LoadQueueTaskMaster.java index 9ef5b61e5a0..2547127bc5a 100644 --- a/server/src/main/java/com/metamx/druid/master/LoadQueueTaskMaster.java +++ b/server/src/main/java/com/metamx/druid/master/LoadQueueTaskMaster.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.curator.framework.CuratorFramework; import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; /** * Provides LoadQueuePeons @@ -31,21 +32,24 @@ public class LoadQueueTaskMaster { private final CuratorFramework curator; private final ObjectMapper jsonMapper; - private final ExecutorService peonExec; + private final ScheduledExecutorService peonExec; + private final DruidMasterConfig config; public LoadQueueTaskMaster( CuratorFramework curator, ObjectMapper jsonMapper, - ExecutorService peonExec + ScheduledExecutorService peonExec, + DruidMasterConfig config ) { this.curator = curator; this.jsonMapper = jsonMapper; this.peonExec = peonExec; + this.config = config; } public LoadQueuePeon giveMePeon(String basePath) { - return new LoadQueuePeon(curator, basePath, jsonMapper, peonExec); + return new LoadQueuePeon(curator, basePath, jsonMapper, peonExec, config); } } 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 c557b839b03..196cc1edf61 100644 --- a/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java +++ b/server/src/test/java/com/metamx/druid/coordination/ServerManagerTest.java @@ -22,27 +22,31 @@ package com.metamx.druid.coordination; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import com.google.common.util.concurrent.MoreExecutors; +import com.metamx.common.IAE; import com.metamx.common.MapUtils; 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.common.guava.Yielder; +import com.metamx.common.guava.YieldingAccumulator; +import com.metamx.common.guava.YieldingSequenceBase; import com.metamx.druid.Druids; import com.metamx.druid.Query; import com.metamx.druid.QueryGranularity; import com.metamx.druid.StorageAdapter; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.QueryableIndex; +import com.metamx.druid.index.ReferenceCountingSegment; import com.metamx.druid.index.Segment; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.loading.SegmentLoader; import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.metrics.NoopServiceEmitter; -import com.metamx.druid.query.CacheStrategy; import com.metamx.druid.query.ConcatQueryRunner; import com.metamx.druid.query.MetricManipulationFn; import com.metamx.druid.query.NoopQueryRunner; @@ -56,7 +60,6 @@ import com.metamx.druid.result.SearchResultValue; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceMetricEvent; - import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; @@ -66,21 +69,30 @@ import java.io.IOException; import java.util.Arrays; import java.util.Iterator; import java.util.List; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; /** */ public class ServerManagerTest { - ServerManager serverManager; - MyQueryRunnerFactory factory; + private ServerManager serverManager; + private MyQueryRunnerFactory factory; + private CountDownLatch queryWaitLatch; + private CountDownLatch queryNotifyLatch; + private ExecutorService serverManagerExec; @Before public void setUp() throws IOException { EmittingLogger.registerEmitter(new NoopServiceEmitter()); - factory = new MyQueryRunnerFactory(); + queryWaitLatch = new CountDownLatch(1); + queryNotifyLatch = new CountDownLatch(1); + factory = new MyQueryRunnerFactory(queryWaitLatch, queryNotifyLatch); + serverManagerExec = Executors.newFixedThreadPool(2); serverManager = new ServerManager( new SegmentLoader() { @@ -114,7 +126,7 @@ public class ServerManagerTest } }, new NoopServiceEmitter(), - MoreExecutors.sameThreadExecutor() + serverManagerExec ); loadQueryable("test", "1", new Interval("P1d/2011-04-01")); @@ -135,7 +147,7 @@ public class ServerManagerTest @Test public void testSimpleGet() { - assertQueryable( + Future future = assertQueryable( QueryGranularity.DAY, "test", new Interval("P1d/2011-04-01"), @@ -143,8 +155,10 @@ public class ServerManagerTest new Pair("1", new Interval("P1d/2011-04-01")) ) ); + waitForTestVerificationAndCleanup(future); - assertQueryable( + + future = assertQueryable( QueryGranularity.DAY, "test", new Interval("P2d/2011-04-02"), ImmutableList.>of( @@ -152,6 +166,7 @@ public class ServerManagerTest new Pair("2", new Interval("P1d/2011-04-02")) ) ); + waitForTestVerificationAndCleanup(future); } @Test @@ -160,22 +175,24 @@ public class ServerManagerTest final String dataSouce = "test"; final Interval interval = new Interval("2011-04-01/2011-04-02"); - assertQueryable( + Future future = assertQueryable( QueryGranularity.DAY, dataSouce, interval, ImmutableList.>of( new Pair("2", interval) ) ); + waitForTestVerificationAndCleanup(future); dropQueryable(dataSouce, "2", interval); - assertQueryable( + future = assertQueryable( QueryGranularity.DAY, dataSouce, interval, ImmutableList.>of( new Pair("1", interval) ) ); + waitForTestVerificationAndCleanup(future); } @Test @@ -183,18 +200,19 @@ public class ServerManagerTest { loadQueryable("test", "3", new Interval("2011-04-04/2011-04-05")); - assertQueryable( + Future future = assertQueryable( QueryGranularity.DAY, "test", new Interval("2011-04-04/2011-04-06"), ImmutableList.>of( new Pair("3", new Interval("2011-04-04/2011-04-05")) ) ); + waitForTestVerificationAndCleanup(future); dropQueryable("test", "3", new Interval("2011-04-04/2011-04-05")); dropQueryable("test", "1", new Interval("2011-04-04/2011-04-05")); - assertQueryable( + future = assertQueryable( QueryGranularity.HOUR, "test", new Interval("2011-04-04/2011-04-04T06"), ImmutableList.>of( @@ -205,8 +223,9 @@ public class ServerManagerTest new Pair("2", new Interval("2011-04-04T05/2011-04-04T06")) ) ); + waitForTestVerificationAndCleanup(future); - assertQueryable( + future = assertQueryable( QueryGranularity.HOUR, "test", new Interval("2011-04-04/2011-04-04T03"), ImmutableList.>of( @@ -215,8 +234,9 @@ public class ServerManagerTest new Pair("2", new Interval("2011-04-04T02/2011-04-04T03")) ) ); + waitForTestVerificationAndCleanup(future); - assertQueryable( + future = assertQueryable( QueryGranularity.HOUR, "test", new Interval("2011-04-04T04/2011-04-04T06"), ImmutableList.>of( @@ -224,9 +244,171 @@ public class ServerManagerTest new Pair("2", new Interval("2011-04-04T05/2011-04-04T06")) ) ); + waitForTestVerificationAndCleanup(future); } - private void loadQueryable(String dataSource, String version, Interval interval) throws IOException + @Test + public void testReferenceCounting() throws Exception + { + loadQueryable("test", "3", new Interval("2011-04-04/2011-04-05")); + + Future future = assertQueryable( + QueryGranularity.DAY, + "test", new Interval("2011-04-04/2011-04-06"), + ImmutableList.>of( + new Pair("3", new Interval("2011-04-04/2011-04-05")) + ) + ); + + queryNotifyLatch.await(); + + Assert.assertTrue(factory.getAdapters().size() == 1); + + for (SegmentForTesting segmentForTesting : factory.getAdapters()) { + Assert.assertFalse(segmentForTesting.isClosed()); + } + + queryWaitLatch.countDown(); + future.get(); + + dropQueryable("test", "3", new Interval("2011-04-04/2011-04-05")); + + for (SegmentForTesting segmentForTesting : factory.getAdapters()) { + Assert.assertTrue(segmentForTesting.isClosed()); + } + } + + @Test + public void testReferenceCountingWhileQueryExecuting() throws Exception + { + loadQueryable("test", "3", new Interval("2011-04-04/2011-04-05")); + + Future future = assertQueryable( + QueryGranularity.DAY, + "test", new Interval("2011-04-04/2011-04-06"), + ImmutableList.>of( + new Pair("3", new Interval("2011-04-04/2011-04-05")) + ) + ); + + queryNotifyLatch.await(); + + Assert.assertTrue(factory.getAdapters().size() == 1); + + for (SegmentForTesting segmentForTesting : factory.getAdapters()) { + Assert.assertFalse(segmentForTesting.isClosed()); + } + + dropQueryable("test", "3", new Interval("2011-04-04/2011-04-05")); + + for (SegmentForTesting segmentForTesting : factory.getAdapters()) { + Assert.assertFalse(segmentForTesting.isClosed()); + } + + queryWaitLatch.countDown(); + future.get(); + + for (SegmentForTesting segmentForTesting : factory.getAdapters()) { + Assert.assertTrue(segmentForTesting.isClosed()); + } + } + + @Test + public void testMultipleDrops() throws Exception + { + loadQueryable("test", "3", new Interval("2011-04-04/2011-04-05")); + + Future future = assertQueryable( + QueryGranularity.DAY, + "test", new Interval("2011-04-04/2011-04-06"), + ImmutableList.>of( + new Pair("3", new Interval("2011-04-04/2011-04-05")) + ) + ); + + queryNotifyLatch.await(); + + Assert.assertTrue(factory.getAdapters().size() == 1); + + for (SegmentForTesting segmentForTesting : factory.getAdapters()) { + Assert.assertFalse(segmentForTesting.isClosed()); + } + + dropQueryable("test", "3", new Interval("2011-04-04/2011-04-05")); + dropQueryable("test", "3", new Interval("2011-04-04/2011-04-05")); + + for (SegmentForTesting segmentForTesting : factory.getAdapters()) { + Assert.assertFalse(segmentForTesting.isClosed()); + } + + queryWaitLatch.countDown(); + future.get(); + + for (SegmentForTesting segmentForTesting : factory.getAdapters()) { + Assert.assertTrue(segmentForTesting.isClosed()); + } + } + + private void waitForTestVerificationAndCleanup(Future future) + { + try { + queryNotifyLatch.await(); + queryWaitLatch.countDown(); + future.get(); + factory.clearAdapters(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + private Future assertQueryable( + QueryGranularity granularity, + String dataSource, + Interval interval, + List> expected + ) + { + final Iterator> expectedIter = expected.iterator(); + final List intervals = Arrays.asList(interval); + final SearchQuery query = Druids.newSearchQueryBuilder() + .dataSource(dataSource) + .intervals(intervals) + .granularity(granularity) + .limit(10000) + .query("wow") + .build(); + final QueryRunner> runner = serverManager.getQueryRunnerForIntervals( + query, + intervals + ); + + return serverManagerExec.submit( + new Runnable() + { + @Override + public void run() + { + Sequence> seq = runner.run(query); + Sequences.toList(seq, Lists.>newArrayList()); + Iterator adaptersIter = factory.getAdapters().iterator(); + + while (expectedIter.hasNext() && adaptersIter.hasNext()) { + Pair expectedVals = expectedIter.next(); + SegmentForTesting value = adaptersIter.next(); + + Assert.assertEquals(expectedVals.lhs, value.getVersion()); + Assert.assertEquals(expectedVals.rhs, value.getInterval()); + } + + Assert.assertFalse(expectedIter.hasNext()); + Assert.assertFalse(adaptersIter.hasNext()); + } + } + ); + } + + public void loadQueryable(String dataSource, String version, Interval interval) throws IOException { try { serverManager.loadSegment( @@ -248,7 +430,7 @@ public class ServerManagerTest } } - private void dropQueryable(String dataSource, String version, Interval interval) + public void dropQueryable(String dataSource, String version, Interval interval) { try { serverManager.dropSegment( @@ -270,99 +452,30 @@ public class ServerManagerTest } } - private void assertQueryable( - QueryGranularity granularity, - String dataSource, - Interval interval, - List> expected - ) - { - Iterator> expectedIter = expected.iterator(); - final List intervals = Arrays.asList(interval); - final SearchQuery query = Druids.newSearchQueryBuilder() - .dataSource(dataSource) - .intervals(intervals) - .granularity(granularity) - .limit(10000) - .query("wow") - .build(); - QueryRunner> runner = serverManager.getQueryRunnerForIntervals(query, intervals); - final Sequence> seq = runner.run(query); - Sequences.toList(seq, Lists.>newArrayList()); - Iterator adaptersIter = factory.getAdapters().iterator(); - - while (expectedIter.hasNext() && adaptersIter.hasNext()) { - Pair expectedVals = expectedIter.next(); - SegmentForTesting value = adaptersIter.next(); - - Assert.assertEquals(expectedVals.lhs, value.getVersion()); - Assert.assertEquals(expectedVals.rhs, value.getInterval()); - } - - Assert.assertFalse(expectedIter.hasNext()); - Assert.assertFalse(adaptersIter.hasNext()); - - factory.clearAdapters(); - } - - private static class SegmentForTesting implements Segment - { - private final String version; - private final Interval interval; - - SegmentForTesting( - String version, - Interval interval - ) - { - this.version = version; - this.interval = interval; - } - - public String getVersion() - { - return version; - } - - public Interval getInterval() - { - return interval; - } - - @Override - public String getIdentifier() - { - return version; - } - - @Override - public Interval getDataInterval() - { - return interval; - } - - @Override - public QueryableIndex asQueryableIndex() - { - throw new UnsupportedOperationException(); - } - - @Override - public StorageAdapter asStorageAdapter() - { - throw new UnsupportedOperationException(); - } - } - public static class MyQueryRunnerFactory implements QueryRunnerFactory, SearchQuery> { + private final CountDownLatch waitLatch; + private final CountDownLatch notifyLatch; private List adapters = Lists.newArrayList(); + public MyQueryRunnerFactory(CountDownLatch waitLatch, CountDownLatch notifyLatch) + { + this.waitLatch = waitLatch; + this.notifyLatch = notifyLatch; + } + @Override public QueryRunner> createRunner(Segment adapter) { - adapters.add((SegmentForTesting) adapter); - return new NoopQueryRunner>(); + if (!(adapter instanceof ReferenceCountingSegment)) { + throw new IAE("Expected instance of ReferenceCountingSegment, got %s", adapter.getClass()); + } + adapters.add((SegmentForTesting) ((ReferenceCountingSegment) adapter).getBaseSegment()); + return new BlockingQueryRunner>( + new NoopQueryRunner>(), + waitLatch, + notifyLatch + ); } @Override @@ -419,7 +532,159 @@ public class ServerManagerTest @Override public TypeReference getResultTypeReference() { - return new TypeReference(){}; + return new TypeReference() + { + }; + } + } + + private static class SegmentForTesting implements Segment + { + private final String version; + private final Interval interval; + + private final Object lock = new Object(); + + private volatile boolean closed = false; + + SegmentForTesting( + String version, + Interval interval + ) + { + this.version = version; + this.interval = interval; + } + + public String getVersion() + { + return version; + } + + public Interval getInterval() + { + return interval; + } + + @Override + public String getIdentifier() + { + return version; + } + + public boolean isClosed() + { + return closed; + } + + @Override + public Interval getDataInterval() + { + return interval; + } + + @Override + public QueryableIndex asQueryableIndex() + { + throw new UnsupportedOperationException(); + } + + @Override + public StorageAdapter asStorageAdapter() + { + throw new UnsupportedOperationException(); + } + + @Override + public void close() throws IOException + { + synchronized (lock) { + closed = true; + } + } + } + + private static class BlockingQueryRunner implements QueryRunner + { + private final QueryRunner runner; + private final CountDownLatch waitLatch; + private final CountDownLatch notifyLatch; + + public BlockingQueryRunner( + QueryRunner runner, + CountDownLatch waitLatch, + CountDownLatch notifyLatch + ) + { + this.runner = runner; + this.waitLatch = waitLatch; + this.notifyLatch = notifyLatch; + } + + @Override + public Sequence run(Query query) + { + return new BlockingSequence(runner.run(query), waitLatch, notifyLatch); + } + } + + private static class BlockingSequence extends YieldingSequenceBase + { + private final Sequence baseSequence; + private final CountDownLatch waitLatch; + private final CountDownLatch notifyLatch; + + public BlockingSequence( + Sequence baseSequence, + CountDownLatch waitLatch, + CountDownLatch notifyLatch + ) + { + this.baseSequence = baseSequence; + this.waitLatch = waitLatch; + this.notifyLatch = notifyLatch; + } + + @Override + public Yielder toYielder( + final OutType initValue, final YieldingAccumulator accumulator + ) + { + notifyLatch.countDown(); + + final Yielder baseYielder = baseSequence.toYielder(initValue, accumulator); + return new Yielder() + { + @Override + public OutType get() + { + try { + waitLatch.await(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + return baseYielder.get(); + } + + @Override + public Yielder next(OutType initValue) + { + return baseYielder.next(initValue); + } + + @Override + public boolean isDone() + { + return baseYielder.isDone(); + } + + @Override + public void close() throws IOException + { + baseYielder.close(); + } + }; } } } diff --git a/server/src/test/java/com/metamx/druid/index/ReferenceCountingSegmentTest.java b/server/src/test/java/com/metamx/druid/index/ReferenceCountingSegmentTest.java new file mode 100644 index 00000000000..a9b6c398a57 --- /dev/null +++ b/server/src/test/java/com/metamx/druid/index/ReferenceCountingSegmentTest.java @@ -0,0 +1,137 @@ +/* + * 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.index; + +import com.google.common.base.Throwables; +import com.metamx.druid.StorageAdapter; +import junit.framework.Assert; +import org.joda.time.Interval; +import org.junit.Before; +import org.junit.Test; + +import java.io.Closeable; +import java.io.IOException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +/** + */ +public class ReferenceCountingSegmentTest +{ + private ReferenceCountingSegment segment; + private ExecutorService exec; + + @Before + public void setUp() throws Exception + { + segment = new ReferenceCountingSegment( + new Segment() + { + @Override + public String getIdentifier() + { + throw new UnsupportedOperationException(); + } + + @Override + public Interval getDataInterval() + { + throw new UnsupportedOperationException(); + } + + @Override + public QueryableIndex asQueryableIndex() + { + throw new UnsupportedOperationException(); + } + + @Override + public StorageAdapter asStorageAdapter() + { + throw new UnsupportedOperationException(); + } + + @Override + public void close() throws IOException + { + } + } + ); + + exec = Executors.newSingleThreadExecutor(); + } + + @Test + public void testMultipleClose() throws Exception + { + Assert.assertFalse(segment.isClosed()); + final Closeable closeable = segment.increment(); + Assert.assertTrue(segment.getNumReferences() == 1); + + closeable.close(); + closeable.close(); + exec.submit( + new Runnable() + { + @Override + public void run() + { + try { + closeable.close(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + } + ); + Assert.assertTrue(segment.getNumReferences() == 0); + Assert.assertFalse(segment.isClosed()); + + segment.close(); + segment.close(); + exec.submit( + new Runnable() + { + @Override + public void run() + { + try { + segment.close(); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + } + ); + + Assert.assertTrue(segment.getNumReferences() == 0); + Assert.assertTrue(segment.isClosed()); + + segment.increment(); + segment.increment(); + segment.increment(); + Assert.assertTrue(segment.getNumReferences() == 0); + + segment.close(); + Assert.assertTrue(segment.getNumReferences() == 0); + } +} diff --git a/server/src/test/java/com/metamx/druid/loading/CacheTestSegmentLoader.java b/server/src/test/java/com/metamx/druid/loading/CacheTestSegmentLoader.java index e9be482116e..ebf6ae09b10 100644 --- a/server/src/test/java/com/metamx/druid/loading/CacheTestSegmentLoader.java +++ b/server/src/test/java/com/metamx/druid/loading/CacheTestSegmentLoader.java @@ -27,6 +27,7 @@ import com.metamx.druid.index.Segment; import org.joda.time.Interval; import java.io.File; +import java.io.IOException; import java.util.Map; /** @@ -68,6 +69,11 @@ public class CacheTestSegmentLoader implements SegmentLoader { throw new UnsupportedOperationException(); } + + @Override + public void close() throws IOException + { + } }; } diff --git a/server/src/test/java/com/metamx/druid/loading/DataSegmentPusherUtilTest.java b/server/src/test/java/com/metamx/druid/loading/DataSegmentPusherUtilTest.java new file mode 100644 index 00000000000..f367fceab47 --- /dev/null +++ b/server/src/test/java/com/metamx/druid/loading/DataSegmentPusherUtilTest.java @@ -0,0 +1,39 @@ +package com.metamx.druid.loading; + +import com.google.common.collect.ImmutableMap; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.index.v1.IndexIO; +import com.metamx.druid.shard.NoneShardSpec; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; + +/** + * @author jan.rudert + */ +public class DataSegmentPusherUtilTest { + @Test + public void shouldNotHaveColonsInHdfsStorageDir() throws Exception { + + Interval interval = new Interval("2011-10-01/2011-10-02"); + ImmutableMap loadSpec = ImmutableMap.of("something", "or_other"); + + DataSegment segment = new DataSegment( + "something", + interval, + "brand:new:version", + loadSpec, + Arrays.asList("dim1", "dim2"), + Arrays.asList("met1", "met2"), + new NoneShardSpec(), + IndexIO.CURRENT_VERSION_ID, + 1 + ); + + String storageDir = DataSegmentPusherUtil.getHdfsStorageDir(segment); + Assert.assertEquals("something/20111001T000000.000Z_20111002T000000.000Z/brand_new_version/0", storageDir); + + } +} diff --git a/server/src/test/java/com/metamx/druid/master/DruidMasterSegmentMergerTest.java b/server/src/test/java/com/metamx/druid/master/DruidMasterSegmentMergerTest.java index 44cac2ad19e..f8bbdcc63f8 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterSegmentMergerTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterSegmentMergerTest.java @@ -397,7 +397,45 @@ public class DruidMasterSegmentMergerTest ); Assert.assertEquals( - ImmutableList.of(ImmutableList.of(segments.get(0), segments.get(1), segments.get(2))), + ImmutableList.of(), + merge(segments) + ); + } + + @Test + public void testMergeMixedShardSpecs() + { + final List segments = ImmutableList.of( + DataSegment.builder() + .dataSource("foo") + .interval(new Interval("2012-01-01/P1D")) + .version("1") + .build(), + DataSegment.builder() + .dataSource("foo") + .interval(new Interval("2012-01-02/P1D")) + .version("1") + .build(), + DataSegment.builder().dataSource("foo") + .interval(new Interval("2012-01-03/P1D")) + .version("1") + .shardSpec(new LinearShardSpec(1500)) + .build(), + DataSegment.builder().dataSource("foo") + .interval(new Interval("2012-01-04/P1D")) + .version("1") + .build(), + DataSegment.builder().dataSource("foo") + .interval(new Interval("2012-01-05/P1D")) + .version("1") + .build() + ); + + Assert.assertEquals( + ImmutableList.of( + ImmutableList.of(segments.get(0), segments.get(1)), + ImmutableList.of(segments.get(3), segments.get(4)) + ), merge(segments) ); } 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 a49dc85a582..213e350f76b 100644 --- a/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java +++ b/server/src/test/java/com/metamx/druid/master/DruidMasterTest.java @@ -36,6 +36,7 @@ import org.junit.Before; import org.junit.Test; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ScheduledExecutorService; /** */ diff --git a/server/src/test/java/com/metamx/druid/master/LoadQueuePeonTester.java b/server/src/test/java/com/metamx/druid/master/LoadQueuePeonTester.java index b0e000d20ad..366cde87e1d 100644 --- a/server/src/test/java/com/metamx/druid/master/LoadQueuePeonTester.java +++ b/server/src/test/java/com/metamx/druid/master/LoadQueuePeonTester.java @@ -10,7 +10,7 @@ public class LoadQueuePeonTester extends LoadQueuePeon public LoadQueuePeonTester() { - super(null, null, null, null); + super(null, null, null, null, null); } @Override diff --git a/server/src/test/java/com/metamx/druid/metrics/NoopServiceEmitter.java b/server/src/test/java/com/metamx/druid/metrics/NoopServiceEmitter.java index 54c158495f9..f980bc4885c 100644 --- a/server/src/test/java/com/metamx/druid/metrics/NoopServiceEmitter.java +++ b/server/src/test/java/com/metamx/druid/metrics/NoopServiceEmitter.java @@ -22,7 +22,8 @@ package com.metamx.druid.metrics; import com.metamx.emitter.core.Event; import com.metamx.emitter.service.ServiceEmitter; -public class NoopServiceEmitter extends ServiceEmitter +public class + NoopServiceEmitter extends ServiceEmitter { public NoopServiceEmitter() { diff --git a/services/pom.xml b/services/pom.xml index cb73a90eb89..657f1f4048a 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.5.19-SNAPSHOT + 0.5.33-SNAPSHOT com.metamx druid - 0.5.19-SNAPSHOT + 0.5.33-SNAPSHOT