mirror of https://github.com/apache/druid.git
merged with master
This commit is contained in:
commit
abe347efd3
|
@ -28,7 +28,7 @@
|
||||||
<parent>
|
<parent>
|
||||||
<groupId>com.metamx</groupId>
|
<groupId>com.metamx</groupId>
|
||||||
<artifactId>druid</artifactId>
|
<artifactId>druid</artifactId>
|
||||||
<version>0.5.19-SNAPSHOT</version>
|
<version>0.5.33-SNAPSHOT</version>
|
||||||
</parent>
|
</parent>
|
||||||
|
|
||||||
<dependencies>
|
<dependencies>
|
||||||
|
@ -68,6 +68,10 @@
|
||||||
<groupId>commons-codec</groupId>
|
<groupId>commons-codec</groupId>
|
||||||
<artifactId>commons-codec</artifactId>
|
<artifactId>commons-codec</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
<dependency>
|
||||||
|
<groupId>commons-httpclient</groupId>
|
||||||
|
<artifactId>commons-httpclient</artifactId>
|
||||||
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.skife.config</groupId>
|
<groupId>org.skife.config</groupId>
|
||||||
<artifactId>config-magic</artifactId>
|
<artifactId>config-magic</artifactId>
|
||||||
|
|
|
@ -34,6 +34,7 @@ import java.util.concurrent.ConcurrentMap;
|
||||||
*/
|
*/
|
||||||
public class DruidServer implements Comparable
|
public class DruidServer implements Comparable
|
||||||
{
|
{
|
||||||
|
public static final String DEFAULT_TIER = "_default_tier";
|
||||||
private static final Logger log = new Logger(DruidServer.class);
|
private static final Logger log = new Logger(DruidServer.class);
|
||||||
|
|
||||||
private final Object lock = new Object();
|
private final Object lock = new Object();
|
||||||
|
|
|
@ -33,9 +33,10 @@ public abstract class DruidServerConfig
|
||||||
public abstract String getHost();
|
public abstract String getHost();
|
||||||
|
|
||||||
@Config("druid.server.maxSize")
|
@Config("druid.server.maxSize")
|
||||||
|
@Default("0")
|
||||||
public abstract long getMaxSize();
|
public abstract long getMaxSize();
|
||||||
|
|
||||||
@Config("druid.server.tier")
|
@Config("druid.server.tier")
|
||||||
@Default("_default_tier")
|
@Default(DruidServer.DEFAULT_TIER)
|
||||||
public abstract String getTier();
|
public abstract String getTier();
|
||||||
}
|
}
|
||||||
|
|
|
@ -62,9 +62,7 @@ public class SingleServerInventoryView extends ServerInventoryView<DataSegment>
|
||||||
curator,
|
curator,
|
||||||
exec,
|
exec,
|
||||||
jsonMapper,
|
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.CuratorFramework;
|
||||||
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
|
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
|
||||||
|
import org.apache.curator.utils.ThreadUtils;
|
||||||
|
|
||||||
import java.util.concurrent.ExecutorService;
|
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);
|
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")
|
@Default("30000")
|
||||||
public abstract int getZkSessionTimeoutMs();
|
public abstract int getZkSessionTimeoutMs();
|
||||||
|
|
||||||
@Config("druid.curator.compression.enable")
|
@Config("druid.curator.compress")
|
||||||
@Default("false")
|
@Default("false")
|
||||||
public abstract boolean enableCompression();
|
public abstract boolean enableCompression();
|
||||||
}
|
}
|
||||||
|
|
|
@ -36,4 +36,11 @@ public abstract class ServiceDiscoveryConfig extends CuratorConfig
|
||||||
|
|
||||||
@Config("druid.zk.paths.discoveryPath")
|
@Config("druid.zk.paths.discoveryPath")
|
||||||
public abstract String getDiscoveryPath();
|
public abstract String getDiscoveryPath();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
@Config("druid.curator.discovery.compress")
|
||||||
|
public boolean enableCompression()
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -28,7 +28,7 @@
|
||||||
<parent>
|
<parent>
|
||||||
<groupId>com.metamx</groupId>
|
<groupId>com.metamx</groupId>
|
||||||
<artifactId>druid</artifactId>
|
<artifactId>druid</artifactId>
|
||||||
<version>0.5.19-SNAPSHOT</version>
|
<version>0.5.33-SNAPSHOT</version>
|
||||||
</parent>
|
</parent>
|
||||||
|
|
||||||
<dependencies>
|
<dependencies>
|
||||||
|
|
|
@ -61,6 +61,11 @@ public class PartitionHolder<T> implements Iterable<PartitionChunk<T>>
|
||||||
holderSet.add(chunk);
|
holderSet.add(chunk);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public int size()
|
||||||
|
{
|
||||||
|
return holderSet.size();
|
||||||
|
}
|
||||||
|
|
||||||
public PartitionChunk<T> remove(PartitionChunk<T> chunk)
|
public PartitionChunk<T> remove(PartitionChunk<T> chunk)
|
||||||
{
|
{
|
||||||
// Somewhat funky implementation in order to return the removed object as it exists in the set
|
// Somewhat funky implementation in order to return the removed object as it exists in the set
|
||||||
|
|
|
@ -9,7 +9,7 @@
|
||||||
<parent>
|
<parent>
|
||||||
<groupId>com.metamx</groupId>
|
<groupId>com.metamx</groupId>
|
||||||
<artifactId>druid</artifactId>
|
<artifactId>druid</artifactId>
|
||||||
<version>0.5.19-SNAPSHOT</version>
|
<version>0.5.33-SNAPSHOT</version>
|
||||||
</parent>
|
</parent>
|
||||||
|
|
||||||
<dependencies>
|
<dependencies>
|
||||||
|
|
|
@ -68,6 +68,7 @@ public class InputSupplierUpdateStreamTest
|
||||||
updateStream.start();
|
updateStream.start();
|
||||||
Map<String, Object> insertedRow = updateStream.pollFromQueue(waitTime, unit);
|
Map<String, Object> insertedRow = updateStream.pollFromQueue(waitTime, unit);
|
||||||
Assert.assertEquals(expectedAnswer, insertedRow);
|
Assert.assertEquals(expectedAnswer, insertedRow);
|
||||||
|
updateStream.stop();
|
||||||
}
|
}
|
||||||
|
|
||||||
//If a timestamp is missing, we should throw away the event
|
//If a timestamp is missing, we should throw away the event
|
||||||
|
@ -85,6 +86,7 @@ public class InputSupplierUpdateStreamTest
|
||||||
);
|
);
|
||||||
updateStream.start();
|
updateStream.start();
|
||||||
Assert.assertEquals(updateStream.getQueueSize(), 0);
|
Assert.assertEquals(updateStream.getQueueSize(), 0);
|
||||||
|
updateStream.stop();
|
||||||
}
|
}
|
||||||
|
|
||||||
//If any other value is missing, we should still add the event and process it properly
|
//If any other value is missing, we should still add the event and process it properly
|
||||||
|
@ -105,6 +107,7 @@ public class InputSupplierUpdateStreamTest
|
||||||
expectedAnswer.put("item1", "value1");
|
expectedAnswer.put("item1", "value1");
|
||||||
expectedAnswer.put("time", 1372121562);
|
expectedAnswer.put("time", 1372121562);
|
||||||
Assert.assertEquals(expectedAnswer, insertedRow);
|
Assert.assertEquals(expectedAnswer, insertedRow);
|
||||||
|
updateStream.stop();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -47,7 +47,7 @@ public class RenamingKeysUpdateStreamTest
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testPollFromQueue() throws Exception
|
public void testPolFromQueue() throws Exception
|
||||||
{
|
{
|
||||||
InputSupplierUpdateStream updateStream = new InputSupplierUpdateStream(testCaseSupplier, timeDimension);
|
InputSupplierUpdateStream updateStream = new InputSupplierUpdateStream(testCaseSupplier, timeDimension);
|
||||||
Map<String, String> renamedKeys = new HashMap<String, String>();
|
Map<String, String> renamedKeys = new HashMap<String, String>();
|
||||||
|
@ -61,8 +61,6 @@ public class RenamingKeysUpdateStreamTest
|
||||||
expectedAnswer.put("i1", "value1");
|
expectedAnswer.put("i1", "value1");
|
||||||
expectedAnswer.put("i2", 2);
|
expectedAnswer.put("i2", 2);
|
||||||
expectedAnswer.put("t", 1372121562);
|
expectedAnswer.put("t", 1372121562);
|
||||||
|
|
||||||
|
|
||||||
Assert.assertEquals(expectedAnswer, renamer.pollFromQueue(waitTime, unit));
|
Assert.assertEquals(expectedAnswer, renamer.pollFromQueue(waitTime, unit));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -88,7 +86,6 @@ public class RenamingKeysUpdateStreamTest
|
||||||
renamedKeys.put("item2", "i2");
|
renamedKeys.put("item2", "i2");
|
||||||
RenamingKeysUpdateStream renamer = new RenamingKeysUpdateStream(updateStream, renamedKeys);
|
RenamingKeysUpdateStream renamer = new RenamingKeysUpdateStream(updateStream, renamedKeys);
|
||||||
Assert.assertEquals("time", renamer.getTimeDimension());
|
Assert.assertEquals("time", renamer.getTimeDimension());
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,11 +21,11 @@ package druid.examples.web;
|
||||||
|
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import java.net.UnknownHostException;
|
import java.io.IOException;
|
||||||
|
|
||||||
public class WebJsonSupplierTest
|
public class WebJsonSupplierTest
|
||||||
{
|
{
|
||||||
@Test(expected = UnknownHostException.class)
|
@Test(expected = IOException.class)
|
||||||
public void checkInvalidUrl() throws Exception
|
public void checkInvalidUrl() throws Exception
|
||||||
{
|
{
|
||||||
|
|
||||||
|
|
|
@ -28,7 +28,7 @@
|
||||||
<parent>
|
<parent>
|
||||||
<groupId>com.metamx</groupId>
|
<groupId>com.metamx</groupId>
|
||||||
<artifactId>druid</artifactId>
|
<artifactId>druid</artifactId>
|
||||||
<version>0.5.19-SNAPSHOT</version>
|
<version>0.5.33-SNAPSHOT</version>
|
||||||
</parent>
|
</parent>
|
||||||
|
|
||||||
<dependencies>
|
<dependencies>
|
||||||
|
|
|
@ -23,6 +23,9 @@ import com.metamx.druid.index.column.ColumnSelector;
|
||||||
import com.metamx.druid.kv.Indexed;
|
import com.metamx.druid.kv.Indexed;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public interface QueryableIndex extends ColumnSelector
|
public interface QueryableIndex extends ColumnSelector
|
||||||
|
@ -31,4 +34,11 @@ public interface QueryableIndex extends ColumnSelector
|
||||||
public int getNumRows();
|
public int getNumRows();
|
||||||
public Indexed<String> getColumnNames();
|
public Indexed<String> getColumnNames();
|
||||||
public Indexed<String> getAvailableDimensions();
|
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;
|
package com.metamx.druid.index;
|
||||||
|
|
||||||
|
import com.metamx.common.io.smoosh.SmooshedFileMapper;
|
||||||
import com.metamx.druid.index.column.Column;
|
import com.metamx.druid.index.column.Column;
|
||||||
import com.metamx.druid.kv.Indexed;
|
import com.metamx.druid.kv.Indexed;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -34,13 +36,15 @@ public class SimpleQueryableIndex implements QueryableIndex
|
||||||
private final Indexed<String> availableDimensions;
|
private final Indexed<String> availableDimensions;
|
||||||
private final Column timeColumn;
|
private final Column timeColumn;
|
||||||
private final Map<String, Column> otherColumns;
|
private final Map<String, Column> otherColumns;
|
||||||
|
private final SmooshedFileMapper fileMapper;
|
||||||
|
|
||||||
public SimpleQueryableIndex(
|
public SimpleQueryableIndex(
|
||||||
Interval dataInterval,
|
Interval dataInterval,
|
||||||
Indexed<String> columnNames,
|
Indexed<String> columnNames,
|
||||||
Indexed<String> dimNames,
|
Indexed<String> dimNames,
|
||||||
Column timeColumn,
|
Column timeColumn,
|
||||||
Map<String, Column> otherColumns
|
Map<String, Column> otherColumns,
|
||||||
|
SmooshedFileMapper fileMapper
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.dataInterval = dataInterval;
|
this.dataInterval = dataInterval;
|
||||||
|
@ -48,6 +52,7 @@ public class SimpleQueryableIndex implements QueryableIndex
|
||||||
this.availableDimensions = dimNames;
|
this.availableDimensions = dimNames;
|
||||||
this.timeColumn = timeColumn;
|
this.timeColumn = timeColumn;
|
||||||
this.otherColumns = otherColumns;
|
this.otherColumns = otherColumns;
|
||||||
|
this.fileMapper = fileMapper;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -85,4 +90,10 @@ public class SimpleQueryableIndex implements QueryableIndex
|
||||||
{
|
{
|
||||||
return otherColumns.get(columnName);
|
return otherColumns.get(columnName);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException
|
||||||
|
{
|
||||||
|
fileMapper.close();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -375,7 +375,8 @@ public class IndexIO
|
||||||
dimValueLookups,
|
dimValueLookups,
|
||||||
dimColumns,
|
dimColumns,
|
||||||
invertedIndexed,
|
invertedIndexed,
|
||||||
spatialIndexed
|
spatialIndexed,
|
||||||
|
smooshedFiles
|
||||||
);
|
);
|
||||||
|
|
||||||
log.debug("Mapped v8 index[%s] in %,d millis", inDir, System.currentTimeMillis() - startTime);
|
log.debug("Mapped v8 index[%s] in %,d millis", inDir, System.currentTimeMillis() - startTime);
|
||||||
|
@ -761,7 +762,8 @@ public class IndexIO
|
||||||
.setType(ValueType.LONG)
|
.setType(ValueType.LONG)
|
||||||
.setGenericColumn(new LongGenericColumnSupplier(index.timestamps))
|
.setGenericColumn(new LongGenericColumnSupplier(index.timestamps))
|
||||||
.build(),
|
.build(),
|
||||||
columns
|
columns,
|
||||||
|
index.getFileMapper()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -795,7 +797,7 @@ public class IndexIO
|
||||||
}
|
}
|
||||||
|
|
||||||
final QueryableIndex index = new SimpleQueryableIndex(
|
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);
|
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.collect.Maps;
|
||||||
import com.google.common.primitives.Ints;
|
import com.google.common.primitives.Ints;
|
||||||
import com.metamx.collections.spatial.ImmutableRTree;
|
import com.metamx.collections.spatial.ImmutableRTree;
|
||||||
|
import com.metamx.common.io.smoosh.SmooshedFileMapper;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
import com.metamx.druid.kv.ConciseCompressedIndexedInts;
|
import com.metamx.druid.kv.ConciseCompressedIndexedInts;
|
||||||
import com.metamx.druid.kv.GenericIndexed;
|
import com.metamx.druid.kv.GenericIndexed;
|
||||||
import com.metamx.druid.kv.Indexed;
|
import com.metamx.druid.kv.Indexed;
|
||||||
import com.metamx.druid.kv.IndexedList;
|
import com.metamx.druid.kv.IndexedList;
|
||||||
import com.metamx.druid.kv.IndexedLongs;
|
import com.metamx.druid.kv.IndexedLongs;
|
||||||
import com.metamx.druid.kv.IndexedRTree;
|
|
||||||
import com.metamx.druid.kv.VSizeIndexed;
|
import com.metamx.druid.kv.VSizeIndexed;
|
||||||
import com.metamx.druid.kv.VSizeIndexedInts;
|
import com.metamx.druid.kv.VSizeIndexedInts;
|
||||||
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
|
import it.uniroma3.mat.extendedset.intset.ImmutableConciseSet;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
import java.nio.ByteOrder;
|
import java.nio.ByteOrder;
|
||||||
import java.nio.LongBuffer;
|
import java.nio.LongBuffer;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
@ -57,6 +58,7 @@ public class MMappedIndex
|
||||||
final Map<String, VSizeIndexed> dimColumns;
|
final Map<String, VSizeIndexed> dimColumns;
|
||||||
final Map<String, GenericIndexed<ImmutableConciseSet>> invertedIndexes;
|
final Map<String, GenericIndexed<ImmutableConciseSet>> invertedIndexes;
|
||||||
final Map<String, ImmutableRTree> spatialIndexes;
|
final Map<String, ImmutableRTree> spatialIndexes;
|
||||||
|
final SmooshedFileMapper fileMapper;
|
||||||
|
|
||||||
private final Map<String, Integer> metricIndexes = Maps.newHashMap();
|
private final Map<String, Integer> metricIndexes = Maps.newHashMap();
|
||||||
|
|
||||||
|
@ -69,7 +71,8 @@ public class MMappedIndex
|
||||||
Map<String, GenericIndexed<String>> dimValueLookups,
|
Map<String, GenericIndexed<String>> dimValueLookups,
|
||||||
Map<String, VSizeIndexed> dimColumns,
|
Map<String, VSizeIndexed> dimColumns,
|
||||||
Map<String, GenericIndexed<ImmutableConciseSet>> invertedIndexes,
|
Map<String, GenericIndexed<ImmutableConciseSet>> invertedIndexes,
|
||||||
Map<String, ImmutableRTree> spatialIndexes
|
Map<String, ImmutableRTree> spatialIndexes,
|
||||||
|
SmooshedFileMapper fileMapper
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.availableDimensions = availableDimensions;
|
this.availableDimensions = availableDimensions;
|
||||||
|
@ -81,6 +84,7 @@ public class MMappedIndex
|
||||||
this.dimColumns = dimColumns;
|
this.dimColumns = dimColumns;
|
||||||
this.invertedIndexes = invertedIndexes;
|
this.invertedIndexes = invertedIndexes;
|
||||||
this.spatialIndexes = spatialIndexes;
|
this.spatialIndexes = spatialIndexes;
|
||||||
|
this.fileMapper = fileMapper;
|
||||||
|
|
||||||
for (int i = 0; i < availableMetrics.size(); i++) {
|
for (int i = 0; i < availableMetrics.size(); i++) {
|
||||||
metricIndexes.put(availableMetrics.get(i), i);
|
metricIndexes.put(availableMetrics.get(i), i);
|
||||||
|
@ -169,6 +173,18 @@ public class MMappedIndex
|
||||||
return (retVal == null) ? emptySet : retVal;
|
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)
|
public static MMappedIndex fromIndex(Index index)
|
||||||
{
|
{
|
||||||
log.info("Converting timestamps");
|
log.info("Converting timestamps");
|
||||||
|
@ -273,7 +289,8 @@ public class MMappedIndex
|
||||||
dimValueLookups,
|
dimValueLookups,
|
||||||
dimColumns,
|
dimColumns,
|
||||||
invertedIndexes,
|
invertedIndexes,
|
||||||
spatialIndexes
|
spatialIndexes,
|
||||||
|
null
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,9 +1,10 @@
|
||||||
package com.metamx.druid.indexer.data;
|
package com.metamx.druid.indexer.data;
|
||||||
|
|
||||||
|
import com.metamx.common.exception.FormattedException;
|
||||||
import com.metamx.druid.input.InputRow;
|
import com.metamx.druid.input.InputRow;
|
||||||
|
|
||||||
public interface InputRowParser<T>
|
public interface InputRowParser<T>
|
||||||
{
|
{
|
||||||
public InputRow parse(T input);
|
public InputRow parse(T input) throws FormattedException;
|
||||||
public void addDimensionExclusion(String dimension);
|
public void addDimensionExclusion(String dimension);
|
||||||
}
|
}
|
||||||
|
|
|
@ -4,6 +4,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
|
import com.metamx.common.exception.FormattedException;
|
||||||
import com.metamx.druid.input.InputRow;
|
import com.metamx.druid.input.InputRow;
|
||||||
import com.metamx.druid.input.MapBasedInputRow;
|
import com.metamx.druid.input.MapBasedInputRow;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
@ -37,13 +38,15 @@ public class MapInputRowParser implements InputRowParser<Map<String, Object>>
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public InputRow parse(Map<String, Object> theMap)
|
public InputRow parse(Map<String, Object> theMap) throws FormattedException
|
||||||
{
|
{
|
||||||
final List<String> dimensions = dataSpec.hasCustomDimensions()
|
final List<String> dimensions = dataSpec.hasCustomDimensions()
|
||||||
? dataSpec.getDimensions()
|
? dataSpec.getDimensions()
|
||||||
: Lists.newArrayList(Sets.difference(theMap.keySet(), dimensionExclusions));
|
: Lists.newArrayList(Sets.difference(theMap.keySet(), dimensionExclusions));
|
||||||
|
|
||||||
final DateTime timestamp = timestampSpec.extractTimestamp(theMap);
|
final DateTime timestamp;
|
||||||
|
try {
|
||||||
|
timestamp = timestampSpec.extractTimestamp(theMap);
|
||||||
if (timestamp == null) {
|
if (timestamp == null) {
|
||||||
final String input = theMap.toString();
|
final String input = theMap.toString();
|
||||||
throw new NullPointerException(
|
throw new NullPointerException(
|
||||||
|
@ -53,6 +56,13 @@ public class MapInputRowParser implements InputRowParser<Map<String, Object>>
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
throw new FormattedException.Builder()
|
||||||
|
.withErrorCode(FormattedException.ErrorCode.UNPARSABLE_TIMESTAMP)
|
||||||
|
.withMessage(e.toString())
|
||||||
|
.build();
|
||||||
|
}
|
||||||
|
|
||||||
return new MapBasedInputRow(timestamp.getMillis(), dimensions, theMap);
|
return new MapBasedInputRow(timestamp.getMillis(), dimensions, theMap);
|
||||||
}
|
}
|
||||||
|
|
|
@ -28,7 +28,7 @@
|
||||||
<parent>
|
<parent>
|
||||||
<groupId>com.metamx</groupId>
|
<groupId>com.metamx</groupId>
|
||||||
<artifactId>druid</artifactId>
|
<artifactId>druid</artifactId>
|
||||||
<version>0.5.19-SNAPSHOT</version>
|
<version>0.5.33-SNAPSHOT</version>
|
||||||
</parent>
|
</parent>
|
||||||
|
|
||||||
<dependencies>
|
<dependencies>
|
||||||
|
|
|
@ -57,9 +57,10 @@ import com.metamx.druid.jackson.DefaultObjectMapper;
|
||||||
import com.metamx.druid.shard.ShardSpec;
|
import com.metamx.druid.shard.ShardSpec;
|
||||||
import com.metamx.druid.utils.JodaUtils;
|
import com.metamx.druid.utils.JodaUtils;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||||
import org.apache.hadoop.mapreduce.Job;
|
import org.apache.hadoop.mapreduce.Job;
|
||||||
|
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.joda.time.format.ISODateTimeFormat;
|
import org.joda.time.format.ISODateTimeFormat;
|
||||||
|
@ -656,10 +657,23 @@ public class HadoopDruidIndexerConfig
|
||||||
return new Path(makeDescriptorInfoDir(), String.format("%s.json", segment.getIdentifier().replace(":", "")));
|
return new Path(makeDescriptorInfoDir(), String.format("%s.json", segment.getIdentifier().replace(":", "")));
|
||||||
}
|
}
|
||||||
|
|
||||||
public Path makeSegmentOutputPath(Bucket bucket)
|
public Path makeSegmentOutputPath(FileSystem fileSystem, Bucket bucket)
|
||||||
{
|
{
|
||||||
final Interval bucketInterval = getGranularitySpec().bucketInterval(bucket.time).get();
|
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(
|
return new Path(
|
||||||
String.format(
|
String.format(
|
||||||
"%s/%s/%s_%s/%s/%s",
|
"%s/%s/%s_%s/%s/%s",
|
||||||
|
@ -669,8 +683,7 @@ public class HadoopDruidIndexerConfig
|
||||||
bucketInterval.getEnd().toString(),
|
bucketInterval.getEnd().toString(),
|
||||||
getVersion(),
|
getVersion(),
|
||||||
bucket.partitionNum
|
bucket.partitionNum
|
||||||
)
|
));
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public Job addInputPaths(Job job) throws IOException
|
public Job addInputPaths(Job job) throws IOException
|
||||||
|
|
|
@ -375,7 +375,9 @@ public class IndexGeneratorJob implements Jobby
|
||||||
Interval interval = config.getGranularitySpec().bucketInterval(bucket.time).get();
|
Interval interval = config.getGranularitySpec().bucketInterval(bucket.time).get();
|
||||||
|
|
||||||
int attemptNumber = context.getTaskAttemptID().getId();
|
int attemptNumber = context.getTaskAttemptID().getId();
|
||||||
Path indexBasePath = config.makeSegmentOutputPath(bucket);
|
|
||||||
|
FileSystem fileSystem = FileSystem.get(context.getConfiguration());
|
||||||
|
Path indexBasePath = config.makeSegmentOutputPath(fileSystem, bucket);
|
||||||
Path indexZipFilePath = new Path(indexBasePath, String.format("index.zip.%s", attemptNumber));
|
Path indexZipFilePath = new Path(indexBasePath, String.format("index.zip.%s", attemptNumber));
|
||||||
final FileSystem infoFS = config.makeDescriptorInfoDir().getFileSystem(context.getConfiguration());
|
final FileSystem infoFS = config.makeDescriptorInfoDir().getFileSystem(context.getConfiguration());
|
||||||
final FileSystem outputFS = indexBasePath.getFileSystem(context.getConfiguration());
|
final FileSystem outputFS = indexBasePath.getFileSystem(context.getConfiguration());
|
||||||
|
|
|
@ -26,7 +26,10 @@ import com.metamx.druid.indexer.granularity.UniformGranularitySpec;
|
||||||
import com.metamx.druid.indexer.partitions.PartitionsSpec;
|
import com.metamx.druid.indexer.partitions.PartitionsSpec;
|
||||||
import com.metamx.druid.indexer.updater.DbUpdaterJobSpec;
|
import com.metamx.druid.indexer.updater.DbUpdaterJobSpec;
|
||||||
import com.metamx.druid.jackson.DefaultObjectMapper;
|
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.joda.time.Interval;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
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)
|
private <T> T jsonReadWriteRead(String s, Class<T> klass)
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
|
|
|
@ -28,7 +28,7 @@
|
||||||
<parent>
|
<parent>
|
||||||
<groupId>com.metamx</groupId>
|
<groupId>com.metamx</groupId>
|
||||||
<artifactId>druid</artifactId>
|
<artifactId>druid</artifactId>
|
||||||
<version>0.5.19-SNAPSHOT</version>
|
<version>0.5.33-SNAPSHOT</version>
|
||||||
</parent>
|
</parent>
|
||||||
|
|
||||||
<dependencies>
|
<dependencies>
|
||||||
|
|
|
@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Objects;
|
import com.google.common.base.Objects;
|
||||||
import com.google.common.base.Preconditions;
|
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
|
* 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)
|
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)
|
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)
|
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 String id;
|
||||||
private final Status status;
|
private final Status status;
|
||||||
private final long duration;
|
private final long duration;
|
||||||
|
private final TaskResource resource;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
private TaskStatus(
|
private TaskStatus(
|
||||||
@JsonProperty("id") String id,
|
@JsonProperty("id") String id,
|
||||||
@JsonProperty("status") Status status,
|
@JsonProperty("status") Status status,
|
||||||
@JsonProperty("duration") long duration
|
@JsonProperty("duration") long duration,
|
||||||
|
@JsonProperty("resource") TaskResource resource
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.id = id;
|
this.id = id;
|
||||||
this.status = status;
|
this.status = status;
|
||||||
this.duration = duration;
|
this.duration = duration;
|
||||||
|
this.resource = resource == null ? new TaskResource(id, 1) : resource;
|
||||||
|
|
||||||
// Check class invariants.
|
// Check class invariants.
|
||||||
Preconditions.checkNotNull(id, "id");
|
Preconditions.checkNotNull(id, "id");
|
||||||
|
@ -93,6 +97,12 @@ public class TaskStatus
|
||||||
return duration;
|
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,
|
* 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.
|
* isSuccess, or isFailure will be true at any one time.
|
||||||
|
@ -134,7 +144,7 @@ public class TaskStatus
|
||||||
|
|
||||||
public TaskStatus withDuration(long _duration)
|
public TaskStatus withDuration(long _duration)
|
||||||
{
|
{
|
||||||
return new TaskStatus(id, status, _duration);
|
return new TaskStatus(id, status, _duration, resource);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -144,6 +154,7 @@ public class TaskStatus
|
||||||
.add("id", id)
|
.add("id", id)
|
||||||
.add("status", status)
|
.add("status", status)
|
||||||
.add("duration", duration)
|
.add("duration", duration)
|
||||||
|
.add("resource", resource)
|
||||||
.toString();
|
.toString();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -37,6 +37,7 @@ import com.metamx.druid.loading.SegmentLoadingException;
|
||||||
import com.metamx.druid.loading.SingleSegmentLoader;
|
import com.metamx.druid.loading.SingleSegmentLoader;
|
||||||
import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
|
import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
|
||||||
import com.metamx.emitter.service.ServiceEmitter;
|
import com.metamx.emitter.service.ServiceEmitter;
|
||||||
|
import com.metamx.metrics.MonitorScheduler;
|
||||||
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
|
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
|
@ -58,6 +59,7 @@ public class TaskToolbox
|
||||||
private final DataSegmentAnnouncer segmentAnnouncer;
|
private final DataSegmentAnnouncer segmentAnnouncer;
|
||||||
private final ServerView newSegmentServerView;
|
private final ServerView newSegmentServerView;
|
||||||
private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate;
|
private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate;
|
||||||
|
private final MonitorScheduler monitorScheduler;
|
||||||
private final ObjectMapper objectMapper;
|
private final ObjectMapper objectMapper;
|
||||||
|
|
||||||
public TaskToolbox(
|
public TaskToolbox(
|
||||||
|
@ -71,6 +73,7 @@ public class TaskToolbox
|
||||||
DataSegmentAnnouncer segmentAnnouncer,
|
DataSegmentAnnouncer segmentAnnouncer,
|
||||||
ServerView newSegmentServerView,
|
ServerView newSegmentServerView,
|
||||||
QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate,
|
QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate,
|
||||||
|
MonitorScheduler monitorScheduler,
|
||||||
ObjectMapper objectMapper
|
ObjectMapper objectMapper
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
@ -84,6 +87,7 @@ public class TaskToolbox
|
||||||
this.segmentAnnouncer = segmentAnnouncer;
|
this.segmentAnnouncer = segmentAnnouncer;
|
||||||
this.newSegmentServerView = newSegmentServerView;
|
this.newSegmentServerView = newSegmentServerView;
|
||||||
this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate;
|
this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate;
|
||||||
|
this.monitorScheduler = monitorScheduler;
|
||||||
this.objectMapper = objectMapper;
|
this.objectMapper = objectMapper;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -127,6 +131,11 @@ public class TaskToolbox
|
||||||
return queryRunnerFactoryConglomerate;
|
return queryRunnerFactoryConglomerate;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public MonitorScheduler getMonitorScheduler()
|
||||||
|
{
|
||||||
|
return monitorScheduler;
|
||||||
|
}
|
||||||
|
|
||||||
public ObjectMapper getObjectMapper()
|
public ObjectMapper getObjectMapper()
|
||||||
{
|
{
|
||||||
return objectMapper;
|
return objectMapper;
|
||||||
|
@ -156,7 +165,8 @@ public class TaskToolbox
|
||||||
return retVal;
|
return retVal;
|
||||||
}
|
}
|
||||||
|
|
||||||
public File getTaskWorkDir() {
|
public File getTaskWorkDir()
|
||||||
|
{
|
||||||
return new File(new File(config.getBaseTaskDir(), task.getId()), "work");
|
return new File(new File(config.getBaseTaskDir(), task.getId()), "work");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -29,6 +29,7 @@ import com.metamx.druid.indexing.common.config.TaskConfig;
|
||||||
import com.metamx.druid.indexing.common.task.Task;
|
import com.metamx.druid.indexing.common.task.Task;
|
||||||
import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
|
import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
|
||||||
import com.metamx.emitter.service.ServiceEmitter;
|
import com.metamx.emitter.service.ServiceEmitter;
|
||||||
|
import com.metamx.metrics.MonitorScheduler;
|
||||||
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
|
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -45,6 +46,7 @@ public class TaskToolboxFactory
|
||||||
private final DataSegmentAnnouncer segmentAnnouncer;
|
private final DataSegmentAnnouncer segmentAnnouncer;
|
||||||
private final ServerView newSegmentServerView;
|
private final ServerView newSegmentServerView;
|
||||||
private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate;
|
private final QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate;
|
||||||
|
private final MonitorScheduler monitorScheduler;
|
||||||
private final ObjectMapper objectMapper;
|
private final ObjectMapper objectMapper;
|
||||||
|
|
||||||
public TaskToolboxFactory(
|
public TaskToolboxFactory(
|
||||||
|
@ -57,6 +59,7 @@ public class TaskToolboxFactory
|
||||||
DataSegmentAnnouncer segmentAnnouncer,
|
DataSegmentAnnouncer segmentAnnouncer,
|
||||||
ServerView newSegmentServerView,
|
ServerView newSegmentServerView,
|
||||||
QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate,
|
QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate,
|
||||||
|
MonitorScheduler monitorScheduler,
|
||||||
ObjectMapper objectMapper
|
ObjectMapper objectMapper
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
@ -69,6 +72,7 @@ public class TaskToolboxFactory
|
||||||
this.segmentAnnouncer = segmentAnnouncer;
|
this.segmentAnnouncer = segmentAnnouncer;
|
||||||
this.newSegmentServerView = newSegmentServerView;
|
this.newSegmentServerView = newSegmentServerView;
|
||||||
this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate;
|
this.queryRunnerFactoryConglomerate = queryRunnerFactoryConglomerate;
|
||||||
|
this.monitorScheduler = monitorScheduler;
|
||||||
this.objectMapper = objectMapper;
|
this.objectMapper = objectMapper;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -85,6 +89,7 @@ public class TaskToolboxFactory
|
||||||
segmentAnnouncer,
|
segmentAnnouncer,
|
||||||
newSegmentServerView,
|
newSegmentServerView,
|
||||||
queryRunnerFactoryConglomerate,
|
queryRunnerFactoryConglomerate,
|
||||||
|
monitorScheduler,
|
||||||
objectMapper
|
objectMapper
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
|
@ -65,6 +65,8 @@ public class RemoteTaskActionClient implements TaskActionClient
|
||||||
|
|
||||||
final String response;
|
final String response;
|
||||||
|
|
||||||
|
log.info("Submitting action for task[%s] to coordinator[%s]: %s", task.getId(), serviceUri, taskAction);
|
||||||
|
|
||||||
try {
|
try {
|
||||||
response = httpClient.post(serviceUri.toURL())
|
response = httpClient.post(serviceUri.toURL())
|
||||||
.setContent("application/json", dataToSend)
|
.setContent("application/json", dataToSend)
|
||||||
|
@ -107,7 +109,7 @@ public class RemoteTaskActionClient implements TaskActionClient
|
||||||
final String scheme;
|
final String scheme;
|
||||||
final String host;
|
final String host;
|
||||||
final int port;
|
final int port;
|
||||||
final String path = "/mmx/merger/v1/action";
|
final String path = "/druid/indexer/v1/action";
|
||||||
|
|
||||||
if (instance == null) {
|
if (instance == null) {
|
||||||
throw new ISE("Cannot find instance of indexer to talk to!");
|
throw new ISE("Cannot find instance of indexer to talk to!");
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package com.metamx.druid.indexing.common.config;
|
package com.metamx.druid.indexing.common.config;
|
||||||
|
|
||||||
|
import com.google.common.base.Joiner;
|
||||||
import org.skife.config.Config;
|
import org.skife.config.Config;
|
||||||
import org.skife.config.Default;
|
import org.skife.config.Default;
|
||||||
|
|
||||||
|
@ -26,13 +27,30 @@ import java.io.File;
|
||||||
|
|
||||||
public abstract class TaskConfig
|
public abstract class TaskConfig
|
||||||
{
|
{
|
||||||
|
private static Joiner joiner = Joiner.on("/");
|
||||||
|
|
||||||
|
@Config("druid.indexer.baseDir")
|
||||||
|
@Default("/tmp/")
|
||||||
|
public abstract String getBaseDir();
|
||||||
|
|
||||||
@Config("druid.indexer.taskDir")
|
@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")
|
@Config("druid.indexer.rowFlushBoundary")
|
||||||
@Default("500000")
|
@Default("500000")
|
||||||
public abstract int getDefaultRowFlushBoundary();
|
public abstract int getDefaultRowFlushBoundary();
|
||||||
|
|
||||||
@Config("druid.indexer.hadoopWorkingPath")
|
private String defaultPath(String subPath)
|
||||||
public abstract String getHadoopWorkingPath();
|
{
|
||||||
|
return joiner.join(getBaseDir(), subPath);
|
||||||
|
}
|
||||||
}
|
}
|
|
@ -1,10 +1,15 @@
|
||||||
package com.metamx.druid.indexing.common.config;
|
package com.metamx.druid.indexing.common.config;
|
||||||
|
|
||||||
import org.skife.config.Config;
|
import org.skife.config.Config;
|
||||||
|
import org.skife.config.Default;
|
||||||
import org.skife.config.DefaultNull;
|
import org.skife.config.DefaultNull;
|
||||||
|
|
||||||
public abstract class TaskLogConfig
|
public abstract class TaskLogConfig
|
||||||
{
|
{
|
||||||
|
@Config("druid.indexer.logs.type")
|
||||||
|
@Default("noop")
|
||||||
|
public abstract String getLogType();
|
||||||
|
|
||||||
@Config("druid.indexer.logs.s3bucket")
|
@Config("druid.indexer.logs.s3bucket")
|
||||||
@DefaultNull
|
@DefaultNull
|
||||||
public abstract String getLogStorageBucket();
|
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;
|
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
|
* 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.
|
* 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;
|
package com.metamx.druid.indexing.common.index;
|
||||||
|
|
||||||
import com.google.common.base.Optional;
|
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;
|
public void unregister(final String key);
|
||||||
private final ServiceAnnouncer serviceAnnouncer;
|
|
||||||
private final ConcurrentMap<String, ChatHandler> handlers;
|
|
||||||
|
|
||||||
public ChatHandlerProvider(
|
public Optional<ChatHandler> get(final String key);
|
||||||
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);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
package com.metamx.druid.indexing.common.index;
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
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
|
* 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")
|
@JsonTypeName("receiver")
|
||||||
public class EventReceiverFirehoseFactory implements FirehoseFactory
|
public class EventReceiverFirehoseFactory implements FirehoseFactory
|
||||||
|
@ -41,14 +60,14 @@ public class EventReceiverFirehoseFactory implements FirehoseFactory
|
||||||
private final String firehoseId;
|
private final String firehoseId;
|
||||||
private final int bufferSize;
|
private final int bufferSize;
|
||||||
private final MapInputRowParser parser;
|
private final MapInputRowParser parser;
|
||||||
private final Optional<ChatHandlerProvider> chatHandlerProvider;
|
private final Optional<EventReceivingChatHandlerProvider> chatHandlerProvider;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public EventReceiverFirehoseFactory(
|
public EventReceiverFirehoseFactory(
|
||||||
@JsonProperty("firehoseId") String firehoseId,
|
@JsonProperty("firehoseId") String firehoseId,
|
||||||
@JsonProperty("bufferSize") Integer bufferSize,
|
@JsonProperty("bufferSize") Integer bufferSize,
|
||||||
@JsonProperty("parser") MapInputRowParser parser,
|
@JsonProperty("parser") MapInputRowParser parser,
|
||||||
@JacksonInject("chatHandlerProvider") ChatHandlerProvider chatHandlerProvider
|
@JacksonInject("chatHandlerProvider") EventReceivingChatHandlerProvider chatHandlerProvider
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.firehoseId = Preconditions.checkNotNull(firehoseId, "firehoseId");
|
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;
|
private final String groupId;
|
||||||
|
|
||||||
@JsonIgnore
|
@JsonIgnore
|
||||||
private final String availabilityGroup;
|
private final TaskResource taskResource;
|
||||||
|
|
||||||
@JsonIgnore
|
@JsonIgnore
|
||||||
private final String dataSource;
|
private final String dataSource;
|
||||||
|
@ -53,23 +53,23 @@ public abstract class AbstractTask implements Task
|
||||||
|
|
||||||
protected AbstractTask(String id, String dataSource, Interval interval)
|
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)
|
protected AbstractTask(String id, String groupId, String dataSource, Interval interval)
|
||||||
{
|
{
|
||||||
this.id = Preconditions.checkNotNull(id, "id");
|
this.id = Preconditions.checkNotNull(id, "id");
|
||||||
this.groupId = Preconditions.checkNotNull(groupId, "groupId");
|
this.groupId = Preconditions.checkNotNull(groupId, "groupId");
|
||||||
this.availabilityGroup = id;
|
this.taskResource = new TaskResource(id, 1);
|
||||||
this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource");
|
this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource");
|
||||||
this.interval = Optional.fromNullable(interval);
|
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.id = Preconditions.checkNotNull(id, "id");
|
||||||
this.groupId = Preconditions.checkNotNull(groupId, "groupId");
|
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.dataSource = Preconditions.checkNotNull(dataSource, "dataSource");
|
||||||
this.interval = Optional.fromNullable(interval);
|
this.interval = Optional.fromNullable(interval);
|
||||||
}
|
}
|
||||||
|
@ -90,9 +90,9 @@ public abstract class AbstractTask implements Task
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
@Override
|
@Override
|
||||||
public String getAvailabilityGroup()
|
public TaskResource getTaskResource()
|
||||||
{
|
{
|
||||||
return availabilityGroup;
|
return taskResource;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -127,12 +127,6 @@ public abstract class AbstractTask implements Task
|
||||||
return TaskStatus.running(id);
|
return TaskStatus.running(id);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public void shutdown()
|
|
||||||
{
|
|
||||||
// Do nothing.
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString()
|
public String toString()
|
||||||
{
|
{
|
||||||
|
@ -172,19 +166,16 @@ public abstract class AbstractTask implements Task
|
||||||
|
|
||||||
AbstractTask that = (AbstractTask) o;
|
AbstractTask that = (AbstractTask) o;
|
||||||
|
|
||||||
if (dataSource != null ? !dataSource.equals(that.dataSource) : that.dataSource != null) {
|
if (!id.equals(that.id)) {
|
||||||
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) {
|
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode()
|
||||||
|
{
|
||||||
|
return id.hashCode();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -88,7 +88,6 @@ public class IndexDeterminePartitionsTask extends AbstractTask
|
||||||
super(
|
super(
|
||||||
id != null ? id : makeTaskId(groupId, interval.getStart(), interval.getEnd()),
|
id != null ? id : makeTaskId(groupId, interval.getStart(), interval.getEnd()),
|
||||||
groupId,
|
groupId,
|
||||||
makeTaskId(groupId, interval.getStart(), interval.getEnd()),
|
|
||||||
schema.getDataSource(),
|
schema.getDataSource(),
|
||||||
Preconditions.checkNotNull(interval, "interval")
|
Preconditions.checkNotNull(interval, "interval")
|
||||||
);
|
);
|
||||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonIgnore;
|
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.common.io.Closeables;
|
import com.google.common.io.Closeables;
|
||||||
import com.metamx.common.exception.FormattedException;
|
import com.metamx.common.exception.FormattedException;
|
||||||
|
@ -41,15 +42,18 @@ import com.metamx.druid.input.InputRow;
|
||||||
import com.metamx.druid.query.FinalizeResultsQueryRunner;
|
import com.metamx.druid.query.FinalizeResultsQueryRunner;
|
||||||
import com.metamx.druid.query.QueryRunner;
|
import com.metamx.druid.query.QueryRunner;
|
||||||
import com.metamx.druid.query.QueryRunnerFactory;
|
import com.metamx.druid.query.QueryRunnerFactory;
|
||||||
|
import com.metamx.druid.query.QueryRunnerFactoryConglomerate;
|
||||||
import com.metamx.druid.query.QueryToolChest;
|
import com.metamx.druid.query.QueryToolChest;
|
||||||
|
import com.metamx.druid.realtime.FireDepartment;
|
||||||
import com.metamx.druid.realtime.FireDepartmentConfig;
|
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.Schema;
|
||||||
import com.metamx.druid.realtime.SegmentPublisher;
|
import com.metamx.druid.realtime.SegmentPublisher;
|
||||||
|
import com.metamx.druid.realtime.firehose.Firehose;
|
||||||
import com.metamx.druid.realtime.firehose.FirehoseFactory;
|
import com.metamx.druid.realtime.firehose.FirehoseFactory;
|
||||||
import com.metamx.druid.realtime.firehose.GracefulShutdownFirehose;
|
|
||||||
import com.metamx.druid.realtime.plumber.Plumber;
|
import com.metamx.druid.realtime.plumber.Plumber;
|
||||||
import com.metamx.druid.realtime.plumber.RealtimePlumberSchool;
|
import com.metamx.druid.realtime.plumber.RealtimePlumberSchool;
|
||||||
|
import com.metamx.druid.realtime.plumber.RejectionPolicyFactory;
|
||||||
import com.metamx.druid.realtime.plumber.Sink;
|
import com.metamx.druid.realtime.plumber.Sink;
|
||||||
import com.metamx.druid.realtime.plumber.VersioningPolicy;
|
import com.metamx.druid.realtime.plumber.VersioningPolicy;
|
||||||
import com.metamx.emitter.EmittingLogger;
|
import com.metamx.emitter.EmittingLogger;
|
||||||
|
@ -87,43 +91,44 @@ public class RealtimeIndexTask extends AbstractTask
|
||||||
@JsonIgnore
|
@JsonIgnore
|
||||||
private final IndexGranularity segmentGranularity;
|
private final IndexGranularity segmentGranularity;
|
||||||
|
|
||||||
|
@JsonIgnore
|
||||||
|
private final RejectionPolicyFactory rejectionPolicyFactory;
|
||||||
|
|
||||||
@JsonIgnore
|
@JsonIgnore
|
||||||
private volatile Plumber plumber = null;
|
private volatile Plumber plumber = null;
|
||||||
|
|
||||||
@JsonIgnore
|
@JsonIgnore
|
||||||
private volatile TaskToolbox toolbox = null;
|
private volatile QueryRunnerFactoryConglomerate queryRunnerFactoryConglomerate = null;
|
||||||
|
|
||||||
@JsonIgnore
|
|
||||||
private volatile GracefulShutdownFirehose firehose = null;
|
|
||||||
|
|
||||||
@JsonIgnore
|
|
||||||
private final Object lock = new Object();
|
|
||||||
|
|
||||||
@JsonIgnore
|
|
||||||
private volatile boolean shutdown = false;
|
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public RealtimeIndexTask(
|
public RealtimeIndexTask(
|
||||||
@JsonProperty("id") String id,
|
@JsonProperty("id") String id,
|
||||||
@JsonProperty("availabilityGroup") String availabilityGroup,
|
@JsonProperty("resource") TaskResource taskResource,
|
||||||
@JsonProperty("schema") Schema schema,
|
@JsonProperty("schema") Schema schema,
|
||||||
@JsonProperty("firehose") FirehoseFactory firehoseFactory,
|
@JsonProperty("firehose") FirehoseFactory firehoseFactory,
|
||||||
@JsonProperty("fireDepartmentConfig") FireDepartmentConfig fireDepartmentConfig,
|
@JsonProperty("fireDepartmentConfig") FireDepartmentConfig fireDepartmentConfig,
|
||||||
@JsonProperty("windowPeriod") Period windowPeriod,
|
@JsonProperty("windowPeriod") Period windowPeriod,
|
||||||
@JsonProperty("segmentGranularity") IndexGranularity segmentGranularity
|
@JsonProperty("segmentGranularity") IndexGranularity segmentGranularity,
|
||||||
|
@JsonProperty("rejectionPolicy") RejectionPolicyFactory rejectionPolicyFactory
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(
|
super(
|
||||||
id != null
|
id == null
|
||||||
? id
|
? makeTaskId(schema.getDataSource(), schema.getShardSpec().getPartitionNum(), new DateTime().toString())
|
||||||
: makeTaskId(schema.getDataSource(), schema.getShardSpec().getPartitionNum(), new DateTime().toString()),
|
:id,
|
||||||
String.format(
|
String.format(
|
||||||
"index_realtime_%s",
|
"index_realtime_%s",
|
||||||
schema.getDataSource()
|
schema.getDataSource()
|
||||||
),
|
),
|
||||||
availabilityGroup != null
|
taskResource == null
|
||||||
? availabilityGroup
|
? new TaskResource(
|
||||||
: makeTaskId(schema.getDataSource(), schema.getShardSpec().getPartitionNum(), new DateTime().toString()),
|
makeTaskId(
|
||||||
|
schema.getDataSource(),
|
||||||
|
schema.getShardSpec().getPartitionNum(),
|
||||||
|
new DateTime().toString()
|
||||||
|
), 1
|
||||||
|
)
|
||||||
|
: taskResource,
|
||||||
schema.getDataSource(),
|
schema.getDataSource(),
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
|
@ -133,6 +138,7 @@ public class RealtimeIndexTask extends AbstractTask
|
||||||
this.fireDepartmentConfig = fireDepartmentConfig;
|
this.fireDepartmentConfig = fireDepartmentConfig;
|
||||||
this.windowPeriod = windowPeriod;
|
this.windowPeriod = windowPeriod;
|
||||||
this.segmentGranularity = segmentGranularity;
|
this.segmentGranularity = segmentGranularity;
|
||||||
|
this.rejectionPolicyFactory = rejectionPolicyFactory;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -151,7 +157,7 @@ public class RealtimeIndexTask extends AbstractTask
|
||||||
public <T> QueryRunner<T> getQueryRunner(Query<T> query)
|
public <T> QueryRunner<T> getQueryRunner(Query<T> query)
|
||||||
{
|
{
|
||||||
if (plumber != null) {
|
if (plumber != null) {
|
||||||
QueryRunnerFactory<T, Query<T>> factory = toolbox.getQueryRunnerFactoryConglomerate().findFactory(query);
|
QueryRunnerFactory<T, Query<T>> factory = queryRunnerFactoryConglomerate.findFactory(query);
|
||||||
QueryToolChest<T, Query<T>> toolChest = factory.getToolchest();
|
QueryToolChest<T, Query<T>> toolChest = factory.getToolchest();
|
||||||
|
|
||||||
return new FinalizeResultsQueryRunner<T>(plumber.getQueryRunner(query), toolChest);
|
return new FinalizeResultsQueryRunner<T>(plumber.getQueryRunner(query), toolChest);
|
||||||
|
@ -175,21 +181,9 @@ public class RealtimeIndexTask extends AbstractTask
|
||||||
|
|
||||||
boolean normalExit = true;
|
boolean normalExit = true;
|
||||||
|
|
||||||
final FireDepartmentMetrics metrics = new FireDepartmentMetrics();
|
// Set up firehose
|
||||||
final Period intermediatePersistPeriod = fireDepartmentConfig.getIntermediatePersistPeriod();
|
final Period intermediatePersistPeriod = fireDepartmentConfig.getIntermediatePersistPeriod();
|
||||||
|
final Firehose firehose = firehoseFactory.connect();
|
||||||
synchronized (lock) {
|
|
||||||
if (shutdown) {
|
|
||||||
return TaskStatus.success(getId());
|
|
||||||
}
|
|
||||||
|
|
||||||
log.info(
|
|
||||||
"Wrapping firehose in GracefulShutdownFirehose with segmentGranularity[%s] and windowPeriod[%s]",
|
|
||||||
segmentGranularity,
|
|
||||||
windowPeriod
|
|
||||||
);
|
|
||||||
firehose = new GracefulShutdownFirehose(firehoseFactory.connect(), segmentGranularity, windowPeriod);
|
|
||||||
}
|
|
||||||
|
|
||||||
// It would be nice to get the PlumberSchool in the constructor. Although that will need jackson injectables for
|
// It would be nice to get the PlumberSchool in the constructor. Although that will need jackson injectables for
|
||||||
// stuff like the ServerView, which seems kind of odd? Perhaps revisit this when Guice has been introduced.
|
// stuff like the ServerView, which seems kind of odd? Perhaps revisit this when Guice has been introduced.
|
||||||
|
@ -286,12 +280,22 @@ public class RealtimeIndexTask extends AbstractTask
|
||||||
realtimePlumberSchool.setServerView(toolbox.getNewSegmentServerView());
|
realtimePlumberSchool.setServerView(toolbox.getNewSegmentServerView());
|
||||||
realtimePlumberSchool.setServiceEmitter(toolbox.getEmitter());
|
realtimePlumberSchool.setServiceEmitter(toolbox.getEmitter());
|
||||||
|
|
||||||
this.toolbox = toolbox;
|
if (this.rejectionPolicyFactory != null) {
|
||||||
this.plumber = realtimePlumberSchool.findPlumber(schema, metrics);
|
realtimePlumberSchool.setRejectionPolicyFactory(rejectionPolicyFactory);
|
||||||
|
}
|
||||||
|
|
||||||
|
final FireDepartment fireDepartment = new FireDepartment(schema, fireDepartmentConfig, null, null);
|
||||||
|
final RealtimeMetricsMonitor metricsMonitor = new RealtimeMetricsMonitor(ImmutableList.of(fireDepartment));
|
||||||
|
this.queryRunnerFactoryConglomerate = toolbox.getQueryRunnerFactoryConglomerate();
|
||||||
|
this.plumber = realtimePlumberSchool.findPlumber(schema, fireDepartment.getMetrics());
|
||||||
|
|
||||||
try {
|
try {
|
||||||
plumber.startJob();
|
plumber.startJob();
|
||||||
|
|
||||||
|
// Set up metrics emission
|
||||||
|
toolbox.getMonitorScheduler().addMonitor(metricsMonitor);
|
||||||
|
|
||||||
|
// Time to read data!
|
||||||
long nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis();
|
long nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis();
|
||||||
while (firehose.hasMore()) {
|
while (firehose.hasMore()) {
|
||||||
final InputRow inputRow;
|
final InputRow inputRow;
|
||||||
|
@ -303,7 +307,7 @@ public class RealtimeIndexTask extends AbstractTask
|
||||||
|
|
||||||
final Sink sink = plumber.getSink(inputRow.getTimestampFromEpoch());
|
final Sink sink = plumber.getSink(inputRow.getTimestampFromEpoch());
|
||||||
if (sink == null) {
|
if (sink == null) {
|
||||||
metrics.incrementThrownAway();
|
fireDepartment.getMetrics().incrementThrownAway();
|
||||||
log.debug("Throwing away event[%s]", inputRow);
|
log.debug("Throwing away event[%s]", inputRow);
|
||||||
|
|
||||||
if (System.currentTimeMillis() > nextFlush) {
|
if (System.currentTimeMillis() > nextFlush) {
|
||||||
|
@ -319,7 +323,7 @@ public class RealtimeIndexTask extends AbstractTask
|
||||||
}
|
}
|
||||||
|
|
||||||
int currCount = sink.add(inputRow);
|
int currCount = sink.add(inputRow);
|
||||||
metrics.incrementProcessed();
|
fireDepartment.getMetrics().incrementProcessed();
|
||||||
if (currCount >= fireDepartmentConfig.getMaxRowsInMemory() || System.currentTimeMillis() > nextFlush) {
|
if (currCount >= fireDepartmentConfig.getMaxRowsInMemory() || System.currentTimeMillis() > nextFlush) {
|
||||||
plumber.persist(firehose.commit());
|
plumber.persist(firehose.commit());
|
||||||
nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis();
|
nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis();
|
||||||
|
@ -327,7 +331,7 @@ public class RealtimeIndexTask extends AbstractTask
|
||||||
}
|
}
|
||||||
catch (FormattedException e) {
|
catch (FormattedException e) {
|
||||||
log.warn(e, "unparseable line");
|
log.warn(e, "unparseable line");
|
||||||
metrics.incrementUnparseable();
|
fireDepartment.getMetrics().incrementUnparseable();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -348,6 +352,7 @@ public class RealtimeIndexTask extends AbstractTask
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
Closeables.closeQuietly(firehose);
|
Closeables.closeQuietly(firehose);
|
||||||
|
toolbox.getMonitorScheduler().removeMonitor(metricsMonitor);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -355,22 +360,6 @@ public class RealtimeIndexTask extends AbstractTask
|
||||||
return TaskStatus.success(getId());
|
return TaskStatus.success(getId());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public void shutdown()
|
|
||||||
{
|
|
||||||
try {
|
|
||||||
synchronized (lock) {
|
|
||||||
shutdown = true;
|
|
||||||
if (firehose != null) {
|
|
||||||
firehose.shutdown();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
catch (IOException e) {
|
|
||||||
throw Throwables.propagate(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
public Schema getSchema()
|
public Schema getSchema()
|
||||||
{
|
{
|
||||||
|
@ -401,6 +390,12 @@ public class RealtimeIndexTask extends AbstractTask
|
||||||
return segmentGranularity;
|
return segmentGranularity;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@JsonProperty("rejectionPolicy")
|
||||||
|
public RejectionPolicyFactory getRejectionPolicyFactory()
|
||||||
|
{
|
||||||
|
return rejectionPolicyFactory;
|
||||||
|
}
|
||||||
|
|
||||||
public static class TaskActionSegmentPublisher implements SegmentPublisher
|
public static class TaskActionSegmentPublisher implements SegmentPublisher
|
||||||
{
|
{
|
||||||
final Task task;
|
final Task task;
|
||||||
|
|
|
@ -72,11 +72,10 @@ public interface Task
|
||||||
public String getGroupId();
|
public String getGroupId();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns availability group ID of this task. Tasks the same availability group cannot be assigned to the same
|
* Returns a {@link com.metamx.druid.indexing.common.task.TaskResource} for this task. Task resources define specific
|
||||||
* worker. If tasks do not have this restriction, a common convention is to set the availability group ID to the
|
* worker requirements a task may require.
|
||||||
* task ID.
|
|
||||||
*/
|
*/
|
||||||
public String getAvailabilityGroup();
|
public TaskResource getTaskResource();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns a descriptive label for this task type. Used for metrics emission and logging.
|
* Returns a descriptive label for this task type. Used for metrics emission and logging.
|
||||||
|
@ -134,10 +133,4 @@ public interface Task
|
||||||
* @throws Exception
|
* @throws Exception
|
||||||
*/
|
*/
|
||||||
public TaskStatus run(TaskToolbox toolbox) throws Exception;
|
public TaskStatus run(TaskToolbox toolbox) throws Exception;
|
||||||
|
|
||||||
/**
|
|
||||||
* Best-effort task cancellation. May or may not do anything. Calling this multiple times may have
|
|
||||||
* a stronger effect.
|
|
||||||
*/
|
|
||||||
public void shutdown();
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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
|
DataSegment segment
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(id, groupId, id, dataSource, interval);
|
super(id, groupId, dataSource, interval);
|
||||||
|
|
||||||
this.segment = segment;
|
this.segment = segment;
|
||||||
}
|
}
|
||||||
|
@ -205,13 +205,6 @@ public class VersionConverterTask extends AbstractTask
|
||||||
segment.getShardSpec().getPartitionNum()
|
segment.getShardSpec().getPartitionNum()
|
||||||
),
|
),
|
||||||
groupId,
|
groupId,
|
||||||
joinId(
|
|
||||||
groupId,
|
|
||||||
"sub",
|
|
||||||
segment.getInterval().getStart(),
|
|
||||||
segment.getInterval().getEnd(),
|
|
||||||
segment.getShardSpec().getPartitionNum()
|
|
||||||
),
|
|
||||||
segment.getDataSource(),
|
segment.getDataSource(),
|
||||||
segment.getInterval()
|
segment.getInterval()
|
||||||
);
|
);
|
||||||
|
|
|
@ -32,6 +32,7 @@ import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
import com.google.common.io.ByteStreams;
|
import com.google.common.io.ByteStreams;
|
||||||
import com.google.common.io.Closeables;
|
import com.google.common.io.Closeables;
|
||||||
|
import com.google.common.io.Closer;
|
||||||
import com.google.common.io.Files;
|
import com.google.common.io.Files;
|
||||||
import com.google.common.io.InputSupplier;
|
import com.google.common.io.InputSupplier;
|
||||||
import com.google.common.util.concurrent.ListenableFuture;
|
import com.google.common.util.concurrent.ListenableFuture;
|
||||||
|
@ -48,6 +49,7 @@ import com.metamx.druid.indexing.worker.executor.ExecutorMain;
|
||||||
import com.metamx.emitter.EmittingLogger;
|
import com.metamx.emitter.EmittingLogger;
|
||||||
import org.apache.commons.io.FileUtils;
|
import org.apache.commons.io.FileUtils;
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
|
@ -94,6 +96,12 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
this.jsonMapper = jsonMapper;
|
this.jsonMapper = jsonMapper;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void bootstrap(List<Task> tasks)
|
||||||
|
{
|
||||||
|
// do nothing
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ListenableFuture<TaskStatus> run(final Task task)
|
public ListenableFuture<TaskStatus> run(final Task task)
|
||||||
{
|
{
|
||||||
|
@ -114,6 +122,8 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
|
|
||||||
final ProcessHolder processHolder;
|
final ProcessHolder processHolder;
|
||||||
|
|
||||||
|
try {
|
||||||
|
final Closer closer = Closer.create();
|
||||||
try {
|
try {
|
||||||
if (!attemptDir.mkdirs()) {
|
if (!attemptDir.mkdirs()) {
|
||||||
throw new IOException(String.format("Could not create directories: %s", attemptDir));
|
throw new IOException(String.format("Could not create directories: %s", attemptDir));
|
||||||
|
@ -203,33 +213,24 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
);
|
);
|
||||||
|
|
||||||
processHolder = taskInfo.processHolder;
|
processHolder = taskInfo.processHolder;
|
||||||
|
processHolder.registerWithCloser(closer);
|
||||||
}
|
}
|
||||||
|
|
||||||
log.info("Logging task %s output to: %s", task.getId(), logFile);
|
log.info("Logging task %s output to: %s", task.getId(), logFile);
|
||||||
|
|
||||||
final OutputStream toProc = processHolder.process.getOutputStream();
|
|
||||||
final InputStream fromProc = processHolder.process.getInputStream();
|
final InputStream fromProc = processHolder.process.getInputStream();
|
||||||
final OutputStream toLogfile = Files.newOutputStreamSupplier(logFile).getOutput();
|
final OutputStream toLogfile = closer.register(
|
||||||
|
Files.newOutputStreamSupplier(logFile).getOutput()
|
||||||
|
);
|
||||||
|
|
||||||
boolean runFailed = false;
|
boolean runFailed = true;
|
||||||
|
|
||||||
try {
|
|
||||||
ByteStreams.copy(fromProc, toLogfile);
|
ByteStreams.copy(fromProc, toLogfile);
|
||||||
final int statusCode = processHolder.process.waitFor();
|
final int statusCode = processHolder.process.waitFor();
|
||||||
log.info("Process exited with status[%d] for task: %s", statusCode, task.getId());
|
log.info("Process exited with status[%d] for task: %s", statusCode, task.getId());
|
||||||
|
|
||||||
if (statusCode != 0) {
|
if (statusCode == 0) {
|
||||||
runFailed = true;
|
runFailed = false;
|
||||||
}
|
|
||||||
}
|
|
||||||
catch (Exception e) {
|
|
||||||
log.warn(e, "Failed to read from process for task: %s", task.getId());
|
|
||||||
runFailed = true;
|
|
||||||
}
|
|
||||||
finally {
|
|
||||||
Closeables.closeQuietly(fromProc);
|
|
||||||
Closeables.closeQuietly(toLogfile);
|
|
||||||
Closeables.closeQuietly(toProc);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Upload task logs
|
// Upload task logs
|
||||||
|
@ -247,6 +248,11 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
// Process exited unsuccessfully
|
// Process exited unsuccessfully
|
||||||
return TaskStatus.failure(task.getId());
|
return TaskStatus.failure(task.getId());
|
||||||
}
|
}
|
||||||
|
} catch (Throwable t) {
|
||||||
|
throw closer.rethrow(t);
|
||||||
|
} finally {
|
||||||
|
closer.close();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
log.info(e, "Exception caught during execution");
|
log.info(e, "Exception caught during execution");
|
||||||
|
@ -311,33 +317,11 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
}
|
}
|
||||||
|
|
||||||
if (taskInfo.processHolder != null) {
|
if (taskInfo.processHolder != null) {
|
||||||
final int shutdowns = taskInfo.processHolder.shutdowns.getAndIncrement();
|
|
||||||
if (shutdowns == 0) {
|
|
||||||
log.info("Attempting to gracefully shutdown task: %s", taskid);
|
|
||||||
try {
|
|
||||||
// This is gross, but it may still be nicer than talking to the forked JVM via HTTP.
|
|
||||||
final OutputStream out = taskInfo.processHolder.process.getOutputStream();
|
|
||||||
out.write(
|
|
||||||
jsonMapper.writeValueAsBytes(
|
|
||||||
ImmutableMap.of(
|
|
||||||
"shutdown",
|
|
||||||
"now"
|
|
||||||
)
|
|
||||||
)
|
|
||||||
);
|
|
||||||
out.write('\n');
|
|
||||||
out.flush();
|
|
||||||
}
|
|
||||||
catch (IOException e) {
|
|
||||||
throw Throwables.propagate(e);
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
// Will trigger normal failure mechanisms due to process exit
|
// Will trigger normal failure mechanisms due to process exit
|
||||||
log.info("Killing process for task: %s", taskid);
|
log.info("Killing process for task: %s", taskid);
|
||||||
taskInfo.processHolder.process.destroy();
|
taskInfo.processHolder.process.destroy();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Collection<TaskRunnerWorkItem> getRunningTasks()
|
public Collection<TaskRunnerWorkItem> getRunningTasks()
|
||||||
|
@ -429,7 +413,6 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
private final Process process;
|
private final Process process;
|
||||||
private final File logFile;
|
private final File logFile;
|
||||||
private final int port;
|
private final int port;
|
||||||
private final AtomicInteger shutdowns = new AtomicInteger(0);
|
|
||||||
|
|
||||||
private ProcessHolder(Process process, File logFile, int port)
|
private ProcessHolder(Process process, File logFile, int port)
|
||||||
{
|
{
|
||||||
|
@ -437,5 +420,11 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
this.logFile = logFile;
|
this.logFile = logFile;
|
||||||
this.port = port;
|
this.port = port;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void registerWithCloser(Closer closer)
|
||||||
|
{
|
||||||
|
closer.register(process.getInputStream());
|
||||||
|
closer.register(process.getOutputStream());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,22 +24,18 @@ import com.google.common.base.Charsets;
|
||||||
import com.google.common.base.Joiner;
|
import com.google.common.base.Joiner;
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Predicate;
|
|
||||||
import com.google.common.base.Stopwatch;
|
import com.google.common.base.Stopwatch;
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.MinMaxPriorityQueue;
|
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
import com.google.common.io.InputSupplier;
|
import com.google.common.io.InputSupplier;
|
||||||
import com.google.common.primitives.Ints;
|
import com.google.common.primitives.Ints;
|
||||||
import com.google.common.util.concurrent.ListenableFuture;
|
import com.google.common.util.concurrent.ListenableFuture;
|
||||||
import com.google.common.util.concurrent.SettableFuture;
|
import com.google.common.util.concurrent.SettableFuture;
|
||||||
import com.metamx.common.ISE;
|
import com.metamx.common.ISE;
|
||||||
import com.metamx.common.guava.FunctionalIterable;
|
|
||||||
import com.metamx.common.lifecycle.LifecycleStart;
|
import com.metamx.common.lifecycle.LifecycleStart;
|
||||||
import com.metamx.common.lifecycle.LifecycleStop;
|
import com.metamx.common.lifecycle.LifecycleStop;
|
||||||
import com.metamx.druid.indexing.common.RetryPolicy;
|
import com.metamx.druid.curator.cache.PathChildrenCacheFactory;
|
||||||
import com.metamx.druid.indexing.common.RetryPolicyFactory;
|
|
||||||
import com.metamx.druid.indexing.common.TaskStatus;
|
import com.metamx.druid.indexing.common.TaskStatus;
|
||||||
import com.metamx.druid.indexing.common.task.Task;
|
import com.metamx.druid.indexing.common.task.Task;
|
||||||
import com.metamx.druid.indexing.common.tasklogs.TaskLogProvider;
|
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.emitter.EmittingLogger;
|
||||||
import com.metamx.http.client.HttpClient;
|
import com.metamx.http.client.HttpClient;
|
||||||
import com.metamx.http.client.response.InputStreamResponseHandler;
|
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.CuratorFramework;
|
||||||
|
import org.apache.curator.framework.recipes.cache.ChildData;
|
||||||
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
|
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
|
||||||
import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
|
import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
|
||||||
import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
|
import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
|
||||||
import org.apache.curator.utils.ZKPaths;
|
import org.apache.curator.utils.ZKPaths;
|
||||||
import org.apache.zookeeper.CreateMode;
|
import org.apache.zookeeper.CreateMode;
|
||||||
|
import org.jboss.netty.handler.codec.http.HttpResponseStatus;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Duration;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
|
@ -68,54 +66,50 @@ import java.util.Comparator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.TreeSet;
|
||||||
import java.util.concurrent.Callable;
|
import java.util.concurrent.Callable;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.ConcurrentSkipListSet;
|
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
import java.util.concurrent.Executors;
|
import java.util.concurrent.Executors;
|
||||||
import java.util.concurrent.ScheduledExecutorService;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The RemoteTaskRunner's primary responsibility is to assign tasks to worker nodes and manage retries in failure
|
* The RemoteTaskRunner's primary responsibility is to assign tasks to worker nodes.
|
||||||
* scenarios. The RemoteTaskRunner keeps track of which workers are running which tasks and manages coordinator and
|
* The RemoteTaskRunner uses Zookeeper to keep track of which workers are running which tasks. Tasks are assigned by
|
||||||
* worker interactions over Zookeeper. The RemoteTaskRunner is event driven and updates state according to ephemeral
|
* creating ephemeral nodes in ZK that workers must remove. Workers announce the statuses of the tasks they are running.
|
||||||
* node changes in ZK.
|
* 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/>
|
* <p/>
|
||||||
* The RemoteTaskRunner will assign tasks to a node until the node hits capacity. At that point, task assignment will
|
* 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.
|
* 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.
|
* For example, {@link com.metamx.druid.indexing.coordinator.scaling.ResourceManagementScheduler} can take care of these duties.
|
||||||
* <p/>
|
* <p/>
|
||||||
* If a worker node becomes inexplicably disconnected from Zk, the RemoteTaskRunner will automatically retry any tasks
|
* If a worker node becomes inexplicably disconnected from Zk, the RemoteTaskRunner will fail any tasks associated with the worker.
|
||||||
* that were associated with the node.
|
|
||||||
* <p/>
|
* <p/>
|
||||||
* The RemoteTaskRunner uses ZK for job management and assignment and http for IPC messages.
|
* The RemoteTaskRunner uses ZK for job management and assignment and http for IPC messages.
|
||||||
*/
|
*/
|
||||||
public class RemoteTaskRunner implements TaskRunner, TaskLogProvider
|
public class RemoteTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
{
|
{
|
||||||
private static final EmittingLogger log = new EmittingLogger(RemoteTaskRunner.class);
|
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 static final Joiner JOINER = Joiner.on("/");
|
||||||
|
|
||||||
private final ObjectMapper jsonMapper;
|
private final ObjectMapper jsonMapper;
|
||||||
private final RemoteTaskRunnerConfig config;
|
private final RemoteTaskRunnerConfig config;
|
||||||
private final CuratorFramework cf;
|
private final CuratorFramework cf;
|
||||||
|
private final PathChildrenCacheFactory pathChildrenCacheFactory;
|
||||||
private final PathChildrenCache workerPathCache;
|
private final PathChildrenCache workerPathCache;
|
||||||
private final ScheduledExecutorService scheduledExec;
|
|
||||||
private final RetryPolicyFactory retryPolicyFactory;
|
|
||||||
private final AtomicReference<WorkerSetupData> workerSetupData;
|
private final AtomicReference<WorkerSetupData> workerSetupData;
|
||||||
private final HttpClient httpClient;
|
private final HttpClient httpClient;
|
||||||
|
|
||||||
// all workers that exist in ZK
|
// all workers that exist in ZK
|
||||||
private final Map<String, ZkWorker> zkWorkers = new ConcurrentHashMap<String, ZkWorker>();
|
private final Map<String, ZkWorker> zkWorkers = new ConcurrentHashMap<String, ZkWorker>();
|
||||||
// all tasks that have been assigned to a worker
|
// 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
|
// tasks that have not yet run
|
||||||
private final TaskRunnerWorkQueue pendingTasks = new TaskRunnerWorkQueue();
|
private final RemoteTaskRunnerWorkQueue pendingTasks = new RemoteTaskRunnerWorkQueue();
|
||||||
// idempotent task retry
|
|
||||||
private final Set<String> tasksToRetry = new ConcurrentSkipListSet<String>();
|
|
||||||
|
|
||||||
private final ExecutorService runPendingTasksExec = Executors.newSingleThreadExecutor();
|
private final ExecutorService runPendingTasksExec = Executors.newSingleThreadExecutor();
|
||||||
|
|
||||||
|
@ -127,9 +121,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
ObjectMapper jsonMapper,
|
ObjectMapper jsonMapper,
|
||||||
RemoteTaskRunnerConfig config,
|
RemoteTaskRunnerConfig config,
|
||||||
CuratorFramework cf,
|
CuratorFramework cf,
|
||||||
PathChildrenCache workerPathCache,
|
PathChildrenCacheFactory pathChildrenCacheFactory,
|
||||||
ScheduledExecutorService scheduledExec,
|
|
||||||
RetryPolicyFactory retryPolicyFactory,
|
|
||||||
AtomicReference<WorkerSetupData> workerSetupData,
|
AtomicReference<WorkerSetupData> workerSetupData,
|
||||||
HttpClient httpClient
|
HttpClient httpClient
|
||||||
)
|
)
|
||||||
|
@ -137,9 +129,8 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
this.jsonMapper = jsonMapper;
|
this.jsonMapper = jsonMapper;
|
||||||
this.config = config;
|
this.config = config;
|
||||||
this.cf = cf;
|
this.cf = cf;
|
||||||
this.workerPathCache = workerPathCache;
|
this.pathChildrenCacheFactory = pathChildrenCacheFactory;
|
||||||
this.scheduledExec = scheduledExec;
|
this.workerPathCache = pathChildrenCacheFactory.make(cf, config.getIndexerAnnouncementPath());
|
||||||
this.retryPolicyFactory = retryPolicyFactory;
|
|
||||||
this.workerSetupData = workerSetupData;
|
this.workerSetupData = workerSetupData;
|
||||||
this.httpClient = httpClient;
|
this.httpClient = httpClient;
|
||||||
}
|
}
|
||||||
|
@ -159,25 +150,37 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
@Override
|
@Override
|
||||||
public void childEvent(CuratorFramework client, final PathChildrenCacheEvent event) throws Exception
|
public void childEvent(CuratorFramework client, final PathChildrenCacheEvent event) throws Exception
|
||||||
{
|
{
|
||||||
if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED)) {
|
Worker worker;
|
||||||
final Worker worker = jsonMapper.readValue(
|
switch (event.getType()) {
|
||||||
|
case CHILD_ADDED:
|
||||||
|
worker = jsonMapper.readValue(
|
||||||
event.getData().getData(),
|
event.getData().getData(),
|
||||||
Worker.class
|
Worker.class
|
||||||
);
|
);
|
||||||
log.info("Worker[%s] reportin' for duty!", worker.getHost());
|
addWorker(worker, PathChildrenCache.StartMode.NORMAL);
|
||||||
addWorker(worker);
|
break;
|
||||||
} else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED)) {
|
case CHILD_REMOVED:
|
||||||
final Worker worker = jsonMapper.readValue(
|
worker = jsonMapper.readValue(
|
||||||
event.getData().getData(),
|
event.getData().getData(),
|
||||||
Worker.class
|
Worker.class
|
||||||
);
|
);
|
||||||
log.info("Kaboom! Worker[%s] removed!", worker.getHost());
|
|
||||||
removeWorker(worker);
|
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;
|
started = true;
|
||||||
}
|
}
|
||||||
|
@ -197,6 +200,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
for (ZkWorker zkWorker : zkWorkers.values()) {
|
for (ZkWorker zkWorker : zkWorkers.values()) {
|
||||||
zkWorker.close();
|
zkWorker.close();
|
||||||
}
|
}
|
||||||
|
workerPathCache.close();
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw Throwables.propagate(e);
|
||||||
|
@ -213,13 +217,13 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Collection<TaskRunnerWorkItem> getRunningTasks()
|
public Collection<RemoteTaskRunnerWorkItem> getRunningTasks()
|
||||||
{
|
{
|
||||||
return runningTasks.values();
|
return runningTasks.values();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Collection<TaskRunnerWorkItem> getPendingTasks()
|
public Collection<RemoteTaskRunnerWorkItem> getPendingTasks()
|
||||||
{
|
{
|
||||||
return pendingTasks.values();
|
return pendingTasks.values();
|
||||||
}
|
}
|
||||||
|
@ -227,18 +231,46 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
public ZkWorker findWorkerRunningTask(String taskId)
|
public ZkWorker findWorkerRunningTask(String taskId)
|
||||||
{
|
{
|
||||||
for (ZkWorker zkWorker : zkWorkers.values()) {
|
for (ZkWorker zkWorker : zkWorkers.values()) {
|
||||||
if (zkWorker.getRunningTasks().contains(taskId)) {
|
if (zkWorker.isRunningTask(taskId)) {
|
||||||
return zkWorker;
|
return zkWorker;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return null;
|
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
|
@Override
|
||||||
public ListenableFuture<TaskStatus> run(final Task task)
|
public ListenableFuture<TaskStatus> run(final Task task)
|
||||||
{
|
{
|
||||||
if (runningTasks.containsKey(task.getId()) || pendingTasks.containsKey(task.getId())) {
|
RemoteTaskRunnerWorkItem runningTask = runningTasks.get(task.getId());
|
||||||
throw new ISE("Assigned a task[%s] that is already running or pending, WTF is happening?!", 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(
|
RemoteTaskRunnerWorkItem pendingTask = pendingTasks.get(task.getId());
|
||||||
task, SettableFuture.<TaskStatus>create(), retryPolicyFactory.makeRetryPolicy(), new DateTime()
|
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);
|
addPendingTask(taskRunnerWorkItem);
|
||||||
return taskRunnerWorkItem.getResult();
|
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.
|
* Finds the worker running the task and forwards the shutdown signal to the worker.
|
||||||
*
|
*
|
||||||
* @param taskId
|
* @param taskId - task id to shutdown
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void shutdown(String taskId)
|
public void shutdown(String taskId)
|
||||||
|
@ -275,39 +315,29 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
final ZkWorker zkWorker = findWorkerRunningTask(taskId);
|
final ZkWorker zkWorker = findWorkerRunningTask(taskId);
|
||||||
|
|
||||||
if (zkWorker == null) {
|
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);
|
log.info("Can't shutdown! No worker running task %s", taskId);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
final RetryPolicy shutdownRetryPolicy = retryPolicyFactory.makeRetryPolicy();
|
|
||||||
final URL url = workerURL(zkWorker.getWorker(), String.format("/task/%s/shutdown", taskId));
|
|
||||||
|
|
||||||
while (!shutdownRetryPolicy.hasExceededRetryThreshold()) {
|
|
||||||
try {
|
try {
|
||||||
final String response = httpClient.post(url)
|
final URL url = makeWorkerURL(zkWorker.getWorker(), String.format("/task/%s/shutdown", taskId));
|
||||||
.go(STRING_RESPONSE_HANDLER)
|
final StatusResponseHolder response = httpClient.post(url)
|
||||||
|
.go(RESPONSE_HANDLER)
|
||||||
.get();
|
.get();
|
||||||
log.info("Sent shutdown message to worker: %s, response: %s", zkWorker.getWorker().getHost(), response);
|
|
||||||
|
|
||||||
return;
|
log.info(
|
||||||
|
"Sent shutdown message to worker: %s, status %s, response: %s",
|
||||||
|
zkWorker.getWorker().getHost(),
|
||||||
|
response.getStatus(),
|
||||||
|
response.getContent()
|
||||||
|
);
|
||||||
|
|
||||||
|
if (!response.getStatus().equals(HttpResponseStatus.ACCEPTED)) {
|
||||||
|
log.error("Shutdown failed for %s! Are you sure the task was running?", taskId);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
log.error(e, "Exception shutting down taskId: %s", taskId);
|
|
||||||
|
|
||||||
if (shutdownRetryPolicy.hasExceededRetryThreshold()) {
|
|
||||||
throw Throwables.propagate(e);
|
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);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -321,7 +351,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
return Optional.absent();
|
return Optional.absent();
|
||||||
} else {
|
} else {
|
||||||
// Worker is still running this task
|
// 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(
|
return Optional.<InputSupplier<InputStream>>of(
|
||||||
new InputSupplier<InputStream>()
|
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);
|
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
|
* 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());
|
log.info("Added pending task %s", taskRunnerWorkItem.getTask().getId());
|
||||||
|
|
||||||
|
@ -388,8 +416,8 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
try {
|
try {
|
||||||
// make a copy of the pending tasks because assignTask may delete tasks from pending and move them
|
// make a copy of the pending tasks because assignTask may delete tasks from pending and move them
|
||||||
// into running status
|
// into running status
|
||||||
List<TaskRunnerWorkItem> copy = Lists.newArrayList(pendingTasks.values());
|
List<RemoteTaskRunnerWorkItem> copy = Lists.newArrayList(pendingTasks.values());
|
||||||
for (TaskRunnerWorkItem taskWrapper : copy) {
|
for (RemoteTaskRunnerWorkItem taskWrapper : copy) {
|
||||||
assignTask(taskWrapper);
|
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.
|
* 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
|
* @param taskRunnerWorkItem - the task to assign
|
||||||
*/
|
*/
|
||||||
private void assignTask(TaskRunnerWorkItem taskRunnerWorkItem)
|
private void assignTask(RemoteTaskRunnerWorkItem taskRunnerWorkItem)
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
final String taskId = taskRunnerWorkItem.getTask().getId();
|
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 (runningTasks.containsKey(taskId) || findWorkerRunningTask(taskId) != null) {
|
||||||
if (zkWorker != null) {
|
log.info("Task[%s] already running.", taskId);
|
||||||
final Worker worker = zkWorker.getWorker();
|
|
||||||
log.info("Worker[%s] is already running task[%s].", worker.getHost(), taskId);
|
|
||||||
runningTasks.put(taskId, pendingTasks.remove(taskId));
|
|
||||||
log.info("Task %s switched from pending to running", taskId);
|
|
||||||
} else {
|
} else {
|
||||||
// Nothing running this task, announce it in ZK for a worker to run it
|
// 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) {
|
if (zkWorker != null) {
|
||||||
announceTask(zkWorker.getWorker(), taskRunnerWorkItem);
|
announceTask(zkWorker.getWorker(), taskRunnerWorkItem);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
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 theWorker The worker the task is assigned to
|
||||||
* @param taskRunnerWorkItem The task to be assigned
|
* @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();
|
final Task task = taskRunnerWorkItem.getTask();
|
||||||
|
|
||||||
|
@ -525,7 +514,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
Stopwatch timeoutStopwatch = new Stopwatch();
|
Stopwatch timeoutStopwatch = new Stopwatch();
|
||||||
timeoutStopwatch.start();
|
timeoutStopwatch.start();
|
||||||
synchronized (statusLock) {
|
synchronized (statusLock) {
|
||||||
while (!isWorkerRunningTask(theWorker.getHost(), task.getId())) {
|
while (!isWorkerRunningTask(theWorker, task)) {
|
||||||
statusLock.wait(config.getTaskAssignmentTimeoutDuration().getMillis());
|
statusLock.wait(config.getTaskAssignmentTimeoutDuration().getMillis());
|
||||||
if (timeoutStopwatch.elapsed(TimeUnit.MILLISECONDS) >= config.getTaskAssignmentTimeoutDuration().getMillis()) {
|
if (timeoutStopwatch.elapsed(TimeUnit.MILLISECONDS) >= config.getTaskAssignmentTimeoutDuration().getMillis()) {
|
||||||
log.error(
|
log.error(
|
||||||
|
@ -534,7 +523,8 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
task.getId(),
|
task.getId(),
|
||||||
config.getTaskAssignmentTimeoutDuration()
|
config.getTaskAssignmentTimeoutDuration()
|
||||||
);
|
);
|
||||||
retryTask(runningTasks.get(task.getId()));
|
|
||||||
|
taskRunnerWorkItem.setResult(TaskStatus.failure(taskRunnerWorkItem.getTask().getId()));
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -548,11 +538,13 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
*
|
*
|
||||||
* @param worker - contains metadata for a worker that has appeared in ZK
|
* @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 {
|
try {
|
||||||
final String workerStatusPath = JOINER.join(config.getIndexerStatusPath(), worker.getHost());
|
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(
|
final ZkWorker zkWorker = new ZkWorker(
|
||||||
worker,
|
worker,
|
||||||
statusCache,
|
statusCache,
|
||||||
|
@ -560,43 +552,39 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
);
|
);
|
||||||
|
|
||||||
// Add status listener to the watcher for status changes
|
// Add status listener to the watcher for status changes
|
||||||
statusCache.getListenable().addListener(
|
zkWorker.addListener(
|
||||||
new PathChildrenCacheListener()
|
new PathChildrenCacheListener()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
|
public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception
|
||||||
{
|
{
|
||||||
|
String taskId;
|
||||||
|
RemoteTaskRunnerWorkItem taskRunnerWorkItem;
|
||||||
synchronized (statusLock) {
|
synchronized (statusLock) {
|
||||||
try {
|
try {
|
||||||
if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_ADDED) ||
|
switch (event.getType()) {
|
||||||
event.getType().equals(PathChildrenCacheEvent.Type.CHILD_UPDATED)) {
|
case CHILD_ADDED:
|
||||||
final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath());
|
case CHILD_UPDATED:
|
||||||
|
taskId = ZKPaths.getNodeFromPath(event.getData().getPath());
|
||||||
final TaskStatus taskStatus = jsonMapper.readValue(
|
final TaskStatus taskStatus = jsonMapper.readValue(
|
||||||
event.getData().getData(), TaskStatus.class
|
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(
|
log.info(
|
||||||
"Worker[%s] wrote %s status for task: %s",
|
"Worker[%s] wrote %s status for task: %s",
|
||||||
worker.getHost(),
|
zkWorker.getWorker().getHost(),
|
||||||
taskStatus.getStatusCode(),
|
taskStatus.getStatusCode(),
|
||||||
taskId
|
taskId
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
||||||
// Synchronizing state with ZK
|
// Synchronizing state with ZK
|
||||||
statusLock.notify();
|
statusLock.notify();
|
||||||
|
|
||||||
final TaskRunnerWorkItem taskRunnerWorkItem = runningTasks.get(taskId);
|
taskRunnerWorkItem = runningTasks.get(taskId);
|
||||||
if (taskRunnerWorkItem == null) {
|
if (taskRunnerWorkItem == null) {
|
||||||
log.warn(
|
log.warn(
|
||||||
"WTF?! Worker[%s] announcing a status for a task I didn't know about: %s",
|
"WTF?! Worker[%s] announcing a status for a task I didn't know about: %s",
|
||||||
worker.getHost(),
|
zkWorker.getWorker().getHost(),
|
||||||
taskId
|
taskId
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -611,21 +599,23 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
|
|
||||||
// Worker is done with this task
|
// Worker is done with this task
|
||||||
zkWorker.setLastCompletedTaskTime(new DateTime());
|
zkWorker.setLastCompletedTaskTime(new DateTime());
|
||||||
cleanup(worker.getHost(), taskId);
|
cleanup(zkWorker.getWorker().getHost(), taskId);
|
||||||
runPendingTasks();
|
runPendingTasks();
|
||||||
}
|
}
|
||||||
} else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED)) {
|
break;
|
||||||
final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath());
|
case CHILD_REMOVED:
|
||||||
TaskRunnerWorkItem taskRunnerWorkItem = runningTasks.get(taskId);
|
taskId = ZKPaths.getNodeFromPath(event.getData().getPath());
|
||||||
|
taskRunnerWorkItem = runningTasks.get(taskId);
|
||||||
if (taskRunnerWorkItem != null) {
|
if (taskRunnerWorkItem != null) {
|
||||||
log.info("Task %s just disappeared!", taskId);
|
log.info("Task %s just disappeared!", taskId);
|
||||||
retryTask(taskRunnerWorkItem);
|
taskRunnerWorkItem.setResult(TaskStatus.failure(taskRunnerWorkItem.getTask().getId()));
|
||||||
}
|
}
|
||||||
|
break;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
log.makeAlert(e, "Failed to handle new worker status")
|
log.makeAlert(e, "Failed to handle new worker status")
|
||||||
.addData("worker", worker.getHost())
|
.addData("worker", zkWorker.getWorker().getHost())
|
||||||
.addData("znode", event.getData().getPath())
|
.addData("znode", event.getData().getPath())
|
||||||
.emit();
|
.emit();
|
||||||
}
|
}
|
||||||
|
@ -633,10 +623,13 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
|
|
||||||
|
zkWorker.start(startMode);
|
||||||
zkWorkers.put(worker.getHost(), zkWorker);
|
zkWorkers.put(worker.getHost(), zkWorker);
|
||||||
statusCache.start();
|
|
||||||
|
|
||||||
runPendingTasks();
|
runPendingTasks();
|
||||||
|
|
||||||
|
return zkWorker;
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw Throwables.propagate(e);
|
||||||
|
@ -652,38 +645,35 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
*/
|
*/
|
||||||
private void removeWorker(final Worker worker)
|
private void removeWorker(final Worker worker)
|
||||||
{
|
{
|
||||||
|
log.info("Kaboom! Worker[%s] removed!", worker.getHost());
|
||||||
|
|
||||||
ZkWorker zkWorker = zkWorkers.get(worker.getHost());
|
ZkWorker zkWorker = zkWorkers.get(worker.getHost());
|
||||||
if (zkWorker != null) {
|
if (zkWorker != null) {
|
||||||
try {
|
try {
|
||||||
Set<String> tasksToRetry = Sets.newHashSet(
|
for (String assignedTask : cf.getChildren()
|
||||||
cf.getChildren()
|
.forPath(JOINER.join(config.getIndexerTaskPath(), worker.getHost()))) {
|
||||||
.forPath(JOINER.join(config.getIndexerTaskPath(), worker.getHost()))
|
RemoteTaskRunnerWorkItem taskRunnerWorkItem = runningTasks.get(assignedTask);
|
||||||
);
|
|
||||||
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);
|
|
||||||
if (taskRunnerWorkItem != null) {
|
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) {
|
if (cf.checkExists().forPath(taskPath) != null) {
|
||||||
cf.delete().guaranteed().forPath(taskPath);
|
cf.delete().guaranteed().forPath(taskPath);
|
||||||
}
|
}
|
||||||
retryTask(taskRunnerWorkItem);
|
taskRunnerWorkItem.setResult(TaskStatus.failure(taskRunnerWorkItem.getTask().getId()));
|
||||||
} else {
|
} 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) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw Throwables.propagate(e);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
|
try {
|
||||||
|
zkWorker.close();
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
log.error(e, "Exception closing worker %s!", worker.getHost());
|
||||||
|
}
|
||||||
zkWorkers.remove(worker.getHost());
|
zkWorkers.remove(worker.getHost());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -691,48 +681,33 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider
|
||||||
|
|
||||||
private ZkWorker findWorkerForTask(final Task task)
|
private ZkWorker findWorkerForTask(final Task task)
|
||||||
{
|
{
|
||||||
try {
|
TreeSet<ZkWorker> sortedWorkers = Sets.newTreeSet(
|
||||||
final MinMaxPriorityQueue<ZkWorker> workerQueue = MinMaxPriorityQueue.<ZkWorker>orderedBy(
|
|
||||||
new Comparator<ZkWorker>()
|
new Comparator<ZkWorker>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public int compare(ZkWorker w1, ZkWorker w2)
|
public int compare(
|
||||||
{
|
ZkWorker zkWorker, ZkWorker zkWorker2
|
||||||
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);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
)
|
)
|
||||||
);
|
{
|
||||||
|
int retVal = -Ints.compare(zkWorker.getCurrCapacityUsed(), zkWorker2.getCurrCapacityUsed());
|
||||||
|
if (retVal == 0) {
|
||||||
|
retVal = zkWorker.getWorker().getHost().compareTo(zkWorker2.getWorker().getHost());
|
||||||
|
}
|
||||||
|
|
||||||
if (workerQueue.isEmpty()) {
|
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;
|
||||||
|
}
|
||||||
|
}
|
||||||
log.debug("Worker nodes %s do not have capacity to run any more tasks!", zkWorkers.values());
|
log.debug("Worker nodes %s do not have capacity to run any more tasks!", zkWorkers.values());
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
return workerQueue.peek();
|
|
||||||
}
|
|
||||||
catch (Exception e) {
|
|
||||||
throw Throwables.propagate(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
|
@ -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
|
@Override
|
||||||
public TaskRunnerWorkItem put(String s, TaskRunnerWorkItem taskRunnerWorkItem)
|
public RemoteTaskRunnerWorkItem put(String s, RemoteTaskRunnerWorkItem taskRunnerWorkItem)
|
||||||
{
|
{
|
||||||
return super.put(s, taskRunnerWorkItem.withQueueInsertionTime(new DateTime()));
|
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.Optional;
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
|
import com.metamx.common.ISE;
|
||||||
import com.metamx.common.lifecycle.Lifecycle;
|
import com.metamx.common.lifecycle.Lifecycle;
|
||||||
import com.metamx.common.lifecycle.LifecycleStart;
|
import com.metamx.common.lifecycle.LifecycleStart;
|
||||||
import com.metamx.common.lifecycle.LifecycleStop;
|
import com.metamx.common.lifecycle.LifecycleStop;
|
||||||
import com.metamx.druid.curator.discovery.ServiceAnnouncer;
|
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.TaskActionClient;
|
||||||
import com.metamx.druid.indexing.common.actions.TaskActionClientFactory;
|
import com.metamx.druid.indexing.common.actions.TaskActionClientFactory;
|
||||||
import com.metamx.druid.indexing.common.task.Task;
|
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.exec.TaskConsumer;
|
||||||
import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementScheduler;
|
import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementScheduler;
|
||||||
import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementSchedulerFactory;
|
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.EmittingLogger;
|
||||||
import com.metamx.emitter.service.ServiceEmitter;
|
import com.metamx.emitter.service.ServiceEmitter;
|
||||||
import org.apache.curator.framework.CuratorFramework;
|
import org.apache.curator.framework.CuratorFramework;
|
||||||
|
@ -88,7 +89,6 @@ public class TaskMasterLifecycle
|
||||||
log.info("By the power of Grayskull, I have the power!");
|
log.info("By the power of Grayskull, I have the power!");
|
||||||
|
|
||||||
taskRunner = runnerFactory.build();
|
taskRunner = runnerFactory.build();
|
||||||
resourceManagementScheduler = managementSchedulerFactory.build(taskRunner);
|
|
||||||
final TaskConsumer taskConsumer = new TaskConsumer(
|
final TaskConsumer taskConsumer = new TaskConsumer(
|
||||||
taskQueue,
|
taskQueue,
|
||||||
taskRunner,
|
taskRunner,
|
||||||
|
@ -101,12 +101,34 @@ public class TaskMasterLifecycle
|
||||||
|
|
||||||
// Sensible order to start stuff:
|
// Sensible order to start stuff:
|
||||||
final Lifecycle leaderLifecycle = new Lifecycle();
|
final Lifecycle leaderLifecycle = new Lifecycle();
|
||||||
leaderLifecycle.addManagedInstance(taskQueue);
|
|
||||||
leaderLifecycle.addManagedInstance(taskRunner);
|
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);
|
Initialization.announceDefaultService(serviceDiscoveryConfig, serviceAnnouncer, leaderLifecycle);
|
||||||
leaderLifecycle.addManagedInstance(taskConsumer);
|
leaderLifecycle.addManagedInstance(taskConsumer);
|
||||||
|
|
||||||
|
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);
|
leaderLifecycle.addManagedInstance(resourceManagementScheduler);
|
||||||
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
leaderLifecycle.start();
|
leaderLifecycle.start();
|
||||||
|
|
|
@ -23,6 +23,7 @@ import com.google.common.base.Optional;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
import com.google.common.collect.ArrayListMultimap;
|
import com.google.common.collect.ArrayListMultimap;
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Multimap;
|
import com.google.common.collect.Multimap;
|
||||||
import com.google.common.collect.Ordering;
|
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
|
* Starts this task queue. Allows {@link #add(Task)} to accept new tasks. This should not be called on
|
||||||
* an already-started queue.
|
* an already-started queue.
|
||||||
|
|
|
@ -24,6 +24,7 @@ import com.metamx.druid.indexing.common.TaskStatus;
|
||||||
import com.metamx.druid.indexing.common.task.Task;
|
import com.metamx.druid.indexing.common.task.Task;
|
||||||
|
|
||||||
import java.util.Collection;
|
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
|
* 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
|
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.
|
* Run a task. The returned status should be some kind of completed status.
|
||||||
*
|
*
|
||||||
|
@ -40,14 +51,13 @@ public interface TaskRunner
|
||||||
public ListenableFuture<TaskStatus> run(Task task);
|
public ListenableFuture<TaskStatus> run(Task task);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Best-effort task cancellation. May or may not do anything. Calling this multiple times may have
|
* Best-effort task shutdown. May or may not do anything.
|
||||||
* a stronger effect.
|
|
||||||
*/
|
*/
|
||||||
public void shutdown(String taskid);
|
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();
|
public Collection<ZkWorker> getWorkers();
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,7 +22,6 @@ package com.metamx.druid.indexing.coordinator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.collect.ComparisonChain;
|
import com.google.common.collect.ComparisonChain;
|
||||||
import com.google.common.util.concurrent.ListenableFuture;
|
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.TaskStatus;
|
||||||
import com.metamx.druid.indexing.common.task.Task;
|
import com.metamx.druid.indexing.common.task.Task;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
@ -35,22 +34,29 @@ public class TaskRunnerWorkItem implements Comparable<TaskRunnerWorkItem>
|
||||||
{
|
{
|
||||||
private final Task task;
|
private final Task task;
|
||||||
private final ListenableFuture<TaskStatus> result;
|
private final ListenableFuture<TaskStatus> result;
|
||||||
private final RetryPolicy retryPolicy;
|
|
||||||
private final DateTime createdTime;
|
private final DateTime createdTime;
|
||||||
|
|
||||||
private volatile DateTime queueInsertionTime;
|
private volatile DateTime queueInsertionTime;
|
||||||
|
|
||||||
|
public TaskRunnerWorkItem(
|
||||||
|
Task task,
|
||||||
|
ListenableFuture<TaskStatus> result
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this(task, result, new DateTime(), new DateTime());
|
||||||
|
}
|
||||||
|
|
||||||
public TaskRunnerWorkItem(
|
public TaskRunnerWorkItem(
|
||||||
Task task,
|
Task task,
|
||||||
ListenableFuture<TaskStatus> result,
|
ListenableFuture<TaskStatus> result,
|
||||||
RetryPolicy retryPolicy,
|
DateTime createdTime,
|
||||||
DateTime createdTime
|
DateTime queueInsertionTime
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.task = task;
|
this.task = task;
|
||||||
this.result = result;
|
this.result = result;
|
||||||
this.retryPolicy = retryPolicy;
|
|
||||||
this.createdTime = createdTime;
|
this.createdTime = createdTime;
|
||||||
|
this.queueInsertionTime = queueInsertionTime;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
|
@ -64,11 +70,6 @@ public class TaskRunnerWorkItem implements Comparable<TaskRunnerWorkItem>
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
public RetryPolicy getRetryPolicy()
|
|
||||||
{
|
|
||||||
return retryPolicy;
|
|
||||||
}
|
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
public DateTime getCreatedTime()
|
public DateTime getCreatedTime()
|
||||||
{
|
{
|
||||||
|
@ -83,8 +84,7 @@ public class TaskRunnerWorkItem implements Comparable<TaskRunnerWorkItem>
|
||||||
|
|
||||||
public TaskRunnerWorkItem withQueueInsertionTime(DateTime time)
|
public TaskRunnerWorkItem withQueueInsertionTime(DateTime time)
|
||||||
{
|
{
|
||||||
this.queueInsertionTime = time;
|
return new TaskRunnerWorkItem(task, result, createdTime, time);
|
||||||
return this;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -102,7 +102,6 @@ public class TaskRunnerWorkItem implements Comparable<TaskRunnerWorkItem>
|
||||||
return "TaskRunnerWorkItem{" +
|
return "TaskRunnerWorkItem{" +
|
||||||
"task=" + task +
|
"task=" + task +
|
||||||
", result=" + result +
|
", result=" + result +
|
||||||
", retryPolicy=" + retryPolicy +
|
|
||||||
", createdTime=" + createdTime +
|
", createdTime=" + createdTime +
|
||||||
'}';
|
'}';
|
||||||
}
|
}
|
||||||
|
|
|
@ -77,13 +77,19 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
|
||||||
exec.shutdownNow();
|
exec.shutdownNow();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void bootstrap(List<Task> tasks)
|
||||||
|
{
|
||||||
|
// do nothing
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ListenableFuture<TaskStatus> run(final Task task)
|
public ListenableFuture<TaskStatus> run(final Task task)
|
||||||
{
|
{
|
||||||
final TaskToolbox toolbox = toolboxFactory.build(task);
|
final TaskToolbox toolbox = toolboxFactory.build(task);
|
||||||
final ListenableFuture<TaskStatus> statusFuture = exec.submit(new ExecutorServiceTaskRunnerCallable(task, toolbox));
|
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);
|
runningItems.add(taskRunnerWorkItem);
|
||||||
Futures.addCallback(
|
Futures.addCallback(
|
||||||
statusFuture, new FutureCallback<TaskStatus>()
|
statusFuture, new FutureCallback<TaskStatus>()
|
||||||
|
@ -110,7 +116,7 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
|
||||||
{
|
{
|
||||||
for (final TaskRunnerWorkItem runningItem : runningItems) {
|
for (final TaskRunnerWorkItem runningItem : runningItems) {
|
||||||
if (runningItem.getTask().getId().equals(taskid)) {
|
if (runningItem.getTask().getId().equals(taskid)) {
|
||||||
runningItem.getTask().shutdown();
|
runningItem.getResult().cancel(true);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -184,14 +190,10 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
|
||||||
private final Task task;
|
private final Task task;
|
||||||
private final TaskToolbox toolbox;
|
private final TaskToolbox toolbox;
|
||||||
|
|
||||||
private final DateTime createdTime;
|
|
||||||
|
|
||||||
public ExecutorServiceTaskRunnerCallable(Task task, TaskToolbox toolbox)
|
public ExecutorServiceTaskRunnerCallable(Task task, TaskToolbox toolbox)
|
||||||
{
|
{
|
||||||
this.task = task;
|
this.task = task;
|
||||||
this.toolbox = toolbox;
|
this.toolbox = toolbox;
|
||||||
|
|
||||||
this.createdTime = new DateTime();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -243,12 +245,7 @@ public class ThreadPoolTaskRunner implements TaskRunner, QuerySegmentWalker
|
||||||
|
|
||||||
public TaskRunnerWorkItem getTaskRunnerWorkItem()
|
public TaskRunnerWorkItem getTaskRunnerWorkItem()
|
||||||
{
|
{
|
||||||
return new TaskRunnerWorkItem(
|
return new TaskRunnerWorkItem(task, null);
|
||||||
task,
|
|
||||||
null,
|
|
||||||
null,
|
|
||||||
createdTime
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,17 +24,20 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
|
import com.google.common.collect.Maps;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
import com.metamx.druid.indexing.common.TaskStatus;
|
import com.metamx.druid.indexing.common.TaskStatus;
|
||||||
|
import com.metamx.druid.indexing.common.task.Task;
|
||||||
import com.metamx.druid.indexing.worker.Worker;
|
import com.metamx.druid.indexing.worker.Worker;
|
||||||
import org.apache.curator.framework.recipes.cache.ChildData;
|
import org.apache.curator.framework.recipes.cache.ChildData;
|
||||||
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
|
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
|
||||||
|
import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -44,7 +47,7 @@ public class ZkWorker implements Closeable
|
||||||
{
|
{
|
||||||
private final Worker worker;
|
private final Worker worker;
|
||||||
private final PathChildrenCache statusCache;
|
private final PathChildrenCache statusCache;
|
||||||
private final Function<ChildData, String> cacheConverter;
|
private final Function<ChildData, TaskStatus> cacheConverter;
|
||||||
|
|
||||||
private volatile DateTime lastCompletedTaskTime = new DateTime();
|
private volatile DateTime lastCompletedTaskTime = new DateTime();
|
||||||
|
|
||||||
|
@ -52,13 +55,13 @@ public class ZkWorker implements Closeable
|
||||||
{
|
{
|
||||||
this.worker = worker;
|
this.worker = worker;
|
||||||
this.statusCache = statusCache;
|
this.statusCache = statusCache;
|
||||||
this.cacheConverter = new Function<ChildData, String>()
|
this.cacheConverter = new Function<ChildData, TaskStatus>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public String apply(@Nullable ChildData input)
|
public TaskStatus apply(ChildData input)
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
return jsonMapper.readValue(input.getData(), TaskStatus.class).getId();
|
return jsonMapper.readValue(input.getData(), TaskStatus.class);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(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()
|
public Worker getWorker()
|
||||||
{
|
{
|
||||||
return worker;
|
return worker;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty("runningTasks")
|
||||||
public Set<String> getRunningTasks()
|
public Collection<String> getRunningTaskIds()
|
||||||
{
|
{
|
||||||
return Sets.newHashSet(
|
return getRunningTasks().keySet();
|
||||||
Lists.transform(
|
}
|
||||||
|
|
||||||
|
public Map<String, TaskStatus> getRunningTasks()
|
||||||
|
{
|
||||||
|
Map<String, TaskStatus> retVal = Maps.newHashMap();
|
||||||
|
for (TaskStatus taskStatus : Lists.transform(
|
||||||
statusCache.getCurrentData(),
|
statusCache.getCurrentData(),
|
||||||
cacheConverter
|
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
|
@JsonProperty
|
||||||
|
@ -90,10 +131,25 @@ public class ZkWorker implements Closeable
|
||||||
return lastCompletedTaskTime;
|
return lastCompletedTaskTime;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
public boolean isRunningTask(String taskId)
|
||||||
|
{
|
||||||
|
return getRunningTasks().containsKey(taskId);
|
||||||
|
}
|
||||||
|
|
||||||
public boolean isAtCapacity()
|
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)
|
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.Config;
|
||||||
import org.skife.config.Default;
|
import org.skife.config.Default;
|
||||||
|
import org.skife.config.DefaultNull;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -29,4 +30,8 @@ public abstract class EC2AutoScalingStrategyConfig
|
||||||
@Config("druid.indexer.worker.port")
|
@Config("druid.indexer.worker.port")
|
||||||
@Default("8080")
|
@Default("8080")
|
||||||
public abstract String getWorkerPort();
|
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
|
public abstract class ForkingTaskRunnerConfig
|
||||||
{
|
{
|
||||||
@Config("druid.indexer.taskDir")
|
@Config("druid.indexer.taskDir")
|
||||||
|
@Default("/tmp/persistent")
|
||||||
public abstract File getBaseTaskDir();
|
public abstract File getBaseTaskDir();
|
||||||
|
|
||||||
@Config("druid.indexer.fork.java")
|
@Config("druid.indexer.fork.java")
|
||||||
|
|
|
@ -41,7 +41,7 @@ public abstract class IndexerCoordinatorConfig extends ZkPathsConfig
|
||||||
public abstract int getNumLocalThreads();
|
public abstract int getNumLocalThreads();
|
||||||
|
|
||||||
@Config("druid.indexer.runner")
|
@Config("druid.indexer.runner")
|
||||||
@Default("remote")
|
@Default("local")
|
||||||
public abstract String getRunnerImpl();
|
public abstract String getRunnerImpl();
|
||||||
|
|
||||||
@Config("druid.indexer.storage")
|
@Config("druid.indexer.storage")
|
||||||
|
|
|
@ -23,6 +23,7 @@ import com.metamx.druid.indexing.common.config.IndexerZkConfig;
|
||||||
import org.joda.time.Duration;
|
import org.joda.time.Duration;
|
||||||
import org.skife.config.Config;
|
import org.skife.config.Config;
|
||||||
import org.skife.config.Default;
|
import org.skife.config.Default;
|
||||||
|
import org.skife.config.DefaultNull;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -31,4 +32,12 @@ public abstract class RemoteTaskRunnerConfig extends IndexerZkConfig
|
||||||
@Config("druid.indexer.taskAssignmentTimeoutDuration")
|
@Config("druid.indexer.taskAssignmentTimeoutDuration")
|
||||||
@Default("PT5M")
|
@Default("PT5M")
|
||||||
public abstract Duration getTaskAssignmentTimeoutDuration();
|
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.Charsets;
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
import com.google.common.collect.ImmutableMap;
|
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.io.InputSupplier;
|
import com.google.common.io.InputSupplier;
|
||||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||||
import com.google.inject.Guice;
|
import com.google.inject.Guice;
|
||||||
import com.google.inject.Injector;
|
import com.google.inject.Injector;
|
||||||
import com.google.inject.servlet.GuiceFilter;
|
import com.google.inject.servlet.GuiceFilter;
|
||||||
|
import com.metamx.common.IAE;
|
||||||
import com.metamx.common.ISE;
|
import com.metamx.common.ISE;
|
||||||
import com.metamx.common.concurrent.ScheduledExecutorFactory;
|
import com.metamx.common.concurrent.ScheduledExecutorFactory;
|
||||||
import com.metamx.common.concurrent.ScheduledExecutors;
|
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.ConfigManager;
|
||||||
import com.metamx.druid.config.ConfigManagerConfig;
|
import com.metamx.druid.config.ConfigManagerConfig;
|
||||||
import com.metamx.druid.config.JacksonConfigManager;
|
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.CuratorServiceAnnouncer;
|
||||||
import com.metamx.druid.curator.discovery.ServiceAnnouncer;
|
import com.metamx.druid.curator.discovery.ServiceAnnouncer;
|
||||||
import com.metamx.druid.curator.discovery.ServiceInstanceFactory;
|
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.RedirectFilter;
|
||||||
import com.metamx.druid.http.RedirectInfo;
|
import com.metamx.druid.http.RedirectInfo;
|
||||||
import com.metamx.druid.http.StatusServlet;
|
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.LocalTaskActionClientFactory;
|
||||||
import com.metamx.druid.indexing.common.actions.TaskActionClientFactory;
|
import com.metamx.druid.indexing.common.actions.TaskActionClientFactory;
|
||||||
import com.metamx.druid.indexing.common.actions.TaskActionToolbox;
|
import com.metamx.druid.indexing.common.actions.TaskActionToolbox;
|
||||||
import com.metamx.druid.indexing.common.config.IndexerZkConfig;
|
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.config.TaskLogConfig;
|
||||||
import com.metamx.druid.indexing.common.index.EventReceiverFirehoseFactory;
|
import com.metamx.druid.indexing.common.index.EventReceiverFirehoseFactory;
|
||||||
import com.metamx.druid.indexing.common.index.StaticS3FirehoseFactory;
|
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.MonitorSchedulerConfig;
|
||||||
import com.metamx.metrics.SysMonitor;
|
import com.metamx.metrics.SysMonitor;
|
||||||
import org.apache.curator.framework.CuratorFramework;
|
import org.apache.curator.framework.CuratorFramework;
|
||||||
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
|
|
||||||
import org.apache.curator.x.discovery.ServiceDiscovery;
|
import org.apache.curator.x.discovery.ServiceDiscovery;
|
||||||
import org.jets3t.service.S3ServiceException;
|
import org.jets3t.service.S3ServiceException;
|
||||||
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
|
import org.jets3t.service.impl.rest.httpclient.RestS3Service;
|
||||||
|
@ -386,15 +384,17 @@ public class IndexerCoordinatorNode extends QueryableNode<IndexerCoordinatorNode
|
||||||
{
|
{
|
||||||
if (persistentTaskLogs == null) {
|
if (persistentTaskLogs == null) {
|
||||||
final TaskLogConfig taskLogConfig = getConfigFactory().build(TaskLogConfig.class);
|
final TaskLogConfig taskLogConfig = getConfigFactory().build(TaskLogConfig.class);
|
||||||
if (taskLogConfig.getLogStorageBucket() != null) {
|
if (taskLogConfig.getLogType().equalsIgnoreCase("s3")) {
|
||||||
initializeS3Service();
|
initializeS3Service();
|
||||||
persistentTaskLogs = new S3TaskLogs(
|
persistentTaskLogs = new S3TaskLogs(
|
||||||
taskLogConfig.getLogStorageBucket(),
|
taskLogConfig.getLogStorageBucket(),
|
||||||
taskLogConfig.getLogStoragePrefix(),
|
taskLogConfig.getLogStoragePrefix(),
|
||||||
s3Service
|
s3Service
|
||||||
);
|
);
|
||||||
} else {
|
} else if (taskLogConfig.getLogType().equalsIgnoreCase("noop")) {
|
||||||
persistentTaskLogs = new NoopTaskLogs();
|
persistentTaskLogs = new NoopTaskLogs();
|
||||||
|
} else {
|
||||||
|
throw new IAE("Unknown log type %s", taskLogConfig.getLogType());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -634,29 +634,14 @@ public class IndexerCoordinatorNode extends QueryableNode<IndexerCoordinatorNode
|
||||||
@Override
|
@Override
|
||||||
public TaskRunner build()
|
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 CuratorFramework curator = getCuratorFramework();
|
||||||
|
final RemoteTaskRunnerConfig remoteTaskRunnerConfig = getConfigFactory().build(RemoteTaskRunnerConfig.class);
|
||||||
return new RemoteTaskRunner(
|
return new RemoteTaskRunner(
|
||||||
getJsonMapper(),
|
getJsonMapper(),
|
||||||
getConfigFactory().build(RemoteTaskRunnerConfig.class),
|
remoteTaskRunnerConfig,
|
||||||
curator,
|
curator,
|
||||||
new PathChildrenCache(curator, indexerZkConfig.getIndexerAnnouncementPath(), true),
|
new SimplePathChildrenCacheFactory.Builder().withCompressed(remoteTaskRunnerConfig.enableCompression())
|
||||||
retryScheduledExec,
|
.build(),
|
||||||
new RetryPolicyFactory(
|
|
||||||
getConfigFactory().buildWithReplacements(
|
|
||||||
RetryPolicyConfig.class,
|
|
||||||
ImmutableMap.of("base_path", "druid.indexing")
|
|
||||||
)
|
|
||||||
),
|
|
||||||
configManager.watch(WorkerSetupData.CONFIG_KEY, WorkerSetupData.class),
|
configManager.watch(WorkerSetupData.CONFIG_KEY, WorkerSetupData.class),
|
||||||
httpClient
|
httpClient
|
||||||
);
|
);
|
||||||
|
@ -692,7 +677,7 @@ public class IndexerCoordinatorNode extends QueryableNode<IndexerCoordinatorNode
|
||||||
resourceManagementSchedulerFactory = new ResourceManagementSchedulerFactory()
|
resourceManagementSchedulerFactory = new ResourceManagementSchedulerFactory()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public ResourceManagementScheduler build(TaskRunner runner)
|
public ResourceManagementScheduler build(RemoteTaskRunner runner)
|
||||||
{
|
{
|
||||||
return new NoopResourceManagementScheduler();
|
return new NoopResourceManagementScheduler();
|
||||||
}
|
}
|
||||||
|
@ -701,7 +686,7 @@ public class IndexerCoordinatorNode extends QueryableNode<IndexerCoordinatorNode
|
||||||
resourceManagementSchedulerFactory = new ResourceManagementSchedulerFactory()
|
resourceManagementSchedulerFactory = new ResourceManagementSchedulerFactory()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public ResourceManagementScheduler build(TaskRunner runner)
|
public ResourceManagementScheduler build(RemoteTaskRunner runner)
|
||||||
{
|
{
|
||||||
final ScheduledExecutorService scalingScheduledExec = Executors.newScheduledThreadPool(
|
final ScheduledExecutorService scalingScheduledExec = Executors.newScheduledThreadPool(
|
||||||
1,
|
1,
|
||||||
|
|
|
@ -267,6 +267,7 @@ public class IndexerCoordinatorResource
|
||||||
retMap.put("result", ret);
|
retMap.put("result", ret);
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
|
log.warn(e, "Failed to perform task action");
|
||||||
return Response.serverError().build();
|
return Response.serverError().build();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -33,6 +33,7 @@ import com.google.common.base.Function;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.metamx.druid.indexing.coordinator.config.EC2AutoScalingStrategyConfig;
|
import com.metamx.druid.indexing.coordinator.config.EC2AutoScalingStrategyConfig;
|
||||||
import com.metamx.druid.indexing.coordinator.setup.EC2NodeData;
|
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.druid.indexing.coordinator.setup.WorkerSetupData;
|
||||||
import com.metamx.emitter.EmittingLogger;
|
import com.metamx.emitter.EmittingLogger;
|
||||||
import org.apache.commons.codec.binary.Base64;
|
import org.apache.commons.codec.binary.Base64;
|
||||||
|
@ -72,6 +73,11 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy<Instance>
|
||||||
WorkerSetupData setupData = workerSetupDataRef.get();
|
WorkerSetupData setupData = workerSetupDataRef.get();
|
||||||
EC2NodeData workerConfig = setupData.getNodeData();
|
EC2NodeData workerConfig = setupData.getNodeData();
|
||||||
|
|
||||||
|
GalaxyUserData userData = setupData.getUserData();
|
||||||
|
if (config.getWorkerVersion() != null) {
|
||||||
|
userData = userData.withVersion(config.getWorkerVersion());
|
||||||
|
}
|
||||||
|
|
||||||
RunInstancesResult result = amazonEC2Client.runInstances(
|
RunInstancesResult result = amazonEC2Client.runInstances(
|
||||||
new RunInstancesRequest(
|
new RunInstancesRequest(
|
||||||
workerConfig.getAmiId(),
|
workerConfig.getAmiId(),
|
||||||
|
@ -84,7 +90,7 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy<Instance>
|
||||||
.withUserData(
|
.withUserData(
|
||||||
Base64.encodeBase64String(
|
Base64.encodeBase64String(
|
||||||
jsonMapper.writeValueAsBytes(
|
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;
|
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;
|
return retVal;
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,6 +24,7 @@ import com.metamx.common.lifecycle.LifecycleStart;
|
||||||
import com.metamx.common.lifecycle.LifecycleStop;
|
import com.metamx.common.lifecycle.LifecycleStop;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
import com.metamx.druid.PeriodGranularity;
|
import com.metamx.druid.PeriodGranularity;
|
||||||
|
import com.metamx.druid.indexing.coordinator.RemoteTaskRunner;
|
||||||
import com.metamx.druid.indexing.coordinator.TaskRunner;
|
import com.metamx.druid.indexing.coordinator.TaskRunner;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Duration;
|
import org.joda.time.Duration;
|
||||||
|
@ -42,7 +43,7 @@ public class ResourceManagementScheduler
|
||||||
{
|
{
|
||||||
private static final Logger log = new Logger(ResourceManagementScheduler.class);
|
private static final Logger log = new Logger(ResourceManagementScheduler.class);
|
||||||
|
|
||||||
private final TaskRunner taskRunner;
|
private final RemoteTaskRunner taskRunner;
|
||||||
private final ResourceManagementStrategy resourceManagementStrategy;
|
private final ResourceManagementStrategy resourceManagementStrategy;
|
||||||
private final ResourceManagementSchedulerConfig config;
|
private final ResourceManagementSchedulerConfig config;
|
||||||
private final ScheduledExecutorService exec;
|
private final ScheduledExecutorService exec;
|
||||||
|
@ -51,7 +52,7 @@ public class ResourceManagementScheduler
|
||||||
private volatile boolean started = false;
|
private volatile boolean started = false;
|
||||||
|
|
||||||
public ResourceManagementScheduler(
|
public ResourceManagementScheduler(
|
||||||
TaskRunner taskRunner,
|
RemoteTaskRunner taskRunner,
|
||||||
ResourceManagementStrategy resourceManagementStrategy,
|
ResourceManagementStrategy resourceManagementStrategy,
|
||||||
ResourceManagementSchedulerConfig config,
|
ResourceManagementSchedulerConfig config,
|
||||||
ScheduledExecutorService exec
|
ScheduledExecutorService exec
|
||||||
|
|
|
@ -19,11 +19,11 @@
|
||||||
|
|
||||||
package com.metamx.druid.indexing.coordinator.scaling;
|
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 interface ResourceManagementSchedulerFactory
|
||||||
{
|
{
|
||||||
public ResourceManagementScheduler build(TaskRunner runner);
|
public ResourceManagementScheduler build(RemoteTaskRunner runner);
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,7 +19,7 @@
|
||||||
|
|
||||||
package com.metamx.druid.indexing.coordinator.scaling;
|
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 com.metamx.druid.indexing.coordinator.ZkWorker;
|
||||||
|
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
@ -30,9 +30,9 @@ import java.util.Collection;
|
||||||
*/
|
*/
|
||||||
public interface ResourceManagementStrategy
|
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();
|
public ScalingStats getStats();
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,6 +25,7 @@ import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
import com.metamx.common.guava.FunctionalIterable;
|
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.TaskRunnerWorkItem;
|
||||||
import com.metamx.druid.indexing.coordinator.ZkWorker;
|
import com.metamx.druid.indexing.coordinator.ZkWorker;
|
||||||
import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData;
|
import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData;
|
||||||
|
@ -68,10 +69,24 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@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()) {
|
final WorkerSetupData workerSetupData = workerSetupdDataRef.get();
|
||||||
log.info(
|
|
||||||
|
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",
|
"Cannot scale anymore. Num workers = %d, Max num workers = %d",
|
||||||
zkWorkers.size(),
|
zkWorkers.size(),
|
||||||
workerSetupdDataRef.get().getMaxNumWorkers()
|
workerSetupdDataRef.get().getMaxNumWorkers()
|
||||||
|
@ -135,7 +150,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean doTerminate(Collection<TaskRunnerWorkItem> pendingTasks, Collection<ZkWorker> zkWorkers)
|
public boolean doTerminate(Collection<RemoteTaskRunnerWorkItem> pendingTasks, Collection<ZkWorker> zkWorkers)
|
||||||
{
|
{
|
||||||
Set<String> workerNodeIds = Sets.newHashSet(
|
Set<String> workerNodeIds = Sets.newHashSet(
|
||||||
autoScalingStrategy.ipToIdLookup(
|
autoScalingStrategy.ipToIdLookup(
|
||||||
|
@ -244,7 +259,7 @@ public class SimpleResourceManagementStrategy implements ResourceManagementStrat
|
||||||
return scalingStats;
|
return scalingStats;
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean hasTaskPendingBeyondThreshold(Collection<TaskRunnerWorkItem> pendingTasks)
|
private boolean hasTaskPendingBeyondThreshold(Collection<RemoteTaskRunnerWorkItem> pendingTasks)
|
||||||
{
|
{
|
||||||
long now = System.currentTimeMillis();
|
long now = System.currentTimeMillis();
|
||||||
for (TaskRunnerWorkItem pendingTask : pendingTasks) {
|
for (TaskRunnerWorkItem pendingTask : pendingTasks) {
|
||||||
|
|
|
@ -22,6 +22,7 @@ package com.metamx.druid.indexing.coordinator.scaling;
|
||||||
import org.joda.time.Duration;
|
import org.joda.time.Duration;
|
||||||
import org.skife.config.Config;
|
import org.skife.config.Config;
|
||||||
import org.skife.config.Default;
|
import org.skife.config.Default;
|
||||||
|
import org.skife.config.DefaultNull;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -42,4 +43,8 @@ public abstract class SimpleResourceManagmentConfig
|
||||||
@Config("druid.indexer.maxPendingTaskDuration")
|
@Config("druid.indexer.maxPendingTaskDuration")
|
||||||
@Default("PT30S")
|
@Default("PT30S")
|
||||||
public abstract Duration getMaxPendingTaskDuration();
|
public abstract Duration getMaxPendingTaskDuration();
|
||||||
|
|
||||||
|
@Config("druid.indexer.worker.version")
|
||||||
|
@DefaultNull
|
||||||
|
public abstract String getWorkerVersion();
|
||||||
}
|
}
|
||||||
|
|
|
@ -60,6 +60,11 @@ public class GalaxyUserData
|
||||||
return type;
|
return type;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public GalaxyUserData withVersion(String ver)
|
||||||
|
{
|
||||||
|
return new GalaxyUserData(env, ver, type);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString()
|
public String toString()
|
||||||
{
|
{
|
||||||
|
|
|
@ -130,6 +130,7 @@ public class WorkerCuratorCoordinator
|
||||||
}
|
}
|
||||||
|
|
||||||
curatorFramework.create()
|
curatorFramework.create()
|
||||||
|
.creatingParentsIfNeeded()
|
||||||
.withMode(mode)
|
.withMode(mode)
|
||||||
.forPath(path, rawBytes);
|
.forPath(path, rawBytes);
|
||||||
}
|
}
|
||||||
|
|
|
@ -29,10 +29,6 @@ public abstract class WorkerConfig
|
||||||
@Config("druid.host")
|
@Config("druid.host")
|
||||||
public abstract String getHost();
|
public abstract String getHost();
|
||||||
|
|
||||||
@Config("druid.worker.threads")
|
|
||||||
@Default("1")
|
|
||||||
public abstract int getNumThreads();
|
|
||||||
|
|
||||||
@Config("druid.worker.ip")
|
@Config("druid.worker.ip")
|
||||||
public abstract String getIp();
|
public abstract String getIp();
|
||||||
|
|
||||||
|
|
|
@ -1,6 +1,5 @@
|
||||||
package com.metamx.druid.indexing.worker.executor;
|
package com.metamx.druid.indexing.worker.executor;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import com.metamx.druid.indexing.common.index.ChatHandler;
|
import com.metamx.druid.indexing.common.index.ChatHandler;
|
||||||
|
@ -13,13 +12,11 @@ import javax.ws.rs.core.Response;
|
||||||
@Path("/druid/worker/v1")
|
@Path("/druid/worker/v1")
|
||||||
public class ChatHandlerResource
|
public class ChatHandlerResource
|
||||||
{
|
{
|
||||||
private final ObjectMapper jsonMapper;
|
|
||||||
private final ChatHandlerProvider handlers;
|
private final ChatHandlerProvider handlers;
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public ChatHandlerResource(ObjectMapper jsonMapper, ChatHandlerProvider handlers)
|
public ChatHandlerResource(ChatHandlerProvider handlers)
|
||||||
{
|
{
|
||||||
this.jsonMapper = jsonMapper;
|
|
||||||
this.handlers = handlers;
|
this.handlers = handlers;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -78,7 +78,6 @@ public class ExecutorLifecycle
|
||||||
}
|
}
|
||||||
|
|
||||||
// Spawn monitor thread to keep a watch on parent's stdin
|
// Spawn monitor thread to keep a watch on parent's stdin
|
||||||
// If a message comes over stdin, we want to handle it
|
|
||||||
// If stdin reaches eof, the parent is gone, and we should shut down
|
// If stdin reaches eof, the parent is gone, and we should shut down
|
||||||
parentMonitorExec.submit(
|
parentMonitorExec.submit(
|
||||||
new Runnable()
|
new Runnable()
|
||||||
|
@ -87,25 +86,8 @@ public class ExecutorLifecycle
|
||||||
public void run()
|
public void run()
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
final BufferedReader parentReader = new BufferedReader(new InputStreamReader(parentStream));
|
while (parentStream.read() != -1) {
|
||||||
String messageString;
|
// Toss the byte
|
||||||
while ((messageString = parentReader.readLine()) != null) {
|
|
||||||
final Map<String, Object> message = jsonMapper
|
|
||||||
.readValue(
|
|
||||||
messageString,
|
|
||||||
new TypeReference<Map<String, Object>>()
|
|
||||||
{
|
|
||||||
}
|
|
||||||
);
|
|
||||||
|
|
||||||
if (message == null) {
|
|
||||||
break;
|
|
||||||
} else if (message.get("shutdown") != null && message.get("shutdown").equals("now")) {
|
|
||||||
log.info("Shutting down!");
|
|
||||||
task.shutdown();
|
|
||||||
} else {
|
|
||||||
throw new ISE("Unrecognized message from parent: %s", message);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
|
|
|
@ -22,8 +22,8 @@ package com.metamx.druid.indexing.worker.executor;
|
||||||
import com.fasterxml.jackson.databind.InjectableValues;
|
import com.fasterxml.jackson.databind.InjectableValues;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
|
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.Lists;
|
|
||||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||||
import com.google.inject.Guice;
|
import com.google.inject.Guice;
|
||||||
import com.google.inject.Injector;
|
import com.google.inject.Injector;
|
||||||
|
@ -37,12 +37,24 @@ import com.metamx.common.lifecycle.LifecycleStart;
|
||||||
import com.metamx.common.lifecycle.LifecycleStop;
|
import com.metamx.common.lifecycle.LifecycleStop;
|
||||||
import com.metamx.druid.BaseServerNode;
|
import com.metamx.druid.BaseServerNode;
|
||||||
import com.metamx.druid.curator.discovery.CuratorServiceAnnouncer;
|
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.ServiceAnnouncer;
|
||||||
import com.metamx.druid.curator.discovery.ServiceInstanceFactory;
|
import com.metamx.druid.curator.discovery.ServiceInstanceFactory;
|
||||||
import com.metamx.druid.http.GuiceServletConfig;
|
import com.metamx.druid.http.GuiceServletConfig;
|
||||||
import com.metamx.druid.http.QueryServlet;
|
import com.metamx.druid.http.QueryServlet;
|
||||||
import com.metamx.druid.http.StatusServlet;
|
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.Initialization;
|
||||||
import com.metamx.druid.initialization.ServerConfig;
|
import com.metamx.druid.initialization.ServerConfig;
|
||||||
import com.metamx.druid.initialization.ServerInit;
|
import com.metamx.druid.initialization.ServerInit;
|
||||||
|
@ -51,17 +63,6 @@ import com.metamx.druid.jackson.DefaultObjectMapper;
|
||||||
import com.metamx.druid.loading.DataSegmentKiller;
|
import com.metamx.druid.loading.DataSegmentKiller;
|
||||||
import com.metamx.druid.loading.DataSegmentPusher;
|
import com.metamx.druid.loading.DataSegmentPusher;
|
||||||
import com.metamx.druid.loading.S3DataSegmentKiller;
|
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.druid.utils.PropUtils;
|
||||||
import com.metamx.emitter.EmittingLogger;
|
import com.metamx.emitter.EmittingLogger;
|
||||||
import com.metamx.emitter.core.Emitters;
|
import com.metamx.emitter.core.Emitters;
|
||||||
|
@ -69,11 +70,10 @@ import com.metamx.emitter.service.ServiceEmitter;
|
||||||
import com.metamx.http.client.HttpClient;
|
import com.metamx.http.client.HttpClient;
|
||||||
import com.metamx.http.client.HttpClientConfig;
|
import com.metamx.http.client.HttpClientConfig;
|
||||||
import com.metamx.http.client.HttpClientInit;
|
import com.metamx.http.client.HttpClientInit;
|
||||||
import com.metamx.metrics.JvmMonitor;
|
|
||||||
import com.metamx.metrics.Monitor;
|
import com.metamx.metrics.Monitor;
|
||||||
import com.metamx.metrics.MonitorScheduler;
|
import com.metamx.metrics.MonitorScheduler;
|
||||||
import com.metamx.metrics.MonitorSchedulerConfig;
|
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.ServiceDiscovery;
|
||||||
import org.apache.curator.x.discovery.ServiceProvider;
|
import org.apache.curator.x.discovery.ServiceProvider;
|
||||||
import org.jets3t.service.S3ServiceException;
|
import org.jets3t.service.S3ServiceException;
|
||||||
|
@ -85,7 +85,6 @@ import org.mortbay.jetty.servlet.DefaultServlet;
|
||||||
import org.mortbay.jetty.servlet.ServletHolder;
|
import org.mortbay.jetty.servlet.ServletHolder;
|
||||||
import org.skife.config.ConfigurationObjectFactory;
|
import org.skife.config.ConfigurationObjectFactory;
|
||||||
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Properties;
|
import java.util.Properties;
|
||||||
import java.util.concurrent.Executors;
|
import java.util.concurrent.Executors;
|
||||||
import java.util.concurrent.ScheduledExecutorService;
|
import java.util.concurrent.ScheduledExecutorService;
|
||||||
|
@ -107,7 +106,7 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
|
||||||
private final ExecutorLifecycleFactory executorLifecycleFactory;
|
private final ExecutorLifecycleFactory executorLifecycleFactory;
|
||||||
|
|
||||||
private RestS3Service s3Service = null;
|
private RestS3Service s3Service = null;
|
||||||
private List<Monitor> monitors = null;
|
private MonitorScheduler monitorScheduler = null;
|
||||||
private HttpClient httpClient = null;
|
private HttpClient httpClient = null;
|
||||||
private ServiceEmitter emitter = null;
|
private ServiceEmitter emitter = null;
|
||||||
private TaskConfig taskConfig = null;
|
private TaskConfig taskConfig = null;
|
||||||
|
@ -140,58 +139,16 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
|
||||||
this.executorLifecycleFactory = executorLifecycleFactory;
|
this.executorLifecycleFactory = executorLifecycleFactory;
|
||||||
}
|
}
|
||||||
|
|
||||||
public ExecutorNode setHttpClient(HttpClient httpClient)
|
|
||||||
{
|
|
||||||
this.httpClient = httpClient;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public ExecutorNode setEmitter(ServiceEmitter emitter)
|
|
||||||
{
|
|
||||||
this.emitter = emitter;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public ExecutorNode setS3Service(RestS3Service s3Service)
|
|
||||||
{
|
|
||||||
this.s3Service = s3Service;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public ExecutorNode setSegmentPusher(DataSegmentPusher segmentPusher)
|
|
||||||
{
|
|
||||||
this.segmentPusher = segmentPusher;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public ExecutorNode setTaskToolboxFactory(TaskToolboxFactory taskToolboxFactory)
|
|
||||||
{
|
|
||||||
this.taskToolboxFactory = taskToolboxFactory;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public ExecutorNode setCoordinatorServiceProvider(ServiceProvider coordinatorServiceProvider)
|
|
||||||
{
|
|
||||||
this.coordinatorServiceProvider = coordinatorServiceProvider;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public ExecutorNode setServiceDiscovery(ServiceDiscovery serviceDiscovery)
|
|
||||||
{
|
|
||||||
this.serviceDiscovery = serviceDiscovery;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void doInit() throws Exception
|
public void doInit() throws Exception
|
||||||
{
|
{
|
||||||
initializeHttpClient();
|
initializeHttpClient();
|
||||||
initializeEmitter();
|
initializeEmitter();
|
||||||
initializeS3Service();
|
initializeS3Service();
|
||||||
initializeMonitors();
|
|
||||||
initializeMergerConfig();
|
initializeMergerConfig();
|
||||||
initializeServiceDiscovery();
|
initializeServiceDiscovery();
|
||||||
initializeDataSegmentPusher();
|
initializeDataSegmentPusher();
|
||||||
|
initializeMonitorScheduler();
|
||||||
initializeTaskToolbox();
|
initializeTaskToolbox();
|
||||||
initializeTaskRunner();
|
initializeTaskRunner();
|
||||||
initializeChatHandlerProvider();
|
initializeChatHandlerProvider();
|
||||||
|
@ -199,13 +156,6 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
|
||||||
initializeJacksonSubtypes();
|
initializeJacksonSubtypes();
|
||||||
initializeServer();
|
initializeServer();
|
||||||
|
|
||||||
final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle);
|
|
||||||
final ScheduledExecutorService globalScheduledExec = scheduledExecutorFactory.create(1, "Global--%d");
|
|
||||||
final MonitorScheduler monitorScheduler = new MonitorScheduler(
|
|
||||||
configFactory.build(MonitorSchedulerConfig.class), globalScheduledExec, emitter, monitors
|
|
||||||
);
|
|
||||||
lifecycle.addManagedInstance(monitorScheduler);
|
|
||||||
|
|
||||||
executorLifecycle = executorLifecycleFactory.build(taskRunner, getJsonMapper());
|
executorLifecycle = executorLifecycleFactory.build(taskRunner, getJsonMapper());
|
||||||
lifecycle.addManagedInstance(executorLifecycle);
|
lifecycle.addManagedInstance(executorLifecycle);
|
||||||
|
|
||||||
|
@ -229,6 +179,19 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void initializeMonitorScheduler()
|
||||||
|
{
|
||||||
|
if (monitorScheduler == null)
|
||||||
|
{
|
||||||
|
final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle);
|
||||||
|
final ScheduledExecutorService globalScheduledExec = scheduledExecutorFactory.create(1, "Global--%d");
|
||||||
|
this.monitorScheduler = new MonitorScheduler(
|
||||||
|
configFactory.build(MonitorSchedulerConfig.class), globalScheduledExec, emitter, ImmutableList.<Monitor>of()
|
||||||
|
);
|
||||||
|
lifecycle.addManagedInstance(monitorScheduler);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@LifecycleStart
|
@LifecycleStart
|
||||||
public synchronized void start() throws Exception
|
public synchronized void start() throws Exception
|
||||||
{
|
{
|
||||||
|
@ -333,15 +296,6 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void initializeMonitors()
|
|
||||||
{
|
|
||||||
if (monitors == null) {
|
|
||||||
monitors = Lists.newArrayList();
|
|
||||||
monitors.add(new JvmMonitor());
|
|
||||||
monitors.add(new SysMonitor());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private void initializeMergerConfig()
|
private void initializeMergerConfig()
|
||||||
{
|
{
|
||||||
if (taskConfig == null) {
|
if (taskConfig == null) {
|
||||||
|
@ -384,6 +338,7 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
|
||||||
getAnnouncer(),
|
getAnnouncer(),
|
||||||
getServerView(),
|
getServerView(),
|
||||||
getConglomerate(),
|
getConglomerate(),
|
||||||
|
monitorScheduler,
|
||||||
getJsonMapper()
|
getJsonMapper()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -393,8 +348,9 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
|
||||||
{
|
{
|
||||||
final ServiceDiscoveryConfig config = configFactory.build(ServiceDiscoveryConfig.class);
|
final ServiceDiscoveryConfig config = configFactory.build(ServiceDiscoveryConfig.class);
|
||||||
if (serviceDiscovery == null) {
|
if (serviceDiscovery == null) {
|
||||||
|
final CuratorFramework serviceDiscoveryCuratorFramework = Initialization.makeCuratorFramework(config, lifecycle);
|
||||||
this.serviceDiscovery = Initialization.makeServiceDiscoveryClient(
|
this.serviceDiscovery = Initialization.makeServiceDiscoveryClient(
|
||||||
getCuratorFramework(), config, lifecycle
|
serviceDiscoveryCuratorFramework, config, lifecycle
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
if (serviceAnnouncer == null) {
|
if (serviceAnnouncer == null) {
|
||||||
|
@ -430,19 +386,17 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
|
||||||
{
|
{
|
||||||
if (chatHandlerProvider == null) {
|
if (chatHandlerProvider == null) {
|
||||||
final ChatHandlerProviderConfig config = configFactory.build(ChatHandlerProviderConfig.class);
|
final ChatHandlerProviderConfig config = configFactory.build(ChatHandlerProviderConfig.class);
|
||||||
final ServiceAnnouncer myServiceAnnouncer;
|
|
||||||
if (config.getServiceFormat() == null) {
|
if (config.getServiceFormat() == null) {
|
||||||
log.info("ChatHandlerProvider: Using NoopServiceAnnouncer. Good luck finding your firehoses!");
|
log.info("ChatHandlerProvider: Using NoopChatHandlerProvider. Good luck finding your firehoses!");
|
||||||
myServiceAnnouncer = new NoopServiceAnnouncer();
|
this.chatHandlerProvider = new NoopChatHandlerProvider();
|
||||||
} else {
|
} else {
|
||||||
myServiceAnnouncer = serviceAnnouncer;
|
this.chatHandlerProvider = new EventReceivingChatHandlerProvider(
|
||||||
}
|
|
||||||
this.chatHandlerProvider = new ChatHandlerProvider(
|
|
||||||
config,
|
config,
|
||||||
myServiceAnnouncer
|
serviceAnnouncer
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public static class Builder
|
public static class Builder
|
||||||
{
|
{
|
||||||
|
@ -482,9 +436,12 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
|
||||||
jsonMapper = new DefaultObjectMapper();
|
jsonMapper = new DefaultObjectMapper();
|
||||||
smileMapper = new DefaultObjectMapper(new SmileFactory());
|
smileMapper = new DefaultObjectMapper(new SmileFactory());
|
||||||
smileMapper.getJsonFactory().setCodec(smileMapper);
|
smileMapper.getJsonFactory().setCodec(smileMapper);
|
||||||
}
|
} else if (jsonMapper == null || smileMapper == null) {
|
||||||
else if (jsonMapper == null || smileMapper == null) {
|
throw new ISE(
|
||||||
throw new ISE("Only jsonMapper[%s] or smileMapper[%s] was set, must set neither or both.", jsonMapper, smileMapper);
|
"Only jsonMapper[%s] or smileMapper[%s] was set, must set neither or both.",
|
||||||
|
jsonMapper,
|
||||||
|
smileMapper
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (lifecycle == null) {
|
if (lifecycle == null) {
|
||||||
|
@ -499,7 +456,15 @@ public class ExecutorNode extends BaseServerNode<ExecutorNode>
|
||||||
configFactory = Config.createFactory(props);
|
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) {
|
if (serviceDiscovery == null) {
|
||||||
final ServiceDiscoveryConfig config = getConfigFactory().build(ServiceDiscoveryConfig.class);
|
final ServiceDiscoveryConfig config = getConfigFactory().build(ServiceDiscoveryConfig.class);
|
||||||
this.serviceDiscovery = Initialization.makeServiceDiscoveryClient(
|
final CuratorFramework serviceDiscoveryCuratorFramework = Initialization.makeCuratorFramework(
|
||||||
getCuratorFramework(),
|
|
||||||
config,
|
config,
|
||||||
getLifecycle()
|
getLifecycle()
|
||||||
);
|
);
|
||||||
|
this.serviceDiscovery = Initialization.makeServiceDiscoveryClient(
|
||||||
|
serviceDiscoveryCuratorFramework, config, getLifecycle()
|
||||||
|
);
|
||||||
}
|
}
|
||||||
if (coordinatorServiceProvider == null) {
|
if (coordinatorServiceProvider == null) {
|
||||||
this.coordinatorServiceProvider = Initialization.makeServiceProvider(
|
this.coordinatorServiceProvider = Initialization.makeServiceProvider(
|
||||||
|
@ -392,7 +394,7 @@ public class WorkerNode extends QueryableNode<WorkerNode>
|
||||||
public void initializeWorkerTaskMonitor()
|
public void initializeWorkerTaskMonitor()
|
||||||
{
|
{
|
||||||
if (workerTaskMonitor == null) {
|
if (workerTaskMonitor == null) {
|
||||||
final ExecutorService workerExec = Executors.newFixedThreadPool(workerConfig.getNumThreads());
|
final ExecutorService workerExec = Executors.newFixedThreadPool(workerConfig.getCapacity());
|
||||||
final CuratorFramework curatorFramework = getCuratorFramework();
|
final CuratorFramework curatorFramework = getCuratorFramework();
|
||||||
|
|
||||||
final PathChildrenCache pathChildrenCache = new PathChildrenCache(
|
final PathChildrenCache pathChildrenCache = new PathChildrenCache(
|
||||||
|
|
|
@ -207,7 +207,8 @@ public class TaskSerdeTest
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
new Period("PT10M"),
|
new Period("PT10M"),
|
||||||
IndexGranularity.HOUR
|
IndexGranularity.HOUR,
|
||||||
|
null
|
||||||
);
|
);
|
||||||
|
|
||||||
final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||||
|
|
|
@ -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.LockReleaseAction;
|
||||||
import com.metamx.druid.indexing.common.actions.SegmentInsertAction;
|
import com.metamx.druid.indexing.common.actions.SegmentInsertAction;
|
||||||
import com.metamx.druid.indexing.common.task.AbstractTask;
|
import com.metamx.druid.indexing.common.task.AbstractTask;
|
||||||
|
import com.metamx.druid.indexing.common.task.TaskResource;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
|
||||||
|
@ -41,12 +42,12 @@ public class RealtimeishTask extends AbstractTask
|
||||||
{
|
{
|
||||||
public RealtimeishTask()
|
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
|
@Override
|
||||||
|
|
|
@ -2,22 +2,26 @@ package com.metamx.druid.indexing.coordinator;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
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.Stopwatch;
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.util.concurrent.FutureCallback;
|
import com.google.common.collect.Sets;
|
||||||
import com.google.common.util.concurrent.Futures;
|
import com.google.common.io.Files;
|
||||||
|
import com.google.common.util.concurrent.ListenableFuture;
|
||||||
import com.metamx.common.ISE;
|
import com.metamx.common.ISE;
|
||||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||||
import com.metamx.druid.client.DataSegment;
|
import com.metamx.druid.client.DataSegment;
|
||||||
import com.metamx.druid.curator.PotentiallyGzippedCompressionProvider;
|
import com.metamx.druid.curator.PotentiallyGzippedCompressionProvider;
|
||||||
|
import com.metamx.druid.curator.cache.SimplePathChildrenCacheFactory;
|
||||||
import com.metamx.druid.indexing.TestTask;
|
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.TaskStatus;
|
||||||
import com.metamx.druid.indexing.common.TaskToolboxFactory;
|
import com.metamx.druid.indexing.common.TaskToolboxFactory;
|
||||||
import com.metamx.druid.indexing.common.config.IndexerZkConfig;
|
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.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.config.RemoteTaskRunnerConfig;
|
||||||
import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData;
|
import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData;
|
||||||
import com.metamx.druid.indexing.worker.Worker;
|
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.druid.jackson.DefaultObjectMapper;
|
||||||
import com.metamx.emitter.EmittingLogger;
|
import com.metamx.emitter.EmittingLogger;
|
||||||
import com.metamx.emitter.service.ServiceEmitter;
|
import com.metamx.emitter.service.ServiceEmitter;
|
||||||
import org.apache.commons.lang.mutable.MutableBoolean;
|
|
||||||
import org.apache.curator.framework.CuratorFramework;
|
import org.apache.curator.framework.CuratorFramework;
|
||||||
import org.apache.curator.framework.CuratorFrameworkFactory;
|
import org.apache.curator.framework.CuratorFrameworkFactory;
|
||||||
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
|
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
|
||||||
|
@ -42,35 +45,34 @@ import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.concurrent.Executors;
|
import java.util.concurrent.Executors;
|
||||||
import java.util.concurrent.ScheduledExecutorService;
|
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
|
||||||
import static junit.framework.Assert.fail;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public class RemoteTaskRunnerTest
|
public class RemoteTaskRunnerTest
|
||||||
{
|
{
|
||||||
private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
|
||||||
private static final String basePath = "/test/druid/indexer";
|
private static final Joiner joiner = Joiner.on("/");
|
||||||
private static final String announcementsPath = String.format("%s/announcements", basePath);
|
private static final String basePath = "/test/druid";
|
||||||
private static final String tasksPath = String.format("%s/tasks", basePath);
|
private static final String announcementsPath = String.format("%s/indexer/announcements/worker", basePath);
|
||||||
private static final String statusPath = String.format("%s/status", 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 TestingCluster testingCluster;
|
||||||
private CuratorFramework cf;
|
private CuratorFramework cf;
|
||||||
private PathChildrenCache pathChildrenCache;
|
|
||||||
private RemoteTaskRunner remoteTaskRunner;
|
private RemoteTaskRunner remoteTaskRunner;
|
||||||
|
private WorkerCuratorCoordinator workerCuratorCoordinator;
|
||||||
private WorkerTaskMonitor workerTaskMonitor;
|
private WorkerTaskMonitor workerTaskMonitor;
|
||||||
|
|
||||||
private ScheduledExecutorService scheduledExec;
|
private TestTask task;
|
||||||
|
|
||||||
private TestTask task1;
|
private Worker worker;
|
||||||
|
|
||||||
private Worker worker1;
|
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setUp() throws Exception
|
public void setUp() throws Exception
|
||||||
|
@ -84,78 +86,27 @@ public class RemoteTaskRunnerTest
|
||||||
.compressionProvider(new PotentiallyGzippedCompressionProvider(false))
|
.compressionProvider(new PotentiallyGzippedCompressionProvider(false))
|
||||||
.build();
|
.build();
|
||||||
cf.start();
|
cf.start();
|
||||||
|
cf.create().creatingParentsIfNeeded().forPath(basePath);
|
||||||
|
|
||||||
cf.create().creatingParentsIfNeeded().forPath(announcementsPath);
|
task = makeTask(TaskStatus.success("task"));
|
||||||
cf.create().forPath(tasksPath);
|
|
||||||
cf.create().forPath(String.format("%s/worker1", tasksPath));
|
|
||||||
cf.create().forPath(statusPath);
|
|
||||||
cf.create().forPath(String.format("%s/worker1", statusPath));
|
|
||||||
|
|
||||||
pathChildrenCache = new PathChildrenCache(cf, announcementsPath, true);
|
|
||||||
|
|
||||||
worker1 = new Worker(
|
|
||||||
"worker1",
|
|
||||||
"localhost",
|
|
||||||
3,
|
|
||||||
"0"
|
|
||||||
);
|
|
||||||
|
|
||||||
task1 = new TestTask(
|
|
||||||
"task1",
|
|
||||||
"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("task1")
|
|
||||||
);
|
|
||||||
|
|
||||||
makeRemoteTaskRunner();
|
|
||||||
makeTaskMonitor();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
public void tearDown() throws Exception
|
public void tearDown() throws Exception
|
||||||
{
|
{
|
||||||
testingCluster.stop();
|
|
||||||
remoteTaskRunner.stop();
|
remoteTaskRunner.stop();
|
||||||
|
workerCuratorCoordinator.stop();
|
||||||
workerTaskMonitor.stop();
|
workerTaskMonitor.stop();
|
||||||
|
cf.close();
|
||||||
|
testingCluster.stop();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testRunNoExistingTask() throws Exception
|
public void testRunNoExistingTask() throws Exception
|
||||||
{
|
{
|
||||||
remoteTaskRunner.run(task1);
|
doSetup();
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
remoteTaskRunner.run(task);
|
||||||
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
|
@Test
|
||||||
|
@ -164,107 +115,36 @@ public class RemoteTaskRunnerTest
|
||||||
ServiceEmitter emitter = EasyMock.createMock(ServiceEmitter.class);
|
ServiceEmitter emitter = EasyMock.createMock(ServiceEmitter.class);
|
||||||
EmittingLogger.registerEmitter(emitter);
|
EmittingLogger.registerEmitter(emitter);
|
||||||
EasyMock.replay(emitter);
|
EasyMock.replay(emitter);
|
||||||
remoteTaskRunner.run(
|
|
||||||
new TestTask(
|
doSetup();
|
||||||
new String(new char[5000]),
|
|
||||||
"dummyDs",
|
remoteTaskRunner.run(makeTask(TaskStatus.success(new String(new char[5000]))));
|
||||||
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);
|
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
|
@Test
|
||||||
public void testRunSameAvailabilityGroup() throws Exception
|
public void testRunSameAvailabilityGroup() throws Exception
|
||||||
{
|
{
|
||||||
TestRealtimeTask theTask = new TestRealtimeTask("rt1", "rt1", "foo", TaskStatus.running("rt1"));
|
doSetup();
|
||||||
|
|
||||||
|
TestRealtimeTask theTask = new TestRealtimeTask(
|
||||||
|
"rt1",
|
||||||
|
new TaskResource("rt1", 1),
|
||||||
|
"foo",
|
||||||
|
TaskStatus.running("rt1")
|
||||||
|
);
|
||||||
remoteTaskRunner.run(theTask);
|
remoteTaskRunner.run(theTask);
|
||||||
remoteTaskRunner.run(
|
remoteTaskRunner.run(
|
||||||
new TestRealtimeTask("rt2", "rt1", "foo", TaskStatus.running("rt2"))
|
new TestRealtimeTask("rt2", new TaskResource("rt1", 1), "foo", TaskStatus.running("rt2"))
|
||||||
);
|
);
|
||||||
remoteTaskRunner.run(
|
remoteTaskRunner.run(
|
||||||
new TestRealtimeTask("rt3", "rt2", "foo", TaskStatus.running("rt3"))
|
new TestRealtimeTask("rt3", new TaskResource("rt2", 1), "foo", TaskStatus.running("rt3"))
|
||||||
);
|
);
|
||||||
|
|
||||||
Stopwatch stopwatch = new Stopwatch();
|
Stopwatch stopwatch = new Stopwatch();
|
||||||
stopwatch.start();
|
stopwatch.start();
|
||||||
while (remoteTaskRunner.getRunningTasks().isEmpty()) {
|
while (remoteTaskRunner.getRunningTasks().size() < 2) {
|
||||||
Thread.sleep(100);
|
Thread.sleep(100);
|
||||||
if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > 1000) {
|
if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > 1000) {
|
||||||
throw new ISE("Cannot find running task");
|
throw new ISE("Cannot find running task");
|
||||||
|
@ -276,35 +156,148 @@ public class RemoteTaskRunnerTest
|
||||||
Assert.assertTrue(remoteTaskRunner.getPendingTasks().iterator().next().getTask().getId().equals("rt2"));
|
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(
|
||||||
|
"dummyDs",
|
||||||
|
new Interval(new DateTime(), new DateTime()),
|
||||||
|
new DateTime().toString(),
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
0,
|
||||||
|
0
|
||||||
|
)
|
||||||
|
),
|
||||||
|
Lists.<AggregatorFactory>newArrayList(),
|
||||||
|
status
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
private void makeTaskMonitor() throws Exception
|
private void makeTaskMonitor() throws Exception
|
||||||
{
|
{
|
||||||
WorkerCuratorCoordinator workerCuratorCoordinator = new WorkerCuratorCoordinator(
|
workerCuratorCoordinator = new WorkerCuratorCoordinator(
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
new IndexerZkConfig()
|
new IndexerZkConfig()
|
||||||
{
|
{
|
||||||
@Override
|
|
||||||
public String getIndexerAnnouncementPath()
|
|
||||||
{
|
|
||||||
return announcementsPath;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String getIndexerTaskPath()
|
|
||||||
{
|
|
||||||
return tasksPath;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String getIndexerStatusPath()
|
|
||||||
{
|
|
||||||
return statusPath;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String getZkBasePath()
|
public String getZkBasePath()
|
||||||
{
|
{
|
||||||
throw new UnsupportedOperationException();
|
return basePath;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -314,13 +307,14 @@ public class RemoteTaskRunnerTest
|
||||||
}
|
}
|
||||||
},
|
},
|
||||||
cf,
|
cf,
|
||||||
worker1
|
worker
|
||||||
);
|
);
|
||||||
workerCuratorCoordinator.start();
|
workerCuratorCoordinator.start();
|
||||||
|
|
||||||
|
// Start a task monitor
|
||||||
workerTaskMonitor = new WorkerTaskMonitor(
|
workerTaskMonitor = new WorkerTaskMonitor(
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
new PathChildrenCache(cf, String.format("%s/worker1", tasksPath), true),
|
new PathChildrenCache(cf, tasksPath, true),
|
||||||
cf,
|
cf,
|
||||||
workerCuratorCoordinator,
|
workerCuratorCoordinator,
|
||||||
new ThreadPoolTaskRunner(
|
new ThreadPoolTaskRunner(
|
||||||
|
@ -328,14 +322,11 @@ public class RemoteTaskRunnerTest
|
||||||
new TaskConfig()
|
new TaskConfig()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public File getBaseTaskDir()
|
public String getBaseDir()
|
||||||
{
|
{
|
||||||
try {
|
File tmp = Files.createTempDir();
|
||||||
return File.createTempFile("billy", "yay");
|
tmp.deleteOnExit();
|
||||||
}
|
return tmp.toString();
|
||||||
catch (Exception e) {
|
|
||||||
throw Throwables.propagate(e);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -349,7 +340,7 @@ public class RemoteTaskRunnerTest
|
||||||
{
|
{
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
}, null, null, null, null, null, null, null, null, jsonMapper
|
}, null, null, null, null, null, null, null, null, null, jsonMapper
|
||||||
), Executors.newSingleThreadExecutor()
|
), Executors.newSingleThreadExecutor()
|
||||||
),
|
),
|
||||||
Executors.newSingleThreadExecutor()
|
Executors.newSingleThreadExecutor()
|
||||||
|
@ -361,80 +352,45 @@ public class RemoteTaskRunnerTest
|
||||||
|
|
||||||
private void makeRemoteTaskRunner() throws Exception
|
private void makeRemoteTaskRunner() throws Exception
|
||||||
{
|
{
|
||||||
scheduledExec = EasyMock.createMock(ScheduledExecutorService.class);
|
|
||||||
|
|
||||||
remoteTaskRunner = new RemoteTaskRunner(
|
remoteTaskRunner = new RemoteTaskRunner(
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
new TestRemoteTaskRunnerConfig(),
|
new TestRemoteTaskRunnerConfig(),
|
||||||
cf,
|
cf,
|
||||||
pathChildrenCache,
|
new SimplePathChildrenCacheFactory.Builder().build(),
|
||||||
scheduledExec,
|
|
||||||
new RetryPolicyFactory(new TestRetryPolicyConfig()),
|
|
||||||
new AtomicReference<WorkerSetupData>(new WorkerSetupData("0", 0, 1, null, null)),
|
new AtomicReference<WorkerSetupData>(new WorkerSetupData("0", 0, 1, null, null)),
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
|
|
||||||
// Create a single worker and wait for things for be ready
|
|
||||||
remoteTaskRunner.start();
|
remoteTaskRunner.start();
|
||||||
cf.create().creatingParentsIfNeeded().withMode(CreateMode.EPHEMERAL).forPath(
|
}
|
||||||
String.format("%s/worker1", announcementsPath),
|
|
||||||
jsonMapper.writeValueAsBytes(worker1)
|
private void makeWorker() throws Exception
|
||||||
|
{
|
||||||
|
worker = new Worker(
|
||||||
|
"worker",
|
||||||
|
"localhost",
|
||||||
|
3,
|
||||||
|
"0"
|
||||||
);
|
);
|
||||||
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
|
cf.create().creatingParentsIfNeeded().withMode(CreateMode.EPHEMERAL).forPath(
|
||||||
{
|
announcementsPath,
|
||||||
@Override
|
jsonMapper.writeValueAsBytes(worker)
|
||||||
public Duration getRetryMinDuration()
|
);
|
||||||
{
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Duration getRetryMaxDuration()
|
|
||||||
{
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public long getMaxRetryCount()
|
|
||||||
{
|
|
||||||
return 0;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class TestRemoteTaskRunnerConfig extends RemoteTaskRunnerConfig
|
private static class TestRemoteTaskRunnerConfig extends RemoteTaskRunnerConfig
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public String getIndexerAnnouncementPath()
|
public boolean enableCompression()
|
||||||
{
|
{
|
||||||
return announcementsPath;
|
return false;
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String getIndexerTaskPath()
|
|
||||||
{
|
|
||||||
return tasksPath;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String getIndexerStatusPath()
|
|
||||||
{
|
|
||||||
return statusPath;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String getZkBasePath()
|
public String getZkBasePath()
|
||||||
{
|
{
|
||||||
throw new UnsupportedOperationException();
|
return basePath;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -448,5 +404,11 @@ public class RemoteTaskRunnerTest
|
||||||
{
|
{
|
||||||
return 1000;
|
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.aggregation.DoubleSumAggregatorFactory;
|
||||||
import com.metamx.druid.client.DataSegment;
|
import com.metamx.druid.client.DataSegment;
|
||||||
import com.metamx.druid.indexer.granularity.UniformGranularitySpec;
|
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.InputRow;
|
||||||
import com.metamx.druid.input.MapBasedInputRow;
|
import com.metamx.druid.input.MapBasedInputRow;
|
||||||
import com.metamx.druid.jackson.DefaultObjectMapper;
|
import com.metamx.druid.jackson.DefaultObjectMapper;
|
||||||
|
@ -120,9 +121,9 @@ public class TaskLifecycleTest
|
||||||
new TaskConfig()
|
new TaskConfig()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public File getBaseTaskDir()
|
public String getBaseDir()
|
||||||
{
|
{
|
||||||
return tmp;
|
return tmp.toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -159,6 +160,7 @@ public class TaskLifecycleTest
|
||||||
null, // segment announcer
|
null, // segment announcer
|
||||||
null, // new segment server view
|
null, // new segment server view
|
||||||
null, // query runner factory conglomerate corporation unionized collective
|
null, // query runner factory conglomerate corporation unionized collective
|
||||||
|
null, // monitor scheduler
|
||||||
new DefaultObjectMapper()
|
new DefaultObjectMapper()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -284,7 +286,7 @@ public class TaskLifecycleTest
|
||||||
@Test
|
@Test
|
||||||
public void testSimple() throws Exception
|
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
|
@Override
|
||||||
public String getType()
|
public String getType()
|
||||||
|
@ -321,7 +323,7 @@ public class TaskLifecycleTest
|
||||||
@Test
|
@Test
|
||||||
public void testBadInterval() throws Exception
|
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
|
@Override
|
||||||
public String getType()
|
public String getType()
|
||||||
|
@ -355,7 +357,7 @@ public class TaskLifecycleTest
|
||||||
@Test
|
@Test
|
||||||
public void testBadVersion() throws Exception
|
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
|
@Override
|
||||||
public String getType()
|
public String getType()
|
||||||
|
|
|
@ -168,6 +168,7 @@ public class TaskQueueTest
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
|
null,
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -228,6 +229,7 @@ public class TaskQueueTest
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
|
null,
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -346,7 +348,7 @@ public class TaskQueueTest
|
||||||
|
|
||||||
private static Task newTask(final String id, final String groupId, final String dataSource, final Interval interval)
|
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
|
@Override
|
||||||
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
public TaskStatus run(TaskToolbox toolbox) throws Exception
|
||||||
|
@ -370,7 +372,7 @@ public class TaskQueueTest
|
||||||
final List<Task> nextTasks
|
final List<Task> nextTasks
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return new AbstractTask(id, groupId, id, dataSource, interval)
|
return new AbstractTask(id, groupId, dataSource, interval)
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public String getType()
|
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.TaskStatus;
|
||||||
import com.metamx.druid.indexing.common.TaskToolbox;
|
import com.metamx.druid.indexing.common.TaskToolbox;
|
||||||
import com.metamx.druid.indexing.common.task.RealtimeIndexTask;
|
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.realtime.Schema;
|
||||||
import com.metamx.druid.shard.NoneShardSpec;
|
import com.metamx.druid.shard.NoneShardSpec;
|
||||||
|
|
||||||
|
@ -40,18 +41,19 @@ public class TestRealtimeTask extends RealtimeIndexTask
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public TestRealtimeTask(
|
public TestRealtimeTask(
|
||||||
@JsonProperty("id") String id,
|
@JsonProperty("id") String id,
|
||||||
@JsonProperty("availabilityGroup") String availGroup,
|
@JsonProperty("resource") TaskResource taskResource,
|
||||||
@JsonProperty("dataSource") String dataSource,
|
@JsonProperty("dataSource") String dataSource,
|
||||||
@JsonProperty("taskStatus") TaskStatus status
|
@JsonProperty("taskStatus") TaskStatus status
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(
|
super(
|
||||||
id,
|
id,
|
||||||
availGroup,
|
taskResource,
|
||||||
new Schema(dataSource, null, new AggregatorFactory[]{}, QueryGranularity.NONE, new NoneShardSpec()),
|
new Schema(dataSource, null, new AggregatorFactory[]{}, QueryGranularity.NONE, new NoneShardSpec()),
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
|
null,
|
||||||
null
|
null
|
||||||
);
|
);
|
||||||
this.status = status;
|
this.status = status;
|
||||||
|
|
|
@ -84,6 +84,12 @@ public class EC2AutoScalingStrategyTest
|
||||||
{
|
{
|
||||||
return "8080";
|
return "8080";
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getWorkerVersion()
|
||||||
|
{
|
||||||
|
return "";
|
||||||
|
}
|
||||||
},
|
},
|
||||||
workerSetupData
|
workerSetupData
|
||||||
);
|
);
|
||||||
|
|
|
@ -19,17 +19,20 @@
|
||||||
|
|
||||||
package com.metamx.druid.indexing.coordinator.scaling;
|
package com.metamx.druid.indexing.coordinator.scaling;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
|
import com.google.common.collect.Maps;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
import com.metamx.druid.aggregation.AggregatorFactory;
|
import com.metamx.druid.aggregation.AggregatorFactory;
|
||||||
import com.metamx.druid.client.DataSegment;
|
import com.metamx.druid.client.DataSegment;
|
||||||
import com.metamx.druid.indexing.TestTask;
|
import com.metamx.druid.indexing.TestTask;
|
||||||
import com.metamx.druid.indexing.common.TaskStatus;
|
import com.metamx.druid.indexing.common.TaskStatus;
|
||||||
import com.metamx.druid.indexing.common.task.Task;
|
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.ZkWorker;
|
||||||
import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData;
|
import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData;
|
||||||
import com.metamx.druid.indexing.worker.Worker;
|
import com.metamx.druid.indexing.worker.Worker;
|
||||||
|
import com.metamx.druid.jackson.DefaultObjectMapper;
|
||||||
import com.metamx.emitter.EmittingLogger;
|
import com.metamx.emitter.EmittingLogger;
|
||||||
import com.metamx.emitter.service.ServiceEmitter;
|
import com.metamx.emitter.service.ServiceEmitter;
|
||||||
import com.metamx.emitter.service.ServiceEventBuilder;
|
import com.metamx.emitter.service.ServiceEventBuilder;
|
||||||
|
@ -42,7 +45,9 @@ import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
import java.util.Collection;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
|
||||||
|
@ -111,6 +116,12 @@ public class SimpleResourceManagementStrategyTest
|
||||||
{
|
{
|
||||||
return new Duration(0);
|
return new Duration(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getWorkerVersion()
|
||||||
|
{
|
||||||
|
return "";
|
||||||
|
}
|
||||||
},
|
},
|
||||||
workerSetupData
|
workerSetupData
|
||||||
);
|
);
|
||||||
|
@ -127,8 +138,8 @@ public class SimpleResourceManagementStrategyTest
|
||||||
EasyMock.replay(autoScalingStrategy);
|
EasyMock.replay(autoScalingStrategy);
|
||||||
|
|
||||||
boolean provisionedSomething = simpleResourceManagementStrategy.doProvision(
|
boolean provisionedSomething = simpleResourceManagementStrategy.doProvision(
|
||||||
Arrays.<TaskRunnerWorkItem>asList(
|
Arrays.<RemoteTaskRunnerWorkItem>asList(
|
||||||
new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime())
|
new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime())
|
||||||
),
|
),
|
||||||
Arrays.<ZkWorker>asList(
|
Arrays.<ZkWorker>asList(
|
||||||
new TestZkWorker(testTask)
|
new TestZkWorker(testTask)
|
||||||
|
@ -155,8 +166,8 @@ public class SimpleResourceManagementStrategyTest
|
||||||
EasyMock.replay(autoScalingStrategy);
|
EasyMock.replay(autoScalingStrategy);
|
||||||
|
|
||||||
boolean provisionedSomething = simpleResourceManagementStrategy.doProvision(
|
boolean provisionedSomething = simpleResourceManagementStrategy.doProvision(
|
||||||
Arrays.<TaskRunnerWorkItem>asList(
|
Arrays.<RemoteTaskRunnerWorkItem>asList(
|
||||||
new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime())
|
new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime())
|
||||||
),
|
),
|
||||||
Arrays.<ZkWorker>asList(
|
Arrays.<ZkWorker>asList(
|
||||||
new TestZkWorker(testTask)
|
new TestZkWorker(testTask)
|
||||||
|
@ -171,8 +182,8 @@ public class SimpleResourceManagementStrategyTest
|
||||||
);
|
);
|
||||||
|
|
||||||
provisionedSomething = simpleResourceManagementStrategy.doProvision(
|
provisionedSomething = simpleResourceManagementStrategy.doProvision(
|
||||||
Arrays.<TaskRunnerWorkItem>asList(
|
Arrays.<RemoteTaskRunnerWorkItem>asList(
|
||||||
new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime())
|
new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime())
|
||||||
),
|
),
|
||||||
Arrays.<ZkWorker>asList(
|
Arrays.<ZkWorker>asList(
|
||||||
new TestZkWorker(testTask)
|
new TestZkWorker(testTask)
|
||||||
|
@ -212,8 +223,8 @@ public class SimpleResourceManagementStrategyTest
|
||||||
EasyMock.replay(autoScalingStrategy);
|
EasyMock.replay(autoScalingStrategy);
|
||||||
|
|
||||||
boolean provisionedSomething = simpleResourceManagementStrategy.doProvision(
|
boolean provisionedSomething = simpleResourceManagementStrategy.doProvision(
|
||||||
Arrays.<TaskRunnerWorkItem>asList(
|
Arrays.<RemoteTaskRunnerWorkItem>asList(
|
||||||
new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime())
|
new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime())
|
||||||
),
|
),
|
||||||
Arrays.<ZkWorker>asList(
|
Arrays.<ZkWorker>asList(
|
||||||
new TestZkWorker(testTask)
|
new TestZkWorker(testTask)
|
||||||
|
@ -230,8 +241,8 @@ public class SimpleResourceManagementStrategyTest
|
||||||
Thread.sleep(2000);
|
Thread.sleep(2000);
|
||||||
|
|
||||||
provisionedSomething = simpleResourceManagementStrategy.doProvision(
|
provisionedSomething = simpleResourceManagementStrategy.doProvision(
|
||||||
Arrays.<TaskRunnerWorkItem>asList(
|
Arrays.<RemoteTaskRunnerWorkItem>asList(
|
||||||
new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime())
|
new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime())
|
||||||
),
|
),
|
||||||
Arrays.<ZkWorker>asList(
|
Arrays.<ZkWorker>asList(
|
||||||
new TestZkWorker(testTask)
|
new TestZkWorker(testTask)
|
||||||
|
@ -264,8 +275,8 @@ public class SimpleResourceManagementStrategyTest
|
||||||
EasyMock.replay(autoScalingStrategy);
|
EasyMock.replay(autoScalingStrategy);
|
||||||
|
|
||||||
boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate(
|
boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate(
|
||||||
Arrays.<TaskRunnerWorkItem>asList(
|
Arrays.<RemoteTaskRunnerWorkItem>asList(
|
||||||
new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime())
|
new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime())
|
||||||
),
|
),
|
||||||
Arrays.<ZkWorker>asList(
|
Arrays.<ZkWorker>asList(
|
||||||
new TestZkWorker(null)
|
new TestZkWorker(null)
|
||||||
|
@ -294,8 +305,8 @@ public class SimpleResourceManagementStrategyTest
|
||||||
EasyMock.replay(autoScalingStrategy);
|
EasyMock.replay(autoScalingStrategy);
|
||||||
|
|
||||||
boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate(
|
boolean terminatedSomething = simpleResourceManagementStrategy.doTerminate(
|
||||||
Arrays.<TaskRunnerWorkItem>asList(
|
Arrays.<RemoteTaskRunnerWorkItem>asList(
|
||||||
new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime())
|
new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime())
|
||||||
),
|
),
|
||||||
Arrays.<ZkWorker>asList(
|
Arrays.<ZkWorker>asList(
|
||||||
new TestZkWorker(null)
|
new TestZkWorker(null)
|
||||||
|
@ -309,8 +320,8 @@ public class SimpleResourceManagementStrategyTest
|
||||||
);
|
);
|
||||||
|
|
||||||
terminatedSomething = simpleResourceManagementStrategy.doTerminate(
|
terminatedSomething = simpleResourceManagementStrategy.doTerminate(
|
||||||
Arrays.<TaskRunnerWorkItem>asList(
|
Arrays.<RemoteTaskRunnerWorkItem>asList(
|
||||||
new TaskRunnerWorkItem(testTask, null, null, null).withQueueInsertionTime(new DateTime())
|
new RemoteTaskRunnerWorkItem(testTask, null).withQueueInsertionTime(new DateTime())
|
||||||
),
|
),
|
||||||
Arrays.<ZkWorker>asList(
|
Arrays.<ZkWorker>asList(
|
||||||
new TestZkWorker(null)
|
new TestZkWorker(null)
|
||||||
|
@ -334,18 +345,18 @@ public class SimpleResourceManagementStrategyTest
|
||||||
Task testTask
|
Task testTask
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(new Worker("host", "ip", 3, "version"), null, null);
|
super(new Worker("host", "ip", 3, "version"), null, new DefaultObjectMapper());
|
||||||
|
|
||||||
this.testTask = testTask;
|
this.testTask = testTask;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Set<String> getRunningTasks()
|
public Map<String, TaskStatus> getRunningTasks()
|
||||||
{
|
{
|
||||||
if (testTask == null) {
|
if (testTask == null) {
|
||||||
return Sets.newHashSet();
|
return Maps.newHashMap();
|
||||||
}
|
}
|
||||||
return Sets.newHashSet(testTask.getId());
|
return ImmutableMap.of(testTask.getId(), TaskStatus.running(testTask.getId()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
6
pom.xml
6
pom.xml
|
@ -23,7 +23,7 @@
|
||||||
<groupId>com.metamx</groupId>
|
<groupId>com.metamx</groupId>
|
||||||
<artifactId>druid</artifactId>
|
<artifactId>druid</artifactId>
|
||||||
<packaging>pom</packaging>
|
<packaging>pom</packaging>
|
||||||
<version>0.5.19-SNAPSHOT</version>
|
<version>0.5.33-SNAPSHOT</version>
|
||||||
<name>druid</name>
|
<name>druid</name>
|
||||||
<description>druid</description>
|
<description>druid</description>
|
||||||
<scm>
|
<scm>
|
||||||
|
@ -38,7 +38,7 @@
|
||||||
|
|
||||||
<properties>
|
<properties>
|
||||||
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
|
<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>
|
<apache.curator.version>2.1.0-incubating</apache.curator.version>
|
||||||
</properties>
|
</properties>
|
||||||
|
|
||||||
|
@ -80,7 +80,7 @@
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>com.metamx</groupId>
|
<groupId>com.metamx</groupId>
|
||||||
<artifactId>server-metrics</artifactId>
|
<artifactId>server-metrics</artifactId>
|
||||||
<version>0.0.2</version>
|
<version>0.0.3</version>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
<dependency>
|
<dependency>
|
||||||
|
|
|
@ -28,7 +28,7 @@
|
||||||
<parent>
|
<parent>
|
||||||
<groupId>com.metamx</groupId>
|
<groupId>com.metamx</groupId>
|
||||||
<artifactId>druid</artifactId>
|
<artifactId>druid</artifactId>
|
||||||
<version>0.5.19-SNAPSHOT</version>
|
<version>0.5.33-SNAPSHOT</version>
|
||||||
</parent>
|
</parent>
|
||||||
|
|
||||||
<dependencies>
|
<dependencies>
|
||||||
|
|
|
@ -164,6 +164,17 @@ public class RealtimeManager implements QuerySegmentWalker
|
||||||
final InputRow inputRow;
|
final InputRow inputRow;
|
||||||
try {
|
try {
|
||||||
inputRow = firehose.nextRow();
|
inputRow = firehose.nextRow();
|
||||||
|
}
|
||||||
|
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());
|
final Sink sink = plumber.getSink(inputRow.getTimestampFromEpoch());
|
||||||
if (sink == null) {
|
if (sink == null) {
|
||||||
|
@ -185,12 +196,6 @@ public class RealtimeManager implements QuerySegmentWalker
|
||||||
nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis();
|
nextFlush = new DateTime().plus(intermediatePersistPeriod).getMillis();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch (FormattedException e) {
|
|
||||||
log.info(e, "unparseable line: %s", e.getDetails());
|
|
||||||
metrics.incrementUnparseable();
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} catch (RuntimeException e) {
|
} catch (RuntimeException e) {
|
||||||
log.makeAlert(e, "RuntimeException aborted realtime processing[%s]", fireDepartment.getSchema().getDataSource())
|
log.makeAlert(e, "RuntimeException aborted realtime processing[%s]", fireDepartment.getSchema().getDataSource())
|
||||||
.emit();
|
.emit();
|
||||||
|
|
|
@ -31,7 +31,7 @@ import java.util.Map;
|
||||||
*/
|
*/
|
||||||
public class RealtimeMetricsMonitor extends AbstractMonitor
|
public class RealtimeMetricsMonitor extends AbstractMonitor
|
||||||
{
|
{
|
||||||
Map<FireDepartment, FireDepartmentMetrics> previousValues;
|
private final Map<FireDepartment, FireDepartmentMetrics> previousValues;
|
||||||
private final List<FireDepartment> fireDepartments;
|
private final List<FireDepartment> fireDepartments;
|
||||||
|
|
||||||
public RealtimeMetricsMonitor(List<FireDepartment> fireDepartments)
|
public RealtimeMetricsMonitor(List<FireDepartment> fireDepartments)
|
||||||
|
|
|
@ -1,142 +0,0 @@
|
||||||
/*
|
|
||||||
* Druid - a distributed column store.
|
|
||||||
* Copyright (C) 2012 Metamarkets Group Inc.
|
|
||||||
*
|
|
||||||
* This program is free software; you can redistribute it and/or
|
|
||||||
* modify it under the terms of the GNU General Public License
|
|
||||||
* as published by the Free Software Foundation; either version 2
|
|
||||||
* of the License, or (at your option) any later version.
|
|
||||||
*
|
|
||||||
* This program is distributed in the hope that it will be useful,
|
|
||||||
* but WITHOUT ANY WARRANTY; without even the implied warranty of
|
|
||||||
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
|
|
||||||
* GNU General Public License for more details.
|
|
||||||
*
|
|
||||||
* You should have received a copy of the GNU General Public License
|
|
||||||
* along with this program; if not, write to the Free Software
|
|
||||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
|
||||||
*/
|
|
||||||
|
|
||||||
package com.metamx.druid.realtime.firehose;
|
|
||||||
|
|
||||||
import com.google.common.base.Throwables;
|
|
||||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
|
||||||
import com.metamx.common.concurrent.ScheduledExecutors;
|
|
||||||
import com.metamx.common.logger.Logger;
|
|
||||||
import com.metamx.druid.index.v1.IndexGranularity;
|
|
||||||
import com.metamx.druid.input.InputRow;
|
|
||||||
import com.metamx.druid.realtime.plumber.IntervalRejectionPolicyFactory;
|
|
||||||
import com.metamx.druid.realtime.plumber.RejectionPolicy;
|
|
||||||
import org.joda.time.DateTime;
|
|
||||||
import org.joda.time.Duration;
|
|
||||||
import org.joda.time.Interval;
|
|
||||||
import org.joda.time.Period;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.concurrent.Callable;
|
|
||||||
import java.util.concurrent.Executors;
|
|
||||||
import java.util.concurrent.ScheduledExecutorService;
|
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
|
||||||
|
|
||||||
/**
|
|
||||||
*/
|
|
||||||
public class GracefulShutdownFirehose implements Firehose
|
|
||||||
{
|
|
||||||
private static final Logger log = new Logger(GracefulShutdownFirehose.class);
|
|
||||||
|
|
||||||
private final Firehose firehose;
|
|
||||||
private final IndexGranularity segmentGranularity;
|
|
||||||
private final long windowMillis;
|
|
||||||
private final ScheduledExecutorService scheduledExecutor;
|
|
||||||
private final RejectionPolicy rejectionPolicy;
|
|
||||||
|
|
||||||
// when this is set to false, the firehose will have no more rows
|
|
||||||
private final AtomicBoolean valveOn = new AtomicBoolean(true);
|
|
||||||
|
|
||||||
// when this is set to true, the firehose will begin rejecting events
|
|
||||||
private volatile boolean beginRejectionPolicy = false;
|
|
||||||
|
|
||||||
public GracefulShutdownFirehose(
|
|
||||||
Firehose firehose,
|
|
||||||
IndexGranularity segmentGranularity,
|
|
||||||
Period windowPeriod
|
|
||||||
)
|
|
||||||
{
|
|
||||||
this.firehose = firehose;
|
|
||||||
this.segmentGranularity = segmentGranularity;
|
|
||||||
this.windowMillis = windowPeriod.toStandardDuration().getMillis() * 2;
|
|
||||||
this.scheduledExecutor = Executors.newScheduledThreadPool(
|
|
||||||
1,
|
|
||||||
new ThreadFactoryBuilder()
|
|
||||||
.setDaemon(true)
|
|
||||||
.setNameFormat("firehose_scheduled_%d")
|
|
||||||
.build()
|
|
||||||
);
|
|
||||||
|
|
||||||
final long truncatedNow = segmentGranularity.truncate(new DateTime()).getMillis();
|
|
||||||
final long end = segmentGranularity.increment(truncatedNow);
|
|
||||||
|
|
||||||
this.rejectionPolicy = new IntervalRejectionPolicyFactory(new Interval(truncatedNow, end)).create(windowPeriod);
|
|
||||||
}
|
|
||||||
|
|
||||||
public void shutdown() throws IOException
|
|
||||||
{
|
|
||||||
final long truncatedNow = segmentGranularity.truncate(new DateTime()).getMillis();
|
|
||||||
final long end = segmentGranularity.increment(truncatedNow) + windowMillis;
|
|
||||||
final Duration timeUntilShutdown = new Duration(System.currentTimeMillis(), end);
|
|
||||||
|
|
||||||
log.info("Shutdown at approx. %s (in %s)", new DateTime(end), timeUntilShutdown);
|
|
||||||
|
|
||||||
ScheduledExecutors.scheduleWithFixedDelay(
|
|
||||||
scheduledExecutor,
|
|
||||||
timeUntilShutdown,
|
|
||||||
new Callable<ScheduledExecutors.Signal>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public ScheduledExecutors.Signal call() throws Exception
|
|
||||||
{
|
|
||||||
try {
|
|
||||||
valveOn.set(false);
|
|
||||||
}
|
|
||||||
catch (Exception e) {
|
|
||||||
throw Throwables.propagate(e);
|
|
||||||
}
|
|
||||||
|
|
||||||
return ScheduledExecutors.Signal.STOP;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
|
|
||||||
beginRejectionPolicy = true;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean hasMore()
|
|
||||||
{
|
|
||||||
return valveOn.get() && firehose.hasMore();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public InputRow nextRow()
|
|
||||||
{
|
|
||||||
InputRow next = firehose.nextRow();
|
|
||||||
|
|
||||||
if (!beginRejectionPolicy || rejectionPolicy.accept(next.getTimestampFromEpoch())) {
|
|
||||||
return next;
|
|
||||||
}
|
|
||||||
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public Runnable commit()
|
|
||||||
{
|
|
||||||
return firehose.commit();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void close() throws IOException
|
|
||||||
{
|
|
||||||
firehose.close();
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -1,42 +0,0 @@
|
||||||
package com.metamx.druid.realtime.plumber;
|
|
||||||
|
|
||||||
import org.joda.time.DateTime;
|
|
||||||
import org.joda.time.Interval;
|
|
||||||
import org.joda.time.Period;
|
|
||||||
|
|
||||||
/**
|
|
||||||
*/
|
|
||||||
public class IntervalRejectionPolicyFactory implements RejectionPolicyFactory
|
|
||||||
{
|
|
||||||
private final Interval interval;
|
|
||||||
|
|
||||||
public IntervalRejectionPolicyFactory(Interval interval)
|
|
||||||
{
|
|
||||||
this.interval = interval;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public RejectionPolicy create(Period windowPeriod)
|
|
||||||
{
|
|
||||||
return new RejectionPolicy()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public DateTime getCurrMaxTime()
|
|
||||||
{
|
|
||||||
return new DateTime();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean accept(long timestamp)
|
|
||||||
{
|
|
||||||
return interval.contains(timestamp);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String toString()
|
|
||||||
{
|
|
||||||
return String.format("interval-%s", interval);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -0,0 +1,26 @@
|
||||||
|
package com.metamx.druid.realtime.plumber;
|
||||||
|
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
import org.joda.time.Period;
|
||||||
|
|
||||||
|
public class NoopRejectionPolicyFactory implements RejectionPolicyFactory
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public RejectionPolicy create(Period windowPeriod)
|
||||||
|
{
|
||||||
|
return new RejectionPolicy()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public DateTime getCurrMaxTime()
|
||||||
|
{
|
||||||
|
return new DateTime(0);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean accept(long timestamp)
|
||||||
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
}
|
|
@ -25,6 +25,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Joiner;
|
import com.google.common.base.Joiner;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.base.Predicates;
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
|
@ -37,6 +38,8 @@ import com.metamx.common.Pair;
|
||||||
import com.metamx.common.concurrent.ScheduledExecutors;
|
import com.metamx.common.concurrent.ScheduledExecutors;
|
||||||
import com.metamx.common.guava.FunctionalIterable;
|
import com.metamx.common.guava.FunctionalIterable;
|
||||||
import com.metamx.druid.Query;
|
import com.metamx.druid.Query;
|
||||||
|
import com.metamx.druid.TimelineObjectHolder;
|
||||||
|
import com.metamx.druid.VersionedIntervalTimeline;
|
||||||
import com.metamx.druid.client.DataSegment;
|
import com.metamx.druid.client.DataSegment;
|
||||||
import com.metamx.druid.client.DruidServer;
|
import com.metamx.druid.client.DruidServer;
|
||||||
import com.metamx.druid.client.ServerView;
|
import com.metamx.druid.client.ServerView;
|
||||||
|
@ -50,6 +53,7 @@ import com.metamx.druid.index.v1.IndexGranularity;
|
||||||
import com.metamx.druid.index.v1.IndexIO;
|
import com.metamx.druid.index.v1.IndexIO;
|
||||||
import com.metamx.druid.index.v1.IndexMerger;
|
import com.metamx.druid.index.v1.IndexMerger;
|
||||||
import com.metamx.druid.loading.DataSegmentPusher;
|
import com.metamx.druid.loading.DataSegmentPusher;
|
||||||
|
import com.metamx.druid.partition.SingleElementPartitionChunk;
|
||||||
import com.metamx.druid.query.MetricsEmittingQueryRunner;
|
import com.metamx.druid.query.MetricsEmittingQueryRunner;
|
||||||
import com.metamx.druid.query.QueryRunner;
|
import com.metamx.druid.query.QueryRunner;
|
||||||
import com.metamx.druid.query.QueryRunnerFactory;
|
import com.metamx.druid.query.QueryRunnerFactory;
|
||||||
|
@ -186,6 +190,9 @@ public class RealtimePlumberSchool implements PlumberSchool
|
||||||
private volatile ScheduledExecutorService scheduledExecutor = null;
|
private volatile ScheduledExecutorService scheduledExecutor = null;
|
||||||
|
|
||||||
private final Map<Long, Sink> sinks = Maps.newConcurrentMap();
|
private final Map<Long, Sink> sinks = Maps.newConcurrentMap();
|
||||||
|
private final VersionedIntervalTimeline<String, Sink> sinkTimeline = new VersionedIntervalTimeline<String, Sink>(
|
||||||
|
String.CASE_INSENSITIVE_ORDER
|
||||||
|
);
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void startJob()
|
public void startJob()
|
||||||
|
@ -219,6 +226,7 @@ public class RealtimePlumberSchool implements PlumberSchool
|
||||||
try {
|
try {
|
||||||
segmentAnnouncer.announceSegment(retVal.getSegment());
|
segmentAnnouncer.announceSegment(retVal.getSegment());
|
||||||
sinks.put(truncatedTime, retVal);
|
sinks.put(truncatedTime, retVal);
|
||||||
|
sinkTimeline.add(retVal.getInterval(), retVal.getVersion(), new SingleElementPartitionChunk<Sink>(retVal));
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
log.makeAlert(e, "Failed to announce new segment[%s]", schema.getDataSource())
|
log.makeAlert(e, "Failed to announce new segment[%s]", schema.getDataSource())
|
||||||
|
@ -247,17 +255,23 @@ public class RealtimePlumberSchool implements PlumberSchool
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
List<TimelineObjectHolder<String, Sink>> querySinks = Lists.newArrayList();
|
||||||
|
for (Interval interval : query.getIntervals()) {
|
||||||
|
querySinks.addAll(sinkTimeline.lookup(interval));
|
||||||
|
}
|
||||||
|
|
||||||
return toolchest.mergeResults(
|
return toolchest.mergeResults(
|
||||||
factory.mergeRunners(
|
factory.mergeRunners(
|
||||||
EXEC,
|
EXEC,
|
||||||
FunctionalIterable
|
FunctionalIterable
|
||||||
.create(sinks.values())
|
.create(querySinks)
|
||||||
.transform(
|
.transform(
|
||||||
new Function<Sink, QueryRunner<T>>()
|
new Function<TimelineObjectHolder<String, Sink>, QueryRunner<T>>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public QueryRunner<T> apply(Sink input)
|
public QueryRunner<T> apply(TimelineObjectHolder<String, Sink> holder)
|
||||||
{
|
{
|
||||||
|
final Sink theSink = holder.getObject().getChunk(0).getObject();
|
||||||
return new SpecificSegmentQueryRunner<T>(
|
return new SpecificSegmentQueryRunner<T>(
|
||||||
new MetricsEmittingQueryRunner<T>(
|
new MetricsEmittingQueryRunner<T>(
|
||||||
emitter,
|
emitter,
|
||||||
|
@ -265,7 +279,7 @@ public class RealtimePlumberSchool implements PlumberSchool
|
||||||
factory.mergeRunners(
|
factory.mergeRunners(
|
||||||
EXEC,
|
EXEC,
|
||||||
Iterables.transform(
|
Iterables.transform(
|
||||||
input,
|
theSink,
|
||||||
new Function<FireHydrant, QueryRunner<T>>()
|
new Function<FireHydrant, QueryRunner<T>>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
@ -279,9 +293,9 @@ public class RealtimePlumberSchool implements PlumberSchool
|
||||||
),
|
),
|
||||||
new SpecificSegmentSpec(
|
new SpecificSegmentSpec(
|
||||||
new SegmentDescriptor(
|
new SegmentDescriptor(
|
||||||
input.getInterval(),
|
holder.getInterval(),
|
||||||
input.getSegment().getVersion(),
|
theSink.getSegment().getVersion(),
|
||||||
input.getSegment().getShardSpec().getPartitionNum()
|
theSink.getSegment().getShardSpec().getPartitionNum()
|
||||||
)
|
)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
@ -319,11 +333,90 @@ public class RealtimePlumberSchool implements PlumberSchool
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Submits persist-n-merge task for a Sink to the persistExecutor
|
||||||
|
private void persistAndMerge(final long truncatedTime, final Sink sink)
|
||||||
|
{
|
||||||
|
final String threadName = String.format(
|
||||||
|
"%s-%s-persist-n-merge", schema.getDataSource(), new DateTime(truncatedTime)
|
||||||
|
);
|
||||||
|
persistExecutor.execute(
|
||||||
|
new ThreadRenamingRunnable(threadName)
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public void doRun()
|
||||||
|
{
|
||||||
|
final Interval interval = sink.getInterval();
|
||||||
|
|
||||||
|
for (FireHydrant hydrant : sink) {
|
||||||
|
if (!hydrant.hasSwapped()) {
|
||||||
|
log.info("Hydrant[%s] hasn't swapped yet, swapping. Sink[%s]", hydrant, sink);
|
||||||
|
final int rowCount = persistHydrant(hydrant, schema, interval);
|
||||||
|
metrics.incrementRowOutputCount(rowCount);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
final File mergedTarget = new File(computePersistDir(schema, interval), "merged");
|
||||||
|
if (mergedTarget.exists()) {
|
||||||
|
log.info("Skipping already-merged sink: %s", sink);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
File mergedFile = null;
|
||||||
|
try {
|
||||||
|
List<QueryableIndex> indexes = Lists.newArrayList();
|
||||||
|
for (FireHydrant fireHydrant : sink) {
|
||||||
|
Segment segment = fireHydrant.getSegment();
|
||||||
|
final QueryableIndex queryableIndex = segment.asQueryableIndex();
|
||||||
|
log.info("Adding hydrant[%s]", fireHydrant);
|
||||||
|
indexes.add(queryableIndex);
|
||||||
|
}
|
||||||
|
|
||||||
|
mergedFile = IndexMerger.mergeQueryableIndex(
|
||||||
|
indexes,
|
||||||
|
schema.getAggregators(),
|
||||||
|
mergedTarget
|
||||||
|
);
|
||||||
|
|
||||||
|
QueryableIndex index = IndexIO.loadIndex(mergedFile);
|
||||||
|
|
||||||
|
DataSegment segment = dataSegmentPusher.push(
|
||||||
|
mergedFile,
|
||||||
|
sink.getSegment().withDimensions(Lists.newArrayList(index.getAvailableDimensions()))
|
||||||
|
);
|
||||||
|
|
||||||
|
segmentPublisher.publishSegment(segment);
|
||||||
|
}
|
||||||
|
catch (IOException e) {
|
||||||
|
log.makeAlert(e, "Failed to persist merged index[%s]", schema.getDataSource())
|
||||||
|
.addData("interval", interval)
|
||||||
|
.emit();
|
||||||
|
}
|
||||||
|
|
||||||
|
if (mergedFile != null) {
|
||||||
|
try {
|
||||||
|
if (mergedFile != null) {
|
||||||
|
log.info("Deleting Index File[%s]", mergedFile);
|
||||||
|
FileUtils.deleteDirectory(mergedFile);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (IOException e) {
|
||||||
|
log.warn(e, "Error deleting directory[%s]", mergedFile);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void finishJob()
|
public void finishJob()
|
||||||
{
|
{
|
||||||
log.info("Shutting down...");
|
log.info("Shutting down...");
|
||||||
|
|
||||||
|
for (final Map.Entry<Long, Sink> entry : sinks.entrySet()) {
|
||||||
|
persistAndMerge(entry.getKey(), entry.getValue());
|
||||||
|
}
|
||||||
|
|
||||||
while (!sinks.isEmpty()) {
|
while (!sinks.isEmpty()) {
|
||||||
try {
|
try {
|
||||||
log.info(
|
log.info(
|
||||||
|
@ -442,6 +535,11 @@ public class RealtimePlumberSchool implements PlumberSchool
|
||||||
|
|
||||||
Sink currSink = new Sink(sinkInterval, schema, versioningPolicy.getVersion(sinkInterval), hydrants);
|
Sink currSink = new Sink(sinkInterval, schema, versioningPolicy.getVersion(sinkInterval), hydrants);
|
||||||
sinks.put(sinkInterval.getStartMillis(), currSink);
|
sinks.put(sinkInterval.getStartMillis(), currSink);
|
||||||
|
sinkTimeline.add(
|
||||||
|
currSink.getInterval(),
|
||||||
|
currSink.getVersion(),
|
||||||
|
new SingleElementPartitionChunk<Sink>(currSink)
|
||||||
|
);
|
||||||
|
|
||||||
segmentAnnouncer.announceSegment(currSink.getSegment());
|
segmentAnnouncer.announceSegment(currSink.getSegment());
|
||||||
}
|
}
|
||||||
|
@ -490,6 +588,11 @@ public class RealtimePlumberSchool implements PlumberSchool
|
||||||
FileUtils.deleteDirectory(computePersistDir(schema, sink.getInterval()));
|
FileUtils.deleteDirectory(computePersistDir(schema, sink.getInterval()));
|
||||||
log.info("Removing sinkKey %d for segment %s", sinkKey, sink.getSegment().getIdentifier());
|
log.info("Removing sinkKey %d for segment %s", sinkKey, sink.getSegment().getIdentifier());
|
||||||
sinks.remove(sinkKey);
|
sinks.remove(sinkKey);
|
||||||
|
sinkTimeline.remove(
|
||||||
|
sink.getInterval(),
|
||||||
|
sink.getVersion(),
|
||||||
|
new SingleElementPartitionChunk<Sink>(sink)
|
||||||
|
);
|
||||||
|
|
||||||
synchronized (handoffCondition) {
|
synchronized (handoffCondition) {
|
||||||
handoffCondition.notifyAll();
|
handoffCondition.notifyAll();
|
||||||
|
@ -560,72 +663,7 @@ public class RealtimePlumberSchool implements PlumberSchool
|
||||||
}
|
}
|
||||||
|
|
||||||
for (final Map.Entry<Long, Sink> entry : sinksToPush) {
|
for (final Map.Entry<Long, Sink> entry : sinksToPush) {
|
||||||
final Sink sink = entry.getValue();
|
persistAndMerge(entry.getKey(), entry.getValue());
|
||||||
|
|
||||||
final String threadName = String.format(
|
|
||||||
"%s-%s-persist-n-merge", schema.getDataSource(), new DateTime(entry.getKey())
|
|
||||||
);
|
|
||||||
persistExecutor.execute(
|
|
||||||
new ThreadRenamingRunnable(threadName)
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public void doRun()
|
|
||||||
{
|
|
||||||
final Interval interval = sink.getInterval();
|
|
||||||
|
|
||||||
for (FireHydrant hydrant : sink) {
|
|
||||||
if (!hydrant.hasSwapped()) {
|
|
||||||
log.info("Hydrant[%s] hasn't swapped yet, swapping. Sink[%s]", hydrant, sink);
|
|
||||||
final int rowCount = persistHydrant(hydrant, schema, interval);
|
|
||||||
metrics.incrementRowOutputCount(rowCount);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
File mergedFile = null;
|
|
||||||
try {
|
|
||||||
List<QueryableIndex> indexes = Lists.newArrayList();
|
|
||||||
for (FireHydrant fireHydrant : sink) {
|
|
||||||
Segment segment = fireHydrant.getSegment();
|
|
||||||
final QueryableIndex queryableIndex = segment.asQueryableIndex();
|
|
||||||
log.info("Adding hydrant[%s]", fireHydrant);
|
|
||||||
indexes.add(queryableIndex);
|
|
||||||
}
|
|
||||||
|
|
||||||
mergedFile = IndexMerger.mergeQueryableIndex(
|
|
||||||
indexes,
|
|
||||||
schema.getAggregators(),
|
|
||||||
new File(computePersistDir(schema, interval), "merged")
|
|
||||||
);
|
|
||||||
|
|
||||||
QueryableIndex index = IndexIO.loadIndex(mergedFile);
|
|
||||||
|
|
||||||
DataSegment segment = dataSegmentPusher.push(
|
|
||||||
mergedFile,
|
|
||||||
sink.getSegment().withDimensions(Lists.newArrayList(index.getAvailableDimensions()))
|
|
||||||
);
|
|
||||||
|
|
||||||
segmentPublisher.publishSegment(segment);
|
|
||||||
}
|
|
||||||
catch (IOException e) {
|
|
||||||
log.makeAlert(e, "Failed to persist merged index[%s]", schema.getDataSource())
|
|
||||||
.addData("interval", interval)
|
|
||||||
.emit();
|
|
||||||
}
|
|
||||||
|
|
||||||
if (mergedFile != null) {
|
|
||||||
try {
|
|
||||||
if (mergedFile != null) {
|
|
||||||
log.info("Deleting Index File[%s]", mergedFile);
|
|
||||||
FileUtils.deleteDirectory(mergedFile);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
catch (IOException e) {
|
|
||||||
log.warn(e, "Error deleting directory[%s]", mergedFile);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if (stopped) {
|
if (stopped) {
|
||||||
|
|
|
@ -7,7 +7,8 @@ import org.joda.time.Period;
|
||||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||||
@JsonSubTypes(value = {
|
@JsonSubTypes(value = {
|
||||||
@JsonSubTypes.Type(name = "serverTime", value = ServerTimeRejectionPolicyFactory.class),
|
@JsonSubTypes.Type(name = "serverTime", value = ServerTimeRejectionPolicyFactory.class),
|
||||||
@JsonSubTypes.Type(name = "messageTime", value = MessageTimeRejectionPolicyFactory.class)
|
@JsonSubTypes.Type(name = "messageTime", value = MessageTimeRejectionPolicyFactory.class),
|
||||||
|
@JsonSubTypes.Type(name = "none", value = NoopRejectionPolicyFactory.class)
|
||||||
})
|
})
|
||||||
public interface RejectionPolicyFactory
|
public interface RejectionPolicyFactory
|
||||||
{
|
{
|
||||||
|
|
|
@ -90,6 +90,11 @@ public class Sink implements Iterable<FireHydrant>
|
||||||
makeNewCurrIndex(interval.getStartMillis(), schema);
|
makeNewCurrIndex(interval.getStartMillis(), schema);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public String getVersion()
|
||||||
|
{
|
||||||
|
return version;
|
||||||
|
}
|
||||||
|
|
||||||
public Interval getInterval()
|
public Interval getInterval()
|
||||||
{
|
{
|
||||||
return interval;
|
return interval;
|
||||||
|
|
|
@ -28,7 +28,7 @@
|
||||||
<parent>
|
<parent>
|
||||||
<groupId>com.metamx</groupId>
|
<groupId>com.metamx</groupId>
|
||||||
<artifactId>druid</artifactId>
|
<artifactId>druid</artifactId>
|
||||||
<version>0.5.19-SNAPSHOT</version>
|
<version>0.5.33-SNAPSHOT</version>
|
||||||
</parent>
|
</parent>
|
||||||
|
|
||||||
<dependencies>
|
<dependencies>
|
||||||
|
|
|
@ -24,11 +24,14 @@ import com.google.common.base.Predicates;
|
||||||
import com.google.common.collect.Ordering;
|
import com.google.common.collect.Ordering;
|
||||||
import com.metamx.common.ISE;
|
import com.metamx.common.ISE;
|
||||||
import com.metamx.common.guava.FunctionalIterable;
|
import com.metamx.common.guava.FunctionalIterable;
|
||||||
|
import com.metamx.common.guava.Sequence;
|
||||||
import com.metamx.druid.Query;
|
import com.metamx.druid.Query;
|
||||||
import com.metamx.druid.TimelineObjectHolder;
|
import com.metamx.druid.TimelineObjectHolder;
|
||||||
import com.metamx.druid.VersionedIntervalTimeline;
|
import com.metamx.druid.VersionedIntervalTimeline;
|
||||||
import com.metamx.druid.client.DataSegment;
|
import com.metamx.druid.client.DataSegment;
|
||||||
import com.metamx.druid.collect.CountingMap;
|
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.index.Segment;
|
||||||
import com.metamx.druid.loading.SegmentLoader;
|
import com.metamx.druid.loading.SegmentLoader;
|
||||||
import com.metamx.druid.loading.SegmentLoadingException;
|
import com.metamx.druid.loading.SegmentLoadingException;
|
||||||
|
@ -53,6 +56,7 @@ import com.metamx.emitter.service.ServiceMetricEvent;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
import java.io.IOException;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
@ -71,7 +75,7 @@ public class ServerManager implements QuerySegmentWalker
|
||||||
private final ServiceEmitter emitter;
|
private final ServiceEmitter emitter;
|
||||||
private final ExecutorService exec;
|
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> dataSourceSizes = new CountingMap<String>();
|
||||||
private final CountingMap<String> dataSourceCounts = new CountingMap<String>();
|
private final CountingMap<String> dataSourceCounts = new CountingMap<String>();
|
||||||
|
|
||||||
|
@ -88,7 +92,7 @@ public class ServerManager implements QuerySegmentWalker
|
||||||
|
|
||||||
this.exec = exec;
|
this.exec = exec;
|
||||||
|
|
||||||
this.dataSources = new HashMap<String, VersionedIntervalTimeline<String, Segment>>();
|
this.dataSources = new HashMap<String, VersionedIntervalTimeline<String, ReferenceCountingSegment>>();
|
||||||
}
|
}
|
||||||
|
|
||||||
public Map<String, Long> getDataSourceSizes()
|
public Map<String, Long> getDataSourceSizes()
|
||||||
|
@ -132,14 +136,14 @@ public class ServerManager implements QuerySegmentWalker
|
||||||
|
|
||||||
synchronized (lock) {
|
synchronized (lock) {
|
||||||
String dataSource = segment.getDataSource();
|
String dataSource = segment.getDataSource();
|
||||||
VersionedIntervalTimeline<String, Segment> loadedIntervals = dataSources.get(dataSource);
|
VersionedIntervalTimeline<String, ReferenceCountingSegment> loadedIntervals = dataSources.get(dataSource);
|
||||||
|
|
||||||
if (loadedIntervals == null) {
|
if (loadedIntervals == null) {
|
||||||
loadedIntervals = new VersionedIntervalTimeline<String, Segment>(Ordering.natural());
|
loadedIntervals = new VersionedIntervalTimeline<String, ReferenceCountingSegment>(Ordering.natural());
|
||||||
dataSources.put(dataSource, loadedIntervals);
|
dataSources.put(dataSource, loadedIntervals);
|
||||||
}
|
}
|
||||||
|
|
||||||
PartitionHolder<Segment> entry = loadedIntervals.findEntry(
|
PartitionHolder<ReferenceCountingSegment> entry = loadedIntervals.findEntry(
|
||||||
segment.getInterval(),
|
segment.getInterval(),
|
||||||
segment.getVersion()
|
segment.getVersion()
|
||||||
);
|
);
|
||||||
|
@ -149,7 +153,9 @@ public class ServerManager implements QuerySegmentWalker
|
||||||
}
|
}
|
||||||
|
|
||||||
loadedIntervals.add(
|
loadedIntervals.add(
|
||||||
segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(adapter)
|
segment.getInterval(),
|
||||||
|
segment.getVersion(),
|
||||||
|
segment.getShardSpec().createChunk(new ReferenceCountingSegment(adapter))
|
||||||
);
|
);
|
||||||
synchronized (dataSourceSizes) {
|
synchronized (dataSourceSizes) {
|
||||||
dataSourceSizes.add(dataSource, segment.getSize());
|
dataSourceSizes.add(dataSource, segment.getSize());
|
||||||
|
@ -164,17 +170,19 @@ public class ServerManager implements QuerySegmentWalker
|
||||||
{
|
{
|
||||||
String dataSource = segment.getDataSource();
|
String dataSource = segment.getDataSource();
|
||||||
synchronized (lock) {
|
synchronized (lock) {
|
||||||
VersionedIntervalTimeline<String, Segment> loadedIntervals = dataSources.get(dataSource);
|
VersionedIntervalTimeline<String, ReferenceCountingSegment> loadedIntervals = dataSources.get(dataSource);
|
||||||
|
|
||||||
if (loadedIntervals == null) {
|
if (loadedIntervals == null) {
|
||||||
log.info("Told to delete a queryable for a dataSource[%s] that doesn't exist.", dataSource);
|
log.info("Told to delete a queryable for a dataSource[%s] that doesn't exist.", dataSource);
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
PartitionChunk<Segment> removed = loadedIntervals.remove(
|
PartitionChunk<ReferenceCountingSegment> removed = loadedIntervals.remove(
|
||||||
segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk((Segment) null)
|
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) {
|
if (oldQueryable != null) {
|
||||||
synchronized (dataSourceSizes) {
|
synchronized (dataSourceSizes) {
|
||||||
|
@ -183,6 +191,16 @@ public class ServerManager implements QuerySegmentWalker
|
||||||
synchronized (dataSourceCounts) {
|
synchronized (dataSourceCounts) {
|
||||||
dataSourceCounts.add(dataSource, -1L);
|
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 {
|
} else {
|
||||||
log.info(
|
log.info(
|
||||||
"Told to delete a queryable on dataSource[%s] for interval[%s] and version [%s] that I don't have.",
|
"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 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) {
|
if (timeline == null) {
|
||||||
return new NoopQueryRunner<T>();
|
return new NoopQueryRunner<T>();
|
||||||
|
@ -214,20 +232,22 @@ public class ServerManager implements QuerySegmentWalker
|
||||||
FunctionalIterable<QueryRunner<T>> adapters = FunctionalIterable
|
FunctionalIterable<QueryRunner<T>> adapters = FunctionalIterable
|
||||||
.create(intervals)
|
.create(intervals)
|
||||||
.transformCat(
|
.transformCat(
|
||||||
new Function<Interval, Iterable<TimelineObjectHolder<String, Segment>>>()
|
new Function<Interval, Iterable<TimelineObjectHolder<String, ReferenceCountingSegment>>>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public Iterable<TimelineObjectHolder<String, Segment>> apply(Interval input)
|
public Iterable<TimelineObjectHolder<String, ReferenceCountingSegment>> apply(Interval input)
|
||||||
{
|
{
|
||||||
return timeline.lookup(input);
|
return timeline.lookup(input);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
)
|
)
|
||||||
.transformCat(
|
.transformCat(
|
||||||
new Function<TimelineObjectHolder<String, Segment>, Iterable<QueryRunner<T>>>()
|
new Function<TimelineObjectHolder<String, ReferenceCountingSegment>, Iterable<QueryRunner<T>>>()
|
||||||
{
|
{
|
||||||
@Override
|
@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) {
|
if (holder == null) {
|
||||||
return null;
|
return null;
|
||||||
|
@ -236,10 +256,10 @@ public class ServerManager implements QuerySegmentWalker
|
||||||
return FunctionalIterable
|
return FunctionalIterable
|
||||||
.create(holder.getObject())
|
.create(holder.getObject())
|
||||||
.transform(
|
.transform(
|
||||||
new Function<PartitionChunk<Segment>, QueryRunner<T>>()
|
new Function<PartitionChunk<ReferenceCountingSegment>, QueryRunner<T>>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public QueryRunner<T> apply(PartitionChunk<Segment> input)
|
public QueryRunner<T> apply(PartitionChunk<ReferenceCountingSegment> input)
|
||||||
{
|
{
|
||||||
return buildAndDecorateQueryRunner(
|
return buildAndDecorateQueryRunner(
|
||||||
factory,
|
factory,
|
||||||
|
@ -280,7 +300,7 @@ public class ServerManager implements QuerySegmentWalker
|
||||||
|
|
||||||
final QueryToolChest<T, Query<T>> toolChest = factory.getToolchest();
|
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) {
|
if (timeline == null) {
|
||||||
return new NoopQueryRunner<T>();
|
return new NoopQueryRunner<T>();
|
||||||
|
@ -293,9 +313,9 @@ public class ServerManager implements QuerySegmentWalker
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
@SuppressWarnings("unchecked")
|
@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()
|
input.getInterval(), input.getVersion()
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -303,12 +323,12 @@ public class ServerManager implements QuerySegmentWalker
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
final PartitionChunk<Segment> chunk = entry.getChunk(input.getPartitionNumber());
|
final PartitionChunk<ReferenceCountingSegment> chunk = entry.getChunk(input.getPartitionNumber());
|
||||||
if (chunk == null) {
|
if (chunk == null) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
final Segment adapter = chunk.getObject();
|
final ReferenceCountingSegment adapter = chunk.getObject();
|
||||||
return Arrays.asList(
|
return Arrays.asList(
|
||||||
buildAndDecorateQueryRunner(factory, toolChest, adapter, new SpecificSegmentSpec(input))
|
buildAndDecorateQueryRunner(factory, toolChest, adapter, new SpecificSegmentSpec(input))
|
||||||
);
|
);
|
||||||
|
@ -323,10 +343,10 @@ public class ServerManager implements QuerySegmentWalker
|
||||||
}
|
}
|
||||||
|
|
||||||
private <T> QueryRunner<T> buildAndDecorateQueryRunner(
|
private <T> QueryRunner<T> buildAndDecorateQueryRunner(
|
||||||
QueryRunnerFactory<T, Query<T>> factory,
|
final QueryRunnerFactory<T, Query<T>> factory,
|
||||||
final QueryToolChest<T, Query<T>> toolChest,
|
final QueryToolChest<T, Query<T>> toolChest,
|
||||||
Segment adapter,
|
final ReferenceCountingSegment adapter,
|
||||||
QuerySegmentSpec segmentSpec
|
final QuerySegmentSpec segmentSpec
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return new SpecificSegmentQueryRunner<T>(
|
return new SpecificSegmentQueryRunner<T>(
|
||||||
|
@ -335,7 +355,7 @@ public class ServerManager implements QuerySegmentWalker
|
||||||
new Function<Query<T>, ServiceMetricEvent.Builder>()
|
new Function<Query<T>, ServiceMetricEvent.Builder>()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public ServiceMetricEvent.Builder apply(@Nullable Query<T> input)
|
public ServiceMetricEvent.Builder apply(@Nullable final Query<T> input)
|
||||||
{
|
{
|
||||||
return toolChest.makeMetricBuilder(input);
|
return toolChest.makeMetricBuilder(input);
|
||||||
}
|
}
|
||||||
|
@ -343,7 +363,14 @@ public class ServerManager implements QuerySegmentWalker
|
||||||
new BySegmentQueryRunner<T>(
|
new BySegmentQueryRunner<T>(
|
||||||
adapter.getIdentifier(),
|
adapter.getIdentifier(),
|
||||||
adapter.getDataInterval().getStart(),
|
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(),
|
).withWaitMeasuredFromNow(),
|
||||||
segmentSpec
|
segmentSpec
|
||||||
|
|
|
@ -49,6 +49,7 @@ import com.metamx.druid.db.DatabaseSegmentManager;
|
||||||
import com.metamx.druid.db.DatabaseSegmentManagerConfig;
|
import com.metamx.druid.db.DatabaseSegmentManagerConfig;
|
||||||
import com.metamx.druid.db.DbConnector;
|
import com.metamx.druid.db.DbConnector;
|
||||||
import com.metamx.druid.db.DbConnectorConfig;
|
import com.metamx.druid.db.DbConnectorConfig;
|
||||||
|
import com.metamx.druid.initialization.CuratorConfig;
|
||||||
import com.metamx.druid.initialization.Initialization;
|
import com.metamx.druid.initialization.Initialization;
|
||||||
import com.metamx.druid.initialization.ServerConfig;
|
import com.metamx.druid.initialization.ServerConfig;
|
||||||
import com.metamx.druid.initialization.ServiceDiscoveryConfig;
|
import com.metamx.druid.initialization.ServiceDiscoveryConfig;
|
||||||
|
@ -124,10 +125,15 @@ public class MasterMain
|
||||||
final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle);
|
final ScheduledExecutorFactory scheduledExecutorFactory = ScheduledExecutors.createFactory(lifecycle);
|
||||||
|
|
||||||
final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class);
|
final ServiceDiscoveryConfig serviceDiscoveryConfig = configFactory.build(ServiceDiscoveryConfig.class);
|
||||||
CuratorFramework curatorFramework = Initialization.makeCuratorFramework(
|
CuratorFramework serviceDiscoveryCuratorFramework = Initialization.makeCuratorFramework(
|
||||||
serviceDiscoveryConfig,
|
serviceDiscoveryConfig,
|
||||||
lifecycle
|
lifecycle
|
||||||
);
|
);
|
||||||
|
final CuratorConfig curatorConfig = configFactory.build(CuratorConfig.class);
|
||||||
|
CuratorFramework curatorFramework = Initialization.makeCuratorFramework(
|
||||||
|
curatorConfig,
|
||||||
|
lifecycle
|
||||||
|
);
|
||||||
|
|
||||||
final ZkPathsConfig zkPaths = configFactory.build(ZkPathsConfig.class);
|
final ZkPathsConfig zkPaths = configFactory.build(ZkPathsConfig.class);
|
||||||
|
|
||||||
|
@ -201,7 +207,7 @@ public class MasterMain
|
||||||
final DruidMasterConfig druidMasterConfig = configFactory.build(DruidMasterConfig.class);
|
final DruidMasterConfig druidMasterConfig = configFactory.build(DruidMasterConfig.class);
|
||||||
|
|
||||||
final ServiceDiscovery serviceDiscovery = Initialization.makeServiceDiscoveryClient(
|
final ServiceDiscovery serviceDiscovery = Initialization.makeServiceDiscoveryClient(
|
||||||
curatorFramework,
|
serviceDiscoveryCuratorFramework,
|
||||||
serviceDiscoveryConfig,
|
serviceDiscoveryConfig,
|
||||||
lifecycle
|
lifecycle
|
||||||
);
|
);
|
||||||
|
@ -227,7 +233,10 @@ public class MasterMain
|
||||||
);
|
);
|
||||||
|
|
||||||
final LoadQueueTaskMaster taskMaster = new LoadQueueTaskMaster(
|
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(
|
final DruidMaster master = new DruidMaster(
|
||||||
|
|
|
@ -24,6 +24,8 @@ import com.metamx.druid.index.v1.IncrementalIndex;
|
||||||
import com.metamx.druid.index.v1.IncrementalIndexStorageAdapter;
|
import com.metamx.druid.index.v1.IncrementalIndexStorageAdapter;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public class IncrementalIndexSegment implements Segment
|
public class IncrementalIndexSegment implements Segment
|
||||||
|
@ -60,4 +62,10 @@ public class IncrementalIndexSegment implements Segment
|
||||||
{
|
{
|
||||||
return new IncrementalIndexStorageAdapter(index);
|
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 com.metamx.druid.index.v1.QueryableIndexStorageAdapter;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public class QueryableIndexSegment implements Segment
|
public class QueryableIndexSegment implements Segment
|
||||||
|
@ -59,4 +61,11 @@ public class QueryableIndexSegment implements Segment
|
||||||
{
|
{
|
||||||
return new QueryableIndexStorageAdapter(index);
|
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 com.metamx.druid.StorageAdapter;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public interface Segment
|
public interface Segment extends Closeable
|
||||||
{
|
{
|
||||||
public String getIdentifier();
|
public String getIdentifier();
|
||||||
public Interval getDataInterval();
|
public Interval getDataInterval();
|
||||||
|
|
|
@ -19,6 +19,8 @@
|
||||||
|
|
||||||
package com.metamx.druid.loading;
|
package com.metamx.druid.loading;
|
||||||
|
|
||||||
|
import org.joda.time.format.ISODateTimeFormat;
|
||||||
|
|
||||||
import com.google.common.base.Joiner;
|
import com.google.common.base.Joiner;
|
||||||
import com.metamx.druid.client.DataSegment;
|
import com.metamx.druid.client.DataSegment;
|
||||||
|
|
||||||
|
@ -41,4 +43,22 @@ public class DataSegmentPusherUtil
|
||||||
segment.getShardSpec().getPartitionNum()
|
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
|
@Override
|
||||||
public DataSegment push(File inDir, DataSegment segment) throws IOException
|
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));
|
Path outFile = new Path(String.format("%s/%s/index.zip", config.getStorageDirectory(), storageDir));
|
||||||
FileSystem fs = outFile.getFileSystem(hadoopConfig);
|
FileSystem fs = outFile.getFileSystem(hadoopConfig);
|
||||||
|
|
||||||
|
|
|
@ -22,6 +22,7 @@ package com.metamx.druid.loading;
|
||||||
import com.metamx.druid.index.QueryableIndex;
|
import com.metamx.druid.index.QueryableIndex;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue