mirror of https://github.com/apache/druid.git
merged with master
This commit is contained in:
commit
eed9e1b373
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.5.23-SNAPSHOT</version>
|
||||
<version>0.5.33-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -62,9 +62,7 @@ public class SingleServerInventoryView extends ServerInventoryView<DataSegment>
|
|||
curator,
|
||||
exec,
|
||||
jsonMapper,
|
||||
new TypeReference<DataSegment>()
|
||||
{
|
||||
}
|
||||
new TypeReference<DataSegment>(){}
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.5.23-SNAPSHOT</version>
|
||||
<version>0.5.33-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -61,6 +61,11 @@ public class PartitionHolder<T> implements Iterable<PartitionChunk<T>>
|
|||
holderSet.add(chunk);
|
||||
}
|
||||
|
||||
public int size()
|
||||
{
|
||||
return holderSet.size();
|
||||
}
|
||||
|
||||
public PartitionChunk<T> remove(PartitionChunk<T> chunk)
|
||||
{
|
||||
// Somewhat funky implementation in order to return the removed object as it exists in the set
|
||||
|
|
|
@ -9,7 +9,7 @@
|
|||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.5.23-SNAPSHOT</version>
|
||||
<version>0.5.33-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.5.23-SNAPSHOT</version>
|
||||
<version>0.5.33-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -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<String> getColumnNames();
|
||||
public Indexed<String> 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;
|
||||
}
|
||||
|
|
|
@ -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<String> availableDimensions;
|
||||
private final Column timeColumn;
|
||||
private final Map<String, Column> otherColumns;
|
||||
private final SmooshedFileMapper fileMapper;
|
||||
|
||||
public SimpleQueryableIndex(
|
||||
Interval dataInterval,
|
||||
Indexed<String> columnNames,
|
||||
Indexed<String> dimNames,
|
||||
Column timeColumn,
|
||||
Map<String, Column> otherColumns
|
||||
Map<String, Column> 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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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<String, VSizeIndexed> dimColumns;
|
||||
final Map<String, GenericIndexed<ImmutableConciseSet>> invertedIndexes;
|
||||
final Map<String, ImmutableRTree> spatialIndexes;
|
||||
final SmooshedFileMapper fileMapper;
|
||||
|
||||
private final Map<String, Integer> metricIndexes = Maps.newHashMap();
|
||||
|
||||
|
@ -69,7 +71,8 @@ public class MMappedIndex
|
|||
Map<String, GenericIndexed<String>> dimValueLookups,
|
||||
Map<String, VSizeIndexed> dimColumns,
|
||||
Map<String, GenericIndexed<ImmutableConciseSet>> invertedIndexes,
|
||||
Map<String, ImmutableRTree> spatialIndexes
|
||||
Map<String, ImmutableRTree> 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
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<T>
|
||||
{
|
||||
public InputRow parse(T input);
|
||||
public InputRow parse(T input) throws FormattedException;
|
||||
public void addDimensionExclusion(String dimension);
|
||||
}
|
||||
|
|
|
@ -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<Map<String, Object>>
|
|||
}
|
||||
|
||||
@Override
|
||||
public InputRow parse(Map<String, Object> theMap)
|
||||
public InputRow parse(Map<String, Object> theMap) throws FormattedException
|
||||
{
|
||||
final List<String> 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);
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.5.23-SNAPSHOT</version>
|
||||
<version>0.5.33-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -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
|
||||
{
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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> T jsonReadWriteRead(String s, Class<T> klass)
|
||||
{
|
||||
try {
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.5.23-SNAPSHOT</version>
|
||||
<version>0.5.33-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
|
|
|
@ -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.
|
||||
*/
|
||||
|
|
|
@ -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<String, ChatHandler> 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<ChatHandler> get(final String key)
|
||||
{
|
||||
return Optional.fromNullable(handlers.get(key));
|
||||
}
|
||||
|
||||
private String serviceName(String key)
|
||||
{
|
||||
return String.format(config.getServiceFormat(), key);
|
||||
}
|
||||
public Optional<ChatHandler> get(final String key);
|
||||
}
|
||||
|
|
|
@ -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> chatHandlerProvider;
|
||||
private final Optional<EventReceivingChatHandlerProvider> 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");
|
||||
|
|
|
@ -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<String, ChatHandler> 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<ChatHandler> get(final String key)
|
||||
{
|
||||
return Optional.fromNullable(handlers.get(key));
|
||||
}
|
||||
|
||||
private String serviceName(String key)
|
||||
{
|
||||
return String.format(config.getServiceFormat(), key);
|
||||
}
|
||||
}
|
|
@ -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<ChatHandler> get(String key)
|
||||
{
|
||||
return Optional.absent();
|
||||
}
|
||||
}
|
|
@ -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
|
||||
|
@ -166,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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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")
|
||||
);
|
||||
|
|
|
@ -46,13 +46,11 @@ 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.plumber.NoopRejectionPolicyFactory;
|
||||
import com.metamx.druid.realtime.plumber.Plumber;
|
||||
import com.metamx.druid.realtime.plumber.RealtimePlumberSchool;
|
||||
import com.metamx.druid.realtime.plumber.RejectionPolicyFactory;
|
||||
|
@ -105,7 +103,7 @@ public class RealtimeIndexTask extends AbstractTask
|
|||
@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,
|
||||
|
@ -115,16 +113,22 @@ public class RealtimeIndexTask extends AbstractTask
|
|||
)
|
||||
{
|
||||
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
|
||||
);
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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 +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -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()
|
||||
);
|
||||
|
|
|
@ -96,6 +96,12 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider
|
|||
this.jsonMapper = jsonMapper;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void bootstrap(List<Task> tasks)
|
||||
{
|
||||
// do nothing
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<TaskStatus> run(final Task task)
|
||||
{
|
||||
|
|
|
@ -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.
|
||||
* <p/>
|
||||
* 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.
|
||||
* <p/>
|
||||
* 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.
|
||||
* <p/>
|
||||
* 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> workerSetupData;
|
||||
private final HttpClient httpClient;
|
||||
|
||||
// all workers that exist in ZK
|
||||
private final Map<String, ZkWorker> zkWorkers = new ConcurrentHashMap<String, ZkWorker>();
|
||||
// all tasks that have been assigned to a worker
|
||||
private final TaskRunnerWorkQueue runningTasks = new TaskRunnerWorkQueue();
|
||||
private final RemoteTaskRunnerWorkQueue runningTasks = new RemoteTaskRunnerWorkQueue();
|
||||
// tasks that have not yet run
|
||||
private final TaskRunnerWorkQueue pendingTasks = new TaskRunnerWorkQueue();
|
||||
// idempotent task retry
|
||||
private final Set<String> tasksToRetry = new ConcurrentSkipListSet<String>();
|
||||
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> 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<TaskRunnerWorkItem> getRunningTasks()
|
||||
public Collection<RemoteTaskRunnerWorkItem> getRunningTasks()
|
||||
{
|
||||
return runningTasks.values();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<TaskRunnerWorkItem> getPendingTasks()
|
||||
public Collection<RemoteTaskRunnerWorkItem> 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<Task> tasks)
|
||||
{
|
||||
try {
|
||||
if (!started) {
|
||||
throw new ISE("Must start RTR first before calling bootstrap!");
|
||||
}
|
||||
|
||||
Set<String> 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.<TaskStatus>create())
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -249,11 +281,19 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider
|
|||
@Override
|
||||
public ListenableFuture<TaskStatus> 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.<TaskStatus>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.<TaskStatus>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.<InputSupplier<InputStream>>of(
|
||||
new InputSupplier<InputStream>()
|
||||
{
|
||||
|
@ -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<TaskRunnerWorkItem> copy = Lists.newArrayList(pendingTasks.values());
|
||||
for (TaskRunnerWorkItem taskWrapper : copy) {
|
||||
List<RemoteTaskRunnerWorkItem> 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<TaskStatus> result = taskRunnerWorkItem.getResult();
|
||||
if (result != null) {
|
||||
((SettableFuture<TaskStatus>) 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<TaskStatus> result = taskRunnerWorkItem.getResult();
|
||||
if (result != null) {
|
||||
((SettableFuture<TaskStatus>) 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<String> 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<ZkWorker> workerQueue = MinMaxPriorityQueue.<ZkWorker>orderedBy(
|
||||
new Comparator<ZkWorker>()
|
||||
{
|
||||
@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<ZkWorker>()
|
||||
{
|
||||
@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<ZkWorker> sortedWorkers = Sets.newTreeSet(
|
||||
new Comparator<ZkWorker>()
|
||||
{
|
||||
@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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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<TaskStatus> result;
|
||||
|
||||
public RemoteTaskRunnerWorkItem(
|
||||
Task task,
|
||||
SettableFuture<TaskStatus> result
|
||||
)
|
||||
{
|
||||
super(task, result);
|
||||
this.result = result;
|
||||
}
|
||||
|
||||
public RemoteTaskRunnerWorkItem(
|
||||
Task task,
|
||||
SettableFuture<TaskStatus> 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);
|
||||
}
|
||||
}
|
|
@ -25,10 +25,10 @@ import java.util.concurrent.ConcurrentSkipListMap;
|
|||
|
||||
/**
|
||||
*/
|
||||
public class TaskRunnerWorkQueue extends ConcurrentSkipListMap<String, TaskRunnerWorkItem>
|
||||
public class RemoteTaskRunnerWorkQueue extends ConcurrentSkipListMap<String, RemoteTaskRunnerWorkItem>
|
||||
{
|
||||
@Override
|
||||
public TaskRunnerWorkItem put(String s, TaskRunnerWorkItem taskRunnerWorkItem)
|
||||
public RemoteTaskRunnerWorkItem put(String s, RemoteTaskRunnerWorkItem taskRunnerWorkItem)
|
||||
{
|
||||
return super.put(s, taskRunnerWorkItem.withQueueInsertionTime(new DateTime()));
|
||||
}
|
|
@ -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();
|
||||
|
|
|
@ -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<Task> 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.
|
||||
|
|
|
@ -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<Task> tasks);
|
||||
|
||||
/**
|
||||
* Run a task. The returned status should be some kind of completed status.
|
||||
*
|
||||
|
@ -44,9 +55,9 @@ public interface TaskRunner
|
|||
*/
|
||||
public void shutdown(String taskid);
|
||||
|
||||
public Collection<TaskRunnerWorkItem> getRunningTasks();
|
||||
public Collection<? extends TaskRunnerWorkItem> getRunningTasks();
|
||||
|
||||
public Collection<TaskRunnerWorkItem> getPendingTasks();
|
||||
public Collection<? extends TaskRunnerWorkItem> getPendingTasks();
|
||||
|
||||
public Collection<ZkWorker> getWorkers();
|
||||
}
|
||||
|
|
|
@ -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<TaskRunnerWorkItem>
|
|||
{
|
||||
private final Task task;
|
||||
private final ListenableFuture<TaskStatus> result;
|
||||
private final RetryPolicy retryPolicy;
|
||||
private final DateTime createdTime;
|
||||
|
||||
private volatile DateTime queueInsertionTime;
|
||||
|
||||
public TaskRunnerWorkItem(
|
||||
Task task,
|
||||
ListenableFuture<TaskStatus> result
|
||||
)
|
||||
{
|
||||
this(task, result, new DateTime(), new DateTime());
|
||||
}
|
||||
|
||||
public TaskRunnerWorkItem(
|
||||
Task task,
|
||||
ListenableFuture<TaskStatus> 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<TaskRunnerWorkItem>
|
|||
return result;
|
||||
}
|
||||
|
||||
public RetryPolicy getRetryPolicy()
|
||||
{
|
||||
return retryPolicy;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public DateTime getCreatedTime()
|
||||
{
|
||||
|
@ -83,8 +84,7 @@ public class TaskRunnerWorkItem implements Comparable<TaskRunnerWorkItem>
|
|||
|
||||
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<TaskRunnerWorkItem>
|
|||
return "TaskRunnerWorkItem{" +
|
||||
"task=" + task +
|
||||
", result=" + result +
|
||||
", retryPolicy=" + retryPolicy +
|
||||
", createdTime=" + createdTime +
|
||||
'}';
|
||||
}
|
||||
|
|
|
@ -77,13 +77,19 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
|
|||
exec.shutdownNow();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void bootstrap(List<Task> tasks)
|
||||
{
|
||||
// do nothing
|
||||
}
|
||||
|
||||
@Override
|
||||
public ListenableFuture<TaskStatus> run(final Task task)
|
||||
{
|
||||
final TaskToolbox toolbox = toolboxFactory.build(task);
|
||||
final ListenableFuture<TaskStatus> 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<TaskStatus>()
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<ChildData, String> cacheConverter;
|
||||
private final Function<ChildData, TaskStatus> 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<ChildData, String>()
|
||||
this.cacheConverter = new Function<ChildData, TaskStatus>()
|
||||
{
|
||||
@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<String> getRunningTasks()
|
||||
@JsonProperty("runningTasks")
|
||||
public Collection<String> getRunningTaskIds()
|
||||
{
|
||||
return Sets.newHashSet(
|
||||
Lists.transform(
|
||||
statusCache.getCurrentData(),
|
||||
cacheConverter
|
||||
)
|
||||
);
|
||||
return getRunningTasks().keySet();
|
||||
}
|
||||
|
||||
public Map<String, TaskStatus> getRunningTasks()
|
||||
{
|
||||
Map<String, TaskStatus> 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<String> getAvailabilityGroups()
|
||||
{
|
||||
Set<String> 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)
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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")
|
||||
|
|
|
@ -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")
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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<IndexerCoordinatorNode
|
|||
{
|
||||
if (persistentTaskLogs == null) {
|
||||
final TaskLogConfig taskLogConfig = getConfigFactory().build(TaskLogConfig.class);
|
||||
if (taskLogConfig.getLogStorageBucket() != null) {
|
||||
if (taskLogConfig.getLogType().equalsIgnoreCase("s3")) {
|
||||
initializeS3Service();
|
||||
persistentTaskLogs = new S3TaskLogs(
|
||||
taskLogConfig.getLogStorageBucket(),
|
||||
taskLogConfig.getLogStoragePrefix(),
|
||||
s3Service
|
||||
);
|
||||
} else {
|
||||
} else if (taskLogConfig.getLogType().equalsIgnoreCase("noop")) {
|
||||
persistentTaskLogs = new NoopTaskLogs();
|
||||
} else {
|
||||
throw new IAE("Unknown log type %s", taskLogConfig.getLogType());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -634,29 +634,14 @@ public class IndexerCoordinatorNode extends QueryableNode<IndexerCoordinatorNode
|
|||
@Override
|
||||
public TaskRunner build()
|
||||
{
|
||||
// Don't use scheduledExecutorFactory, since it's linked to the wrong lifecycle (global lifecycle instead
|
||||
// of leadership lifecycle)
|
||||
final ScheduledExecutorService retryScheduledExec = Executors.newScheduledThreadPool(
|
||||
1,
|
||||
new ThreadFactoryBuilder()
|
||||
.setDaemon(true)
|
||||
.setNameFormat("RemoteRunnerRetryExec--%d")
|
||||
.build()
|
||||
);
|
||||
|
||||
final CuratorFramework curator = getCuratorFramework();
|
||||
final RemoteTaskRunnerConfig remoteTaskRunnerConfig = getConfigFactory().build(RemoteTaskRunnerConfig.class);
|
||||
return new RemoteTaskRunner(
|
||||
getJsonMapper(),
|
||||
getConfigFactory().build(RemoteTaskRunnerConfig.class),
|
||||
remoteTaskRunnerConfig,
|
||||
curator,
|
||||
new PathChildrenCache(curator, indexerZkConfig.getIndexerAnnouncementPath(), true),
|
||||
retryScheduledExec,
|
||||
new RetryPolicyFactory(
|
||||
getConfigFactory().buildWithReplacements(
|
||||
RetryPolicyConfig.class,
|
||||
ImmutableMap.of("base_path", "druid.indexing")
|
||||
)
|
||||
),
|
||||
new SimplePathChildrenCacheFactory.Builder().withCompressed(remoteTaskRunnerConfig.enableCompression())
|
||||
.build(),
|
||||
configManager.watch(WorkerSetupData.CONFIG_KEY, WorkerSetupData.class),
|
||||
httpClient
|
||||
);
|
||||
|
@ -692,7 +677,7 @@ public class IndexerCoordinatorNode extends QueryableNode<IndexerCoordinatorNode
|
|||
resourceManagementSchedulerFactory = new ResourceManagementSchedulerFactory()
|
||||
{
|
||||
@Override
|
||||
public ResourceManagementScheduler build(TaskRunner runner)
|
||||
public ResourceManagementScheduler build(RemoteTaskRunner runner)
|
||||
{
|
||||
return new NoopResourceManagementScheduler();
|
||||
}
|
||||
|
@ -701,7 +686,7 @@ public class IndexerCoordinatorNode extends QueryableNode<IndexerCoordinatorNode
|
|||
resourceManagementSchedulerFactory = new ResourceManagementSchedulerFactory()
|
||||
{
|
||||
@Override
|
||||
public ResourceManagementScheduler build(TaskRunner runner)
|
||||
public ResourceManagementScheduler build(RemoteTaskRunner runner)
|
||||
{
|
||||
final ScheduledExecutorService scalingScheduledExec = Executors.newScheduledThreadPool(
|
||||
1,
|
||||
|
|
|
@ -33,6 +33,7 @@ import com.google.common.base.Function;
|
|||
import com.google.common.collect.Lists;
|
||||
import com.metamx.druid.indexing.coordinator.config.EC2AutoScalingStrategyConfig;
|
||||
import com.metamx.druid.indexing.coordinator.setup.EC2NodeData;
|
||||
import com.metamx.druid.indexing.coordinator.setup.GalaxyUserData;
|
||||
import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import org.apache.commons.codec.binary.Base64;
|
||||
|
@ -72,6 +73,11 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy<Instance>
|
|||
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<Instance>
|
|||
.withUserData(
|
||||
Base64.encodeBase64String(
|
||||
jsonMapper.writeValueAsBytes(
|
||||
setupData.getUserData()
|
||||
userData
|
||||
)
|
||||
)
|
||||
)
|
||||
|
@ -212,7 +218,7 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy<Instance>
|
|||
}
|
||||
);
|
||||
|
||||
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<Instance>
|
|||
}
|
||||
);
|
||||
|
||||
log.info("Performing lookup: %s --> %s", nodeIds, retVal);
|
||||
log.debug("Performing lookup: %s --> %s", nodeIds, retVal);
|
||||
|
||||
return retVal;
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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<TaskRunnerWorkItem> runningTasks, Collection<ZkWorker> zkWorkers);
|
||||
public boolean doProvision(Collection<RemoteTaskRunnerWorkItem> runningTasks, Collection<ZkWorker> zkWorkers);
|
||||
|
||||
public boolean doTerminate(Collection<TaskRunnerWorkItem> runningTasks, Collection<ZkWorker> zkWorkers);
|
||||
public boolean doTerminate(Collection<RemoteTaskRunnerWorkItem> runningTasks, Collection<ZkWorker> zkWorkers);
|
||||
|
||||
public ScalingStats getStats();
|
||||
}
|
||||
|
|
|
@ -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<TaskRunnerWorkItem> pendingTasks, Collection<ZkWorker> zkWorkers)
|
||||
public boolean doProvision(Collection<RemoteTaskRunnerWorkItem> pendingTasks, Collection<ZkWorker> 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<TaskRunnerWorkItem> pendingTasks, Collection<ZkWorker> zkWorkers)
|
||||
public boolean doTerminate(Collection<RemoteTaskRunnerWorkItem> pendingTasks, Collection<ZkWorker> zkWorkers)
|
||||
{
|
||||
Set<String> workerNodeIds = Sets.newHashSet(
|
||||
autoScalingStrategy.ipToIdLookup(
|
||||
|
@ -244,7 +259,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat
|
|||
return scalingStats;
|
||||
}
|
||||
|
||||
private boolean hasTaskPendingBeyondThreshold(Collection<TaskRunnerWorkItem> pendingTasks)
|
||||
private boolean hasTaskPendingBeyondThreshold(Collection<RemoteTaskRunnerWorkItem> pendingTasks)
|
||||
{
|
||||
long now = System.currentTimeMillis();
|
||||
for (TaskRunnerWorkItem pendingTask : pendingTasks) {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -60,6 +60,11 @@ public class GalaxyUserData
|
|||
return type;
|
||||
}
|
||||
|
||||
public GalaxyUserData withVersion(String ver)
|
||||
{
|
||||
return new GalaxyUserData(env, ver, type);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
|
|
|
@ -130,6 +130,7 @@ public class WorkerCuratorCoordinator
|
|||
}
|
||||
|
||||
curatorFramework.create()
|
||||
.creatingParentsIfNeeded()
|
||||
.withMode(mode)
|
||||
.forPath(path, rawBytes);
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -24,7 +24,6 @@ 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;
|
||||
|
@ -38,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;
|
||||
|
@ -52,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;
|
||||
|
@ -70,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;
|
||||
|
@ -86,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;
|
||||
|
@ -288,7 +286,7 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
|
|||
|
||||
private void initializeS3Service() throws S3ServiceException
|
||||
{
|
||||
if(s3Service == null) {
|
||||
if (s3Service == null) {
|
||||
s3Service = new RestS3Service(
|
||||
new AWSCredentials(
|
||||
PropUtils.getProperty(props, "com.metamx.aws.accessKey"),
|
||||
|
@ -350,8 +348,9 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
|
|||
{
|
||||
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) {
|
||||
|
@ -387,17 +386,15 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
|
|||
{
|
||||
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
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -439,9 +436,12 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
|
|||
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) {
|
||||
|
@ -456,7 +456,15 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
|
|||
configFactory = Config.createFactory(props);
|
||||
}
|
||||
|
||||
return new ExecutorNode(nodeType, props, lifecycle, jsonMapper, smileMapper, configFactory, executorLifecycleFactory);
|
||||
return new ExecutorNode(
|
||||
nodeType,
|
||||
props,
|
||||
lifecycle,
|
||||
jsonMapper,
|
||||
smileMapper,
|
||||
configFactory,
|
||||
executorLifecycleFactory
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -331,11 +331,13 @@ public class WorkerNode extends QueryableNode<WorkerNode>
|
|||
{
|
||||
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<WorkerNode>
|
|||
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(
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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<TaskStatus> 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<String> 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.<Task>asList(makeTask(TaskStatus.running("second"))));
|
||||
|
||||
Set<String> runningTasks = Sets.newHashSet(
|
||||
Iterables.transform(
|
||||
remoteTaskRunner.getRunningTasks(),
|
||||
new Function<RemoteTaskRunnerWorkItem, String>()
|
||||
{
|
||||
@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.<DataSegment>newArrayList(
|
||||
new DataSegment(
|
||||
|
@ -117,194 +284,20 @@ public class RemoteTaskRunnerTest
|
|||
)
|
||||
),
|
||||
Lists.<AggregatorFactory>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.<AggregatorFactory>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.<DataSegment>newArrayList(
|
||||
new DataSegment(
|
||||
"dummyDs",
|
||||
new Interval(new DateTime(), new DateTime()),
|
||||
new DateTime().toString(),
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
0,
|
||||
0
|
||||
)
|
||||
),
|
||||
Lists.<AggregatorFactory>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.<AggregatorFactory>newArrayList(),
|
||||
TaskStatus.running(task1.getId())
|
||||
)
|
||||
), new FutureCallback<TaskStatus>()
|
||||
{
|
||||
@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
|
||||
|
@ -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<WorkerSetupData>(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 "";
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
@ -285,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()
|
||||
|
@ -322,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()
|
||||
|
@ -356,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()
|
||||
|
|
|
@ -348,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
|
||||
|
@ -372,7 +372,7 @@ public class TaskQueueTest
|
|||
final List<Task> nextTasks
|
||||
)
|
||||
{
|
||||
return new AbstractTask(id, groupId, id, dataSource, interval)
|
||||
return new AbstractTask(id, groupId, dataSource, interval)
|
||||
{
|
||||
@Override
|
||||
public String getType()
|
||||
|
|
|
@ -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,14 +41,14 @@ 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,
|
||||
|
|
|
@ -84,6 +84,12 @@ public class EC2AutoScalingStrategyTest
|
|||
{
|
||||
return "8080";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getWorkerVersion()
|
||||
{
|
||||
return "";
|
||||
}
|
||||
},
|
||||
workerSetupData
|
||||
);
|
||||
|
|
|
@ -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.<TaskRunnerWorkItem>asList(
|
||||
new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime())
|
||||
Arrays.<RemoteTaskRunnerWorkItem>asList(
|
||||
new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime())
|
||||
),
|
||||
Arrays.<ZkWorker>asList(
|
||||
new TestZkWorker(testTask)
|
||||
|
@ -155,8 +166,8 @@ public class SimpleResourceManagementStrategyTest
|
|||
EasyMock.replay(autoScalingStrategy);
|
||||
|
||||
boolean provisionedSomething = simpleResourceManagementStrategy.doProvision(
|
||||
Arrays.<TaskRunnerWorkItem>asList(
|
||||
new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime())
|
||||
Arrays.<RemoteTaskRunnerWorkItem>asList(
|
||||
new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime())
|
||||
),
|
||||
Arrays.<ZkWorker>asList(
|
||||
new TestZkWorker(testTask)
|
||||
|
@ -171,8 +182,8 @@ public class SimpleResourceManagementStrategyTest
|
|||
);
|
||||
|
||||
provisionedSomething = simpleResourceManagementStrategy.doProvision(
|
||||
Arrays.<TaskRunnerWorkItem>asList(
|
||||
new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime())
|
||||
Arrays.<RemoteTaskRunnerWorkItem>asList(
|
||||
new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime())
|
||||
),
|
||||
Arrays.<ZkWorker>asList(
|
||||
new TestZkWorker(testTask)
|
||||
|
@ -212,8 +223,8 @@ public class SimpleResourceManagementStrategyTest
|
|||
EasyMock.replay(autoScalingStrategy);
|
||||
|
||||
boolean provisionedSomething = simpleResourceManagementStrategy.doProvision(
|
||||
Arrays.<TaskRunnerWorkItem>asList(
|
||||
new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime())
|
||||
Arrays.<RemoteTaskRunnerWorkItem>asList(
|
||||
new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime())
|
||||
),
|
||||
Arrays.<ZkWorker>asList(
|
||||
new TestZkWorker(testTask)
|
||||
|
@ -230,8 +241,8 @@ public class SimpleResourceManagementStrategyTest
|
|||
Thread.sleep(2000);
|
||||
|
||||
provisionedSomething = simpleResourceManagementStrategy.doProvision(
|
||||
Arrays.<TaskRunnerWorkItem>asList(
|
||||
new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime())
|
||||
Arrays.<RemoteTaskRunnerWorkItem>asList(
|
||||
new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime())
|
||||
),
|
||||
Arrays.<ZkWorker>asList(
|
||||
new TestZkWorker(testTask)
|
||||
|
@ -264,8 +275,8 @@ public class SimpleResourceManagementStrategyTest
|
|||
EasyMock.replay(autoScalingStrategy);
|
||||
|
||||
boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate(
|
||||
Arrays.<TaskRunnerWorkItem>asList(
|
||||
new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime())
|
||||
Arrays.<RemoteTaskRunnerWorkItem>asList(
|
||||
new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime())
|
||||
),
|
||||
Arrays.<ZkWorker>asList(
|
||||
new TestZkWorker(null)
|
||||
|
@ -294,8 +305,8 @@ public class SimpleResourceManagementStrategyTest
|
|||
EasyMock.replay(autoScalingStrategy);
|
||||
|
||||
boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate(
|
||||
Arrays.<TaskRunnerWorkItem>asList(
|
||||
new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime())
|
||||
Arrays.<RemoteTaskRunnerWorkItem>asList(
|
||||
new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime())
|
||||
),
|
||||
Arrays.<ZkWorker>asList(
|
||||
new TestZkWorker(null)
|
||||
|
@ -309,8 +320,8 @@ public class SimpleResourceManagementStrategyTest
|
|||
);
|
||||
|
||||
terminatedSomething = simpleResourceManagementStrategy.doTerminate(
|
||||
Arrays.<TaskRunnerWorkItem>asList(
|
||||
new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime())
|
||||
Arrays.<RemoteTaskRunnerWorkItem>asList(
|
||||
new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime())
|
||||
),
|
||||
Arrays.<ZkWorker>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<String> getRunningTasks()
|
||||
public Map<String, TaskStatus> getRunningTasks()
|
||||
{
|
||||
if (testTask == null) {
|
||||
return Sets.newHashSet();
|
||||
return Maps.newHashMap();
|
||||
}
|
||||
return Sets.newHashSet(testTask.getId());
|
||||
return ImmutableMap.of(testTask.getId(), TaskStatus.running(testTask.getId()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
4
pom.xml
4
pom.xml
|
@ -23,7 +23,7 @@
|
|||
<groupId>com.metamx</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<packaging>pom</packaging>
|
||||
<version>0.5.23-SNAPSHOT</version>
|
||||
<version>0.5.33-SNAPSHOT</version>
|
||||
<name>druid</name>
|
||||
<description>druid</description>
|
||||
<scm>
|
||||
|
@ -38,7 +38,7 @@
|
|||
|
||||
<properties>
|
||||
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
|
||||
<metamx.java-util.version>0.22.3</metamx.java-util.version>
|
||||
<metamx.java-util.version>0.22.6</metamx.java-util.version>
|
||||
<apache.curator.version>2.1.0-incubating</apache.curator.version>
|
||||
</properties>
|
||||
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.5.23-SNAPSHOT</version>
|
||||
<version>0.5.33-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -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())
|
||||
|
|
|
@ -28,7 +28,7 @@
|
|||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.5.23-SNAPSHOT</version>
|
||||
<version>0.5.33-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
|
@ -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<String, VersionedIntervalTimeline<String, Segment>> dataSources;
|
||||
private final Map<String, VersionedIntervalTimeline<String, ReferenceCountingSegment>> dataSources;
|
||||
private final CountingMap<String> dataSourceSizes = new CountingMap<String>();
|
||||
private final CountingMap<String> dataSourceCounts = new CountingMap<String>();
|
||||
|
||||
|
@ -88,7 +92,7 @@ public class ServerManager implements QuerySegmentWalker
|
|||
|
||||
this.exec = exec;
|
||||
|
||||
this.dataSources = new HashMap<String, VersionedIntervalTimeline<String, Segment>>();
|
||||
this.dataSources = new HashMap<String, VersionedIntervalTimeline<String, ReferenceCountingSegment>>();
|
||||
}
|
||||
|
||||
public Map<String, Long> getDataSourceSizes()
|
||||
|
@ -132,14 +136,14 @@ public class ServerManager implements QuerySegmentWalker
|
|||
|
||||
synchronized (lock) {
|
||||
String dataSource = segment.getDataSource();
|
||||
VersionedIntervalTimeline<String, Segment> loadedIntervals = dataSources.get(dataSource);
|
||||
VersionedIntervalTimeline<String, ReferenceCountingSegment> loadedIntervals = dataSources.get(dataSource);
|
||||
|
||||
if (loadedIntervals == null) {
|
||||
loadedIntervals = new VersionedIntervalTimeline<String, Segment>(Ordering.natural());
|
||||
loadedIntervals = new VersionedIntervalTimeline<String, ReferenceCountingSegment>(Ordering.natural());
|
||||
dataSources.put(dataSource, loadedIntervals);
|
||||
}
|
||||
|
||||
PartitionHolder<Segment> entry = loadedIntervals.findEntry(
|
||||
PartitionHolder<ReferenceCountingSegment> 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<String, Segment> loadedIntervals = dataSources.get(dataSource);
|
||||
VersionedIntervalTimeline<String, ReferenceCountingSegment> 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<Segment> removed = loadedIntervals.remove(
|
||||
segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk((Segment) null)
|
||||
PartitionChunk<ReferenceCountingSegment> 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<T, Query<T>> toolChest = factory.getToolchest();
|
||||
|
||||
final VersionedIntervalTimeline<String, Segment> timeline = dataSources.get(query.getDataSource());
|
||||
final VersionedIntervalTimeline<String, ReferenceCountingSegment> timeline = dataSources.get(query.getDataSource());
|
||||
|
||||
if (timeline == null) {
|
||||
return new NoopQueryRunner<T>();
|
||||
|
@ -214,20 +232,22 @@ public class ServerManager implements QuerySegmentWalker
|
|||
FunctionalIterable<QueryRunner<T>> adapters = FunctionalIterable
|
||||
.create(intervals)
|
||||
.transformCat(
|
||||
new Function<Interval, Iterable<TimelineObjectHolder<String, Segment>>>()
|
||||
new Function<Interval, Iterable<TimelineObjectHolder<String, ReferenceCountingSegment>>>()
|
||||
{
|
||||
@Override
|
||||
public Iterable<TimelineObjectHolder<String, Segment>> apply(Interval input)
|
||||
public Iterable<TimelineObjectHolder<String, ReferenceCountingSegment>> apply(Interval input)
|
||||
{
|
||||
return timeline.lookup(input);
|
||||
}
|
||||
}
|
||||
)
|
||||
.transformCat(
|
||||
new Function<TimelineObjectHolder<String, Segment>, Iterable<QueryRunner<T>>>()
|
||||
new Function<TimelineObjectHolder<String, ReferenceCountingSegment>, Iterable<QueryRunner<T>>>()
|
||||
{
|
||||
@Override
|
||||
public Iterable<QueryRunner<T>> apply(@Nullable final TimelineObjectHolder<String, Segment> holder)
|
||||
public Iterable<QueryRunner<T>> apply(
|
||||
@Nullable final TimelineObjectHolder<String, ReferenceCountingSegment> holder
|
||||
)
|
||||
{
|
||||
if (holder == null) {
|
||||
return null;
|
||||
|
@ -236,10 +256,10 @@ public class ServerManager implements QuerySegmentWalker
|
|||
return FunctionalIterable
|
||||
.create(holder.getObject())
|
||||
.transform(
|
||||
new Function<PartitionChunk<Segment>, QueryRunner<T>>()
|
||||
new Function<PartitionChunk<ReferenceCountingSegment>, QueryRunner<T>>()
|
||||
{
|
||||
@Override
|
||||
public QueryRunner<T> apply(PartitionChunk<Segment> input)
|
||||
public QueryRunner<T> apply(PartitionChunk<ReferenceCountingSegment> input)
|
||||
{
|
||||
return buildAndDecorateQueryRunner(
|
||||
factory,
|
||||
|
@ -280,7 +300,7 @@ public class ServerManager implements QuerySegmentWalker
|
|||
|
||||
final QueryToolChest<T, Query<T>> toolChest = factory.getToolchest();
|
||||
|
||||
final VersionedIntervalTimeline<String, Segment> timeline = dataSources.get(query.getDataSource());
|
||||
final VersionedIntervalTimeline<String, ReferenceCountingSegment> timeline = dataSources.get(query.getDataSource());
|
||||
|
||||
if (timeline == null) {
|
||||
return new NoopQueryRunner<T>();
|
||||
|
@ -293,9 +313,9 @@ public class ServerManager implements QuerySegmentWalker
|
|||
{
|
||||
@Override
|
||||
@SuppressWarnings("unchecked")
|
||||
public Iterable<QueryRunner<T>> apply(@Nullable SegmentDescriptor input)
|
||||
public Iterable<QueryRunner<T>> apply(SegmentDescriptor input)
|
||||
{
|
||||
final PartitionHolder<Segment> entry = timeline.findEntry(
|
||||
final PartitionHolder<ReferenceCountingSegment> entry = timeline.findEntry(
|
||||
input.getInterval(), input.getVersion()
|
||||
);
|
||||
|
||||
|
@ -303,12 +323,12 @@ public class ServerManager implements QuerySegmentWalker
|
|||
return null;
|
||||
}
|
||||
|
||||
final PartitionChunk<Segment> chunk = entry.getChunk(input.getPartitionNumber());
|
||||
final PartitionChunk<ReferenceCountingSegment> 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 <T> QueryRunner<T> buildAndDecorateQueryRunner(
|
||||
QueryRunnerFactory<T, Query<T>> factory,
|
||||
final QueryRunnerFactory<T, Query<T>> factory,
|
||||
final QueryToolChest<T, Query<T>> toolChest,
|
||||
Segment adapter,
|
||||
QuerySegmentSpec segmentSpec
|
||||
final ReferenceCountingSegment adapter,
|
||||
final QuerySegmentSpec segmentSpec
|
||||
)
|
||||
{
|
||||
return new SpecificSegmentQueryRunner<T>(
|
||||
|
@ -335,7 +355,7 @@ public class ServerManager implements QuerySegmentWalker
|
|||
new Function<Query<T>, ServiceMetricEvent.Builder>()
|
||||
{
|
||||
@Override
|
||||
public ServiceMetricEvent.Builder apply(@Nullable Query<T> input)
|
||||
public ServiceMetricEvent.Builder apply(@Nullable final Query<T> input)
|
||||
{
|
||||
return toolChest.makeMetricBuilder(input);
|
||||
}
|
||||
|
@ -343,10 +363,17 @@ public class ServerManager implements QuerySegmentWalker
|
|||
new BySegmentQueryRunner<T>(
|
||||
adapter.getIdentifier(),
|
||||
adapter.getDataInterval().getStart(),
|
||||
factory.createRunner(adapter)
|
||||
new QueryRunner<T>()
|
||||
{
|
||||
@Override
|
||||
public Sequence<T> run(final Query<T> query)
|
||||
{
|
||||
return new ReferenceCountingSequence<T>(factory.createRunner(adapter).run(query), adapter);
|
||||
}
|
||||
}
|
||||
)
|
||||
).withWaitMeasuredFromNow(),
|
||||
segmentSpec
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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(
|
||||
|
|
|
@ -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
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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<T> extends YieldingSequenceBase<T>
|
||||
{
|
||||
private final Sequence<T> baseSequence;
|
||||
private final ReferenceCountingSegment segment;
|
||||
|
||||
public ReferenceCountingSequence(Sequence<T> baseSequence, ReferenceCountingSegment segment)
|
||||
{
|
||||
this.baseSequence = baseSequence;
|
||||
this.segment = segment;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <OutType> Yielder<OutType> toYielder(
|
||||
OutType initValue, YieldingAccumulator<OutType, T> accumulator
|
||||
)
|
||||
{
|
||||
return new ResourceClosingYielder<OutType>(baseSequence.toYielder(initValue, accumulator), segment.increment());
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
|
|
|
@ -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()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -22,6 +22,7 @@ package com.metamx.druid.loading;
|
|||
import com.metamx.druid.index.QueryableIndex;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
|
|
@ -89,4 +89,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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
@ -216,7 +217,7 @@ public class DruidMasterSegmentMerger implements DruidMasterHelper
|
|||
).asList();
|
||||
}
|
||||
|
||||
public void add(TimelineObjectHolder<String, DataSegment> timelineObject)
|
||||
public boolean add(TimelineObjectHolder<String, DataSegment> timelineObject)
|
||||
{
|
||||
final Interval timelineObjectInterval = timelineObject.getInterval();
|
||||
|
||||
|
@ -235,6 +236,10 @@ public class DruidMasterSegmentMerger implements DruidMasterHelper
|
|||
Interval underlyingInterval = firstChunk.getObject().getInterval();
|
||||
|
||||
for (final PartitionChunk<DataSegment> 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 +261,8 @@ public class DruidMasterSegmentMerger implements DruidMasterHelper
|
|||
|
||||
timelineObjects.add(Pair.of(timelineObject, new Interval(start, end)));
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
public Interval getMergedTimelineInterval()
|
||||
|
|
|
@ -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<SegmentHolder> segmentHolderComparator = new Comparator<SegmentHolder>()
|
||||
{
|
||||
private Comparator<DataSegment> 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<SegmentHolder> segmentsToLoad = new ConcurrentSkipListSet<SegmentHolder>(
|
||||
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<DataSegment> 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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<String, Interval>("1", new Interval("P1d/2011-04-01"))
|
||||
)
|
||||
);
|
||||
waitForTestVerificationAndCleanup(future);
|
||||
|
||||
assertQueryable(
|
||||
|
||||
future = assertQueryable(
|
||||
QueryGranularity.DAY,
|
||||
"test", new Interval("P2d/2011-04-02"),
|
||||
ImmutableList.<Pair<String, Interval>>of(
|
||||
|
@ -152,6 +166,7 @@ public class ServerManagerTest
|
|||
new Pair<String, Interval>("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.<Pair<String, Interval>>of(
|
||||
new Pair<String, Interval>("2", interval)
|
||||
)
|
||||
);
|
||||
waitForTestVerificationAndCleanup(future);
|
||||
|
||||
dropQueryable(dataSouce, "2", interval);
|
||||
assertQueryable(
|
||||
future = assertQueryable(
|
||||
QueryGranularity.DAY,
|
||||
dataSouce, interval,
|
||||
ImmutableList.<Pair<String, Interval>>of(
|
||||
new Pair<String, Interval>("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.<Pair<String, Interval>>of(
|
||||
new Pair<String, Interval>("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.<Pair<String, Interval>>of(
|
||||
|
@ -205,8 +223,9 @@ public class ServerManagerTest
|
|||
new Pair<String, Interval>("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.<Pair<String, Interval>>of(
|
||||
|
@ -215,8 +234,9 @@ public class ServerManagerTest
|
|||
new Pair<String, Interval>("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.<Pair<String, Interval>>of(
|
||||
|
@ -224,9 +244,171 @@ public class ServerManagerTest
|
|||
new Pair<String, Interval>("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.<Pair<String, Interval>>of(
|
||||
new Pair<String, Interval>("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.<Pair<String, Interval>>of(
|
||||
new Pair<String, Interval>("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.<Pair<String, Interval>>of(
|
||||
new Pair<String, Interval>("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 <T> Future assertQueryable(
|
||||
QueryGranularity granularity,
|
||||
String dataSource,
|
||||
Interval interval,
|
||||
List<Pair<String, Interval>> expected
|
||||
)
|
||||
{
|
||||
final Iterator<Pair<String, Interval>> expectedIter = expected.iterator();
|
||||
final List<Interval> intervals = Arrays.asList(interval);
|
||||
final SearchQuery query = Druids.newSearchQueryBuilder()
|
||||
.dataSource(dataSource)
|
||||
.intervals(intervals)
|
||||
.granularity(granularity)
|
||||
.limit(10000)
|
||||
.query("wow")
|
||||
.build();
|
||||
final QueryRunner<Result<SearchResultValue>> runner = serverManager.getQueryRunnerForIntervals(
|
||||
query,
|
||||
intervals
|
||||
);
|
||||
|
||||
return serverManagerExec.submit(
|
||||
new Runnable()
|
||||
{
|
||||
@Override
|
||||
public void run()
|
||||
{
|
||||
Sequence<Result<SearchResultValue>> seq = runner.run(query);
|
||||
Sequences.toList(seq, Lists.<Result<SearchResultValue>>newArrayList());
|
||||
Iterator<SegmentForTesting> adaptersIter = factory.getAdapters().iterator();
|
||||
|
||||
while (expectedIter.hasNext() && adaptersIter.hasNext()) {
|
||||
Pair<String, Interval> 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 <T> void assertQueryable(
|
||||
QueryGranularity granularity,
|
||||
String dataSource,
|
||||
Interval interval,
|
||||
List<Pair<String, Interval>> expected
|
||||
)
|
||||
{
|
||||
Iterator<Pair<String, Interval>> expectedIter = expected.iterator();
|
||||
final List<Interval> intervals = Arrays.asList(interval);
|
||||
final SearchQuery query = Druids.newSearchQueryBuilder()
|
||||
.dataSource(dataSource)
|
||||
.intervals(intervals)
|
||||
.granularity(granularity)
|
||||
.limit(10000)
|
||||
.query("wow")
|
||||
.build();
|
||||
QueryRunner<Result<SearchResultValue>> runner = serverManager.getQueryRunnerForIntervals(query, intervals);
|
||||
final Sequence<Result<SearchResultValue>> seq = runner.run(query);
|
||||
Sequences.toList(seq, Lists.<Result<SearchResultValue>>newArrayList());
|
||||
Iterator<SegmentForTesting> adaptersIter = factory.getAdapters().iterator();
|
||||
|
||||
while (expectedIter.hasNext() && adaptersIter.hasNext()) {
|
||||
Pair<String, Interval> 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<Result<SearchResultValue>, SearchQuery>
|
||||
{
|
||||
private final CountDownLatch waitLatch;
|
||||
private final CountDownLatch notifyLatch;
|
||||
private List<SegmentForTesting> adapters = Lists.newArrayList();
|
||||
|
||||
public MyQueryRunnerFactory(CountDownLatch waitLatch, CountDownLatch notifyLatch)
|
||||
{
|
||||
this.waitLatch = waitLatch;
|
||||
this.notifyLatch = notifyLatch;
|
||||
}
|
||||
|
||||
@Override
|
||||
public QueryRunner<Result<SearchResultValue>> createRunner(Segment adapter)
|
||||
{
|
||||
adapters.add((SegmentForTesting) adapter);
|
||||
return new NoopQueryRunner<Result<SearchResultValue>>();
|
||||
if (!(adapter instanceof ReferenceCountingSegment)) {
|
||||
throw new IAE("Expected instance of ReferenceCountingSegment, got %s", adapter.getClass());
|
||||
}
|
||||
adapters.add((SegmentForTesting) ((ReferenceCountingSegment) adapter).getBaseSegment());
|
||||
return new BlockingQueryRunner<Result<SearchResultValue>>(
|
||||
new NoopQueryRunner<Result<SearchResultValue>>(),
|
||||
waitLatch,
|
||||
notifyLatch
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -419,7 +532,159 @@ public class ServerManagerTest
|
|||
@Override
|
||||
public TypeReference<T> getResultTypeReference()
|
||||
{
|
||||
return new TypeReference<T>(){};
|
||||
return new TypeReference<T>()
|
||||
{
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
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<T> implements QueryRunner<T>
|
||||
{
|
||||
private final QueryRunner<T> runner;
|
||||
private final CountDownLatch waitLatch;
|
||||
private final CountDownLatch notifyLatch;
|
||||
|
||||
public BlockingQueryRunner(
|
||||
QueryRunner<T> runner,
|
||||
CountDownLatch waitLatch,
|
||||
CountDownLatch notifyLatch
|
||||
)
|
||||
{
|
||||
this.runner = runner;
|
||||
this.waitLatch = waitLatch;
|
||||
this.notifyLatch = notifyLatch;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Sequence<T> run(Query<T> query)
|
||||
{
|
||||
return new BlockingSequence<T>(runner.run(query), waitLatch, notifyLatch);
|
||||
}
|
||||
}
|
||||
|
||||
private static class BlockingSequence<T> extends YieldingSequenceBase<T>
|
||||
{
|
||||
private final Sequence<T> baseSequence;
|
||||
private final CountDownLatch waitLatch;
|
||||
private final CountDownLatch notifyLatch;
|
||||
|
||||
public BlockingSequence(
|
||||
Sequence<T> baseSequence,
|
||||
CountDownLatch waitLatch,
|
||||
CountDownLatch notifyLatch
|
||||
)
|
||||
{
|
||||
this.baseSequence = baseSequence;
|
||||
this.waitLatch = waitLatch;
|
||||
this.notifyLatch = notifyLatch;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <OutType> Yielder<OutType> toYielder(
|
||||
final OutType initValue, final YieldingAccumulator<OutType, T> accumulator
|
||||
)
|
||||
{
|
||||
notifyLatch.countDown();
|
||||
|
||||
final Yielder<OutType> baseYielder = baseSequence.toYielder(initValue, accumulator);
|
||||
return new Yielder<OutType>()
|
||||
{
|
||||
@Override
|
||||
public OutType get()
|
||||
{
|
||||
try {
|
||||
waitLatch.await();
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
return baseYielder.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Yielder<OutType> next(OutType initValue)
|
||||
{
|
||||
return baseYielder.next(initValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isDone()
|
||||
{
|
||||
return baseYielder.isDone();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
baseYielder.close();
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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
|
||||
{
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
|
|
@ -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<String, Object> loadSpec = ImmutableMap.<String, Object>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);
|
||||
|
||||
}
|
||||
}
|
|
@ -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<DataSegment> 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)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.junit.Before;
|
|||
import org.junit.Test;
|
||||
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
|
|
@ -10,7 +10,7 @@ public class LoadQueuePeonTester extends LoadQueuePeon
|
|||
|
||||
public LoadQueuePeonTester()
|
||||
{
|
||||
super(null, null, null, null);
|
||||
super(null, null, null, null, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -24,11 +24,11 @@
|
|||
<artifactId>druid-services</artifactId>
|
||||
<name>druid-services</name>
|
||||
<description>druid-services</description>
|
||||
<version>0.5.23-SNAPSHOT</version>
|
||||
<version>0.5.33-SNAPSHOT</version>
|
||||
<parent>
|
||||
<groupId>com.metamx</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.5.23-SNAPSHOT</version>
|
||||
<version>0.5.33-SNAPSHOT</version>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
|
|
Loading…
Reference in New Issue