diff --git a/client/pom.xml b/client/pom.xml index 87e84c8e38a..69e2999ec32 100644 --- a/client/pom.xml +++ b/client/pom.xml @@ -18,7 +18,8 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-client @@ -28,7 +29,7 @@ com.metamx druid - 0.4.33-SNAPSHOT + 0.5.0-SNAPSHOT @@ -39,7 +40,7 @@ com.metamx.druid - druid-index-common + druid-indexing-common ${project.parent.version} @@ -177,6 +178,10 @@ commons-cli commons-cli + + com.metamx + bytebuffer-collections + diff --git a/client/src/main/java/com/metamx/druid/client/indexing/IndexingServiceClient.java b/client/src/main/java/com/metamx/druid/client/indexing/IndexingServiceClient.java index e09a609a7bd..ea17b6b6838 100644 --- a/client/src/main/java/com/metamx/druid/client/indexing/IndexingServiceClient.java +++ b/client/src/main/java/com/metamx/druid/client/indexing/IndexingServiceClient.java @@ -108,7 +108,7 @@ public class IndexingServiceClient throw new ISE("Cannot find instance of indexingService"); } - return String.format("http://%s:%s/mmx/merger/v1", instance.getAddress(), instance.getPort()); + return String.format("http://%s:%s/druid/indexer/v1", instance.getAddress(), instance.getPort()); } catch (Exception e) { throw Throwables.propagate(e); diff --git a/common/pom.xml b/common/pom.xml index 0bfc10d87af..f8f6a5c6ffa 100644 --- a/common/pom.xml +++ b/common/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.4.33-SNAPSHOT + 0.5.0-SNAPSHOT diff --git a/publications/vldb/.gitignore b/doc/publications/vldb/.gitignore similarity index 100% rename from publications/vldb/.gitignore rename to doc/publications/vldb/.gitignore diff --git a/publications/vldb/Makefile b/doc/publications/vldb/Makefile similarity index 100% rename from publications/vldb/Makefile rename to doc/publications/vldb/Makefile diff --git a/publications/vldb/README.md b/doc/publications/vldb/README.md similarity index 100% rename from publications/vldb/README.md rename to doc/publications/vldb/README.md diff --git a/publications/vldb/druid.bib b/doc/publications/vldb/druid.bib similarity index 100% rename from publications/vldb/druid.bib rename to doc/publications/vldb/druid.bib diff --git a/publications/vldb/druid.pdf b/doc/publications/vldb/druid.pdf similarity index 100% rename from publications/vldb/druid.pdf rename to doc/publications/vldb/druid.pdf diff --git a/publications/vldb/druid.tex b/doc/publications/vldb/druid.tex similarity index 100% rename from publications/vldb/druid.tex rename to doc/publications/vldb/druid.tex diff --git a/publications/vldb/figures/Druid_cluster.png b/doc/publications/vldb/figures/Druid_cluster.png similarity index 100% rename from publications/vldb/figures/Druid_cluster.png rename to doc/publications/vldb/figures/Druid_cluster.png diff --git a/publications/vldb/figures/cluster_scan_rate.pdf b/doc/publications/vldb/figures/cluster_scan_rate.pdf similarity index 100% rename from publications/vldb/figures/cluster_scan_rate.pdf rename to doc/publications/vldb/figures/cluster_scan_rate.pdf diff --git a/publications/vldb/figures/concise_plot.pdf b/doc/publications/vldb/figures/concise_plot.pdf similarity index 100% rename from publications/vldb/figures/concise_plot.pdf rename to doc/publications/vldb/figures/concise_plot.pdf diff --git a/publications/vldb/figures/core_scan_rate.pdf b/doc/publications/vldb/figures/core_scan_rate.pdf similarity index 100% rename from publications/vldb/figures/core_scan_rate.pdf rename to doc/publications/vldb/figures/core_scan_rate.pdf diff --git a/publications/vldb/figures/data_reading.png b/doc/publications/vldb/figures/data_reading.png similarity index 100% rename from publications/vldb/figures/data_reading.png rename to doc/publications/vldb/figures/data_reading.png diff --git a/publications/vldb/figures/druid_message_bus.png b/doc/publications/vldb/figures/druid_message_bus.png similarity index 100% rename from publications/vldb/figures/druid_message_bus.png rename to doc/publications/vldb/figures/druid_message_bus.png diff --git a/publications/vldb/figures/druid_realtime_flow.png b/doc/publications/vldb/figures/druid_realtime_flow.png similarity index 100% rename from publications/vldb/figures/druid_realtime_flow.png rename to doc/publications/vldb/figures/druid_realtime_flow.png diff --git a/publications/vldb/figures/druid_segment.png b/doc/publications/vldb/figures/druid_segment.png similarity index 100% rename from publications/vldb/figures/druid_segment.png rename to doc/publications/vldb/figures/druid_segment.png diff --git a/publications/vldb/figures/zookeeper.png b/doc/publications/vldb/figures/zookeeper.png similarity index 100% rename from publications/vldb/figures/zookeeper.png rename to doc/publications/vldb/figures/zookeeper.png diff --git a/publications/vldb/src/druid_plot.R b/doc/publications/vldb/src/druid_plot.R similarity index 100% rename from publications/vldb/src/druid_plot.R rename to doc/publications/vldb/src/druid_plot.R diff --git a/publications/vldb/src/druid_tables.R b/doc/publications/vldb/src/druid_tables.R similarity index 100% rename from publications/vldb/src/druid_tables.R rename to doc/publications/vldb/src/druid_tables.R diff --git a/publications/vldb/vldb.cls b/doc/publications/vldb/vldb.cls similarity index 100% rename from publications/vldb/vldb.cls rename to doc/publications/vldb/vldb.cls diff --git a/examples/pom.xml b/examples/pom.xml index 495a3f61511..0c5e000dddc 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -9,7 +9,7 @@ com.metamx druid - 0.4.33-SNAPSHOT + 0.5.0-SNAPSHOT diff --git a/index-common/pom.xml b/indexing-common/pom.xml similarity index 94% rename from index-common/pom.xml rename to indexing-common/pom.xml index c66470ef96d..2561b10f3bf 100644 --- a/index-common/pom.xml +++ b/indexing-common/pom.xml @@ -21,14 +21,14 @@ 4.0.0 com.metamx.druid - druid-index-common - druid-index-common - Druid Indexer + druid-indexing-common + druid-indexing-common + Druid Indexing Common com.metamx druid - 0.4.33-SNAPSHOT + 0.5.0-SNAPSHOT diff --git a/index-common/src/main/java/com/metamx/druid/common/s3/S3Utils.java b/indexing-common/src/main/java/com/metamx/druid/common/s3/S3Utils.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/common/s3/S3Utils.java rename to indexing-common/src/main/java/com/metamx/druid/common/s3/S3Utils.java diff --git a/index-common/src/main/java/com/metamx/druid/index/QueryableIndex.java b/indexing-common/src/main/java/com/metamx/druid/index/QueryableIndex.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/QueryableIndex.java rename to indexing-common/src/main/java/com/metamx/druid/index/QueryableIndex.java diff --git a/index-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java b/indexing-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java rename to indexing-common/src/main/java/com/metamx/druid/index/SimpleQueryableIndex.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/AbstractColumn.java b/indexing-common/src/main/java/com/metamx/druid/index/column/AbstractColumn.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/AbstractColumn.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/AbstractColumn.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/BitmapIndex.java b/indexing-common/src/main/java/com/metamx/druid/index/column/BitmapIndex.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/BitmapIndex.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/BitmapIndex.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/Column.java b/indexing-common/src/main/java/com/metamx/druid/index/column/Column.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/Column.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/Column.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ColumnBuilder.java b/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnBuilder.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/ColumnBuilder.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/ColumnBuilder.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilities.java b/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilities.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilities.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilities.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java b/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/ColumnCapabilitiesImpl.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java b/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/ColumnDescriptor.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ColumnSelector.java b/indexing-common/src/main/java/com/metamx/druid/index/column/ColumnSelector.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/ColumnSelector.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/ColumnSelector.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ComplexColumn.java b/indexing-common/src/main/java/com/metamx/druid/index/column/ComplexColumn.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/ComplexColumn.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/ComplexColumn.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ComplexColumnImpl.java b/indexing-common/src/main/java/com/metamx/druid/index/column/ComplexColumnImpl.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/ComplexColumnImpl.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/ComplexColumnImpl.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/DictionaryEncodedColumn.java b/indexing-common/src/main/java/com/metamx/druid/index/column/DictionaryEncodedColumn.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/DictionaryEncodedColumn.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/DictionaryEncodedColumn.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/FloatColumn.java b/indexing-common/src/main/java/com/metamx/druid/index/column/FloatColumn.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/FloatColumn.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/FloatColumn.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/GenericColumn.java b/indexing-common/src/main/java/com/metamx/druid/index/column/GenericColumn.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/GenericColumn.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/GenericColumn.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/IndexedComplexColumn.java b/indexing-common/src/main/java/com/metamx/druid/index/column/IndexedComplexColumn.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/IndexedComplexColumn.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/IndexedComplexColumn.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java b/indexing-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/IndexedFloatsGenericColumn.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java b/indexing-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/IndexedLongsGenericColumn.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/LongColumn.java b/indexing-common/src/main/java/com/metamx/druid/index/column/LongColumn.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/LongColumn.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/LongColumn.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/RunLengthColumn.java b/indexing-common/src/main/java/com/metamx/druid/index/column/RunLengthColumn.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/RunLengthColumn.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/RunLengthColumn.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java b/indexing-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/SimpleColumn.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java b/indexing-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/SimpleDictionaryEncodedColumn.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/SpatialIndex.java b/indexing-common/src/main/java/com/metamx/druid/index/column/SpatialIndex.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/SpatialIndex.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/SpatialIndex.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/StringMultiValueColumn.java b/indexing-common/src/main/java/com/metamx/druid/index/column/StringMultiValueColumn.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/StringMultiValueColumn.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/StringMultiValueColumn.java diff --git a/index-common/src/main/java/com/metamx/druid/index/column/ValueType.java b/indexing-common/src/main/java/com/metamx/druid/index/column/ValueType.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/column/ValueType.java rename to indexing-common/src/main/java/com/metamx/druid/index/column/ValueType.java diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java b/indexing-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java rename to indexing-common/src/main/java/com/metamx/druid/index/serde/BitmapIndexColumnPartSupplier.java diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java b/indexing-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java rename to indexing-common/src/main/java/com/metamx/druid/index/serde/ColumnPartSerde.java diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java b/indexing-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java rename to indexing-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSerde.java diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSupplier.java b/indexing-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSupplier.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSupplier.java rename to indexing-common/src/main/java/com/metamx/druid/index/serde/ComplexColumnPartSupplier.java diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java b/indexing-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java rename to indexing-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnPartSerde.java diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnSupplier.java b/indexing-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnSupplier.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnSupplier.java rename to indexing-common/src/main/java/com/metamx/druid/index/serde/DictionaryEncodedColumnSupplier.java diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java b/indexing-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java rename to indexing-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnPartSerde.java diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnSupplier.java b/indexing-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnSupplier.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnSupplier.java rename to indexing-common/src/main/java/com/metamx/druid/index/serde/FloatGenericColumnSupplier.java diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java b/indexing-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java rename to indexing-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnPartSerde.java diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnSupplier.java b/indexing-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnSupplier.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnSupplier.java rename to indexing-common/src/main/java/com/metamx/druid/index/serde/LongGenericColumnSupplier.java diff --git a/index-common/src/main/java/com/metamx/druid/index/serde/SpatialIndexColumnPartSupplier.java b/indexing-common/src/main/java/com/metamx/druid/index/serde/SpatialIndexColumnPartSupplier.java similarity index 90% rename from index-common/src/main/java/com/metamx/druid/index/serde/SpatialIndexColumnPartSupplier.java rename to indexing-common/src/main/java/com/metamx/druid/index/serde/SpatialIndexColumnPartSupplier.java index 660d1cb3561..246da53ab7b 100644 --- a/index-common/src/main/java/com/metamx/druid/index/serde/SpatialIndexColumnPartSupplier.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/serde/SpatialIndexColumnPartSupplier.java @@ -27,15 +27,13 @@ import com.metamx.druid.kv.GenericIndexed; */ public class SpatialIndexColumnPartSupplier implements Supplier { - private static final ImmutableRTree EMPTY_SET = new ImmutableRTree(); - private final ImmutableRTree indexedTree; public SpatialIndexColumnPartSupplier( ImmutableRTree indexedTree ) { - this.indexedTree = (indexedTree == null) ? EMPTY_SET : indexedTree; + this.indexedTree = indexedTree; } @Override diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/ComplexMetricColumnSerializer.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/ComplexMetricColumnSerializer.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/ComplexMetricColumnSerializer.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/ComplexMetricColumnSerializer.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatBufferObjectStrategy.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatBufferObjectStrategy.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatBufferObjectStrategy.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatBufferObjectStrategy.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplier.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplier.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplier.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplier.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsSupplierSerializer.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsSupplierSerializer.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsSupplierSerializer.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedFloatsSupplierSerializer.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/CompressedLongBufferObjectStrategy.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedLongBufferObjectStrategy.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/CompressedLongBufferObjectStrategy.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedLongBufferObjectStrategy.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplier.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplier.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplier.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplier.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/CompressedLongsSupplierSerializer.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedLongsSupplierSerializer.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/CompressedLongsSupplierSerializer.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedLongsSupplierSerializer.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/CompressedObjectStrategy.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedObjectStrategy.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/CompressedObjectStrategy.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedObjectStrategy.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/CompressedPools.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedPools.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/CompressedPools.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/CompressedPools.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/DimensionColumn.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/DimensionColumn.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/DimensionColumn.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/DimensionColumn.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/FloatMetricColumnSerializer.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/FloatMetricColumnSerializer.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/FloatMetricColumnSerializer.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/FloatMetricColumnSerializer.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/InMemoryCompressedFloats.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/InMemoryCompressedFloats.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/InMemoryCompressedFloats.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/InMemoryCompressedFloats.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/InMemoryCompressedLongs.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/InMemoryCompressedLongs.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/InMemoryCompressedLongs.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/InMemoryCompressedLongs.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndex.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndex.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndex.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndex.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndexAdapter.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndexAdapter.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndexAdapter.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndexAdapter.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndexSchema.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndexSchema.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndexSchema.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/IncrementalIndexSchema.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/Index.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/Index.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/Index.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/Index.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java similarity index 97% rename from index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java index d9dc25a7f23..24f6750785f 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexIO.java @@ -692,25 +692,29 @@ public class IndexIO Map columns = Maps.newHashMap(); for (String dimension : index.getAvailableDimensions()) { + ColumnBuilder builder = new ColumnBuilder() + .setType(ValueType.STRING) + .setHasMultipleValues(true) + .setDictionaryEncodedColumn( + new DictionaryEncodedColumnSupplier( + index.getDimValueLookup(dimension), null, (index.getDimColumn(dimension)) + ) + ) + .setBitmapIndex( + new BitmapIndexColumnPartSupplier( + index.getInvertedIndexes().get(dimension), index.getDimValueLookup(dimension) + ) + ); + if (index.getSpatialIndexes().get(dimension) != null) { + builder.setSpatialIndex( + new SpatialIndexColumnPartSupplier( + index.getSpatialIndexes().get(dimension) + ) + ); + } columns.put( dimension.toLowerCase(), - new ColumnBuilder() - .setType(ValueType.STRING) - .setHasMultipleValues(true) - .setDictionaryEncodedColumn( - new DictionaryEncodedColumnSupplier( - index.getDimValueLookup(dimension), null, (index.getDimColumn(dimension)) - ) - ) - .setBitmapIndex( - new BitmapIndexColumnPartSupplier( - index.getInvertedIndexes().get(dimension), index.getDimValueLookup(dimension) - ) - ).setSpatialIndex( - new SpatialIndexColumnPartSupplier( - index.getSpatialIndexes().get(dimension) - ) - ).build() + builder.build() ); } diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/IndexMerger.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexMerger.java similarity index 95% rename from index-common/src/main/java/com/metamx/druid/index/v1/IndexMerger.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/IndexMerger.java index 11b77cf8172..3cd842f998b 100644 --- a/index-common/src/main/java/com/metamx/druid/index/v1/IndexMerger.java +++ b/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexMerger.java @@ -705,6 +705,11 @@ public class IndexMerger final File invertedFile = new File(v8OutDir, "inverted.drd"); Files.touch(invertedFile); out = Files.newOutputStreamSupplier(invertedFile, true); + + final File geoFile = new File(v8OutDir, "spatial.drd"); + Files.touch(geoFile); + OutputSupplier spatialOut = Files.newOutputStreamSupplier(geoFile, true); + for (int i = 0; i < mergedDimensions.size(); ++i) { long dimStartTime = System.currentTimeMillis(); String dimension = mergedDimensions.get(i); @@ -723,6 +728,18 @@ public class IndexMerger ); writer.open(); + boolean isSpatialDim = "spatial".equals(descriptions.get(dimension)); + ByteBufferWriter spatialWriter = null; + RTree tree = null; + IOPeon spatialIoPeon = new TmpFileIOPeon(); + if (isSpatialDim) { + spatialWriter = new ByteBufferWriter( + spatialIoPeon, dimension, IndexedRTree.objectStrategy + ); + spatialWriter.open(); + tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50)); + } + for (String dimVal : IndexedIterable.create(dimVals)) { progress.progress(); List> convertedInverteds = Lists.newArrayListWithCapacity(indexes.size()); @@ -745,6 +762,15 @@ public class IndexMerger } writer.write(ImmutableConciseSet.newImmutableFromMutable(bitset)); + + if (isSpatialDim && dimVal != null) { + List stringCoords = Lists.newArrayList(SPLITTER.split(dimVal)); + float[] coords = new float[stringCoords.size()]; + for (int j = 0; j < coords.length; j++) { + coords[j] = Float.valueOf(stringCoords.get(j)); + } + tree.insert(coords, bitset); + } } writer.close(); @@ -753,64 +779,16 @@ public class IndexMerger ioPeon.cleanup(); log.info("Completed dimension[%s] in %,d millis.", dimension, System.currentTimeMillis() - dimStartTime); - } - /************ Create Geographical Indexes *************/ - // FIXME: Rewrite when indexing is updated - Stopwatch stopwatch = new Stopwatch(); - stopwatch.start(); + if (isSpatialDim) { + spatialWriter.write(ImmutableRTree.newImmutableFromMutable(tree)); + spatialWriter.close(); - final File geoFile = new File(v8OutDir, "spatial.drd"); - Files.touch(geoFile); - out = Files.newOutputStreamSupplier(geoFile, true); - - for (int i = 0; i < mergedDimensions.size(); ++i) { - String dimension = mergedDimensions.get(i); - - if (!"spatial".equals(descriptions.get(dimension))) { - continue; + serializerUtils.writeString(spatialOut, dimension); + ByteStreams.copy(spatialWriter.combineStreams(), spatialOut); + spatialIoPeon.cleanup(); } - File dimOutFile = dimOuts.get(i).getFile(); - final MappedByteBuffer dimValsMapped = Files.map(dimOutFile); - - if (!dimension.equals(serializerUtils.readString(dimValsMapped))) { - throw new ISE("dimensions[%s] didn't equate!? This is a major WTF moment.", dimension); - } - Indexed dimVals = GenericIndexed.read(dimValsMapped, GenericIndexed.stringStrategy); - log.info("Indexing geo dimension[%s] with cardinality[%,d]", dimension, dimVals.size()); - - ByteBufferWriter writer = new ByteBufferWriter( - ioPeon, dimension, IndexedRTree.objectStrategy - ); - writer.open(); - - RTree tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50)); - - int count = 0; - for (String dimVal : IndexedIterable.create(dimVals)) { - progress.progress(); - if (dimVal == null) { - continue; - } - - List stringCoords = Lists.newArrayList(SPLITTER.split(dimVal)); - float[] coords = new float[stringCoords.size()]; - for (int j = 0; j < coords.length; j++) { - coords[j] = Float.valueOf(stringCoords.get(j)); - } - tree.insert(coords, count); - count++; - } - - writer.write(ImmutableRTree.newImmutableFromMutable(tree)); - writer.close(); - - serializerUtils.writeString(out, dimension); - ByteStreams.copy(writer.combineStreams(), out); - ioPeon.cleanup(); - - log.info("Completed spatial dimension[%s] in %,d millis.", dimension, stopwatch.elapsedMillis()); } log.info("outDir[%s] completed inverted.drd in %,d millis.", v8OutDir, System.currentTimeMillis() - startTime); diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/IndexableAdapter.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/IndexableAdapter.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/IndexableAdapter.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/IndexableAdapter.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/MMappedIndex.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/MMappedIndex.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/MMappedIndex.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/MMappedIndex.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/MMappedIndexAdapter.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/MMappedIndexAdapter.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/MMappedIndexAdapter.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/MMappedIndexAdapter.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/MetricColumnSerializer.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/MetricColumnSerializer.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/MetricColumnSerializer.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/MetricColumnSerializer.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/MetricHolder.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/MetricHolder.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/MetricHolder.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/MetricHolder.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/QueryableIndexIndexableAdapter.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/QueryableIndexIndexableAdapter.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/QueryableIndexIndexableAdapter.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/QueryableIndexIndexableAdapter.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/Rowboat.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/Rowboat.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/Rowboat.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/Rowboat.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionRowFormatter.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionRowFormatter.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionRowFormatter.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionRowFormatter.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionSchema.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionSchema.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionSchema.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/SpatialDimensionSchema.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricExtractor.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricExtractor.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricExtractor.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricExtractor.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricSerde.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricSerde.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricSerde.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetricSerde.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetrics.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetrics.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetrics.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/serde/ComplexMetrics.java diff --git a/index-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java b/indexing-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java rename to indexing-common/src/main/java/com/metamx/druid/index/v1/serde/Registererer.java diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/CSVDataSpec.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/CSVDataSpec.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/indexer/data/CSVDataSpec.java rename to indexing-common/src/main/java/com/metamx/druid/indexer/data/CSVDataSpec.java diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java rename to indexing-common/src/main/java/com/metamx/druid/indexer/data/DataSpec.java diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/DelimitedDataSpec.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/DelimitedDataSpec.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/indexer/data/DelimitedDataSpec.java rename to indexing-common/src/main/java/com/metamx/druid/indexer/data/DelimitedDataSpec.java diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java rename to indexing-common/src/main/java/com/metamx/druid/indexer/data/InputRowParser.java diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/JSONDataSpec.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/JSONDataSpec.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/indexer/data/JSONDataSpec.java rename to indexing-common/src/main/java/com/metamx/druid/indexer/data/JSONDataSpec.java diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/MapInputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/MapInputRowParser.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/indexer/data/MapInputRowParser.java rename to indexing-common/src/main/java/com/metamx/druid/indexer/data/MapInputRowParser.java diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java rename to indexing-common/src/main/java/com/metamx/druid/indexer/data/StringInputRowParser.java diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/TimestampSpec.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/TimestampSpec.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/indexer/data/TimestampSpec.java rename to indexing-common/src/main/java/com/metamx/druid/indexer/data/TimestampSpec.java diff --git a/index-common/src/main/java/com/metamx/druid/indexer/data/ToLowercaseDataSpec.java b/indexing-common/src/main/java/com/metamx/druid/indexer/data/ToLowercaseDataSpec.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/indexer/data/ToLowercaseDataSpec.java rename to indexing-common/src/main/java/com/metamx/druid/indexer/data/ToLowercaseDataSpec.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/ArrayBasedIndexedInts.java b/indexing-common/src/main/java/com/metamx/druid/kv/ArrayBasedIndexedInts.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/ArrayBasedIndexedInts.java rename to indexing-common/src/main/java/com/metamx/druid/kv/ArrayBasedIndexedInts.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/ArrayIndexed.java b/indexing-common/src/main/java/com/metamx/druid/kv/ArrayIndexed.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/ArrayIndexed.java rename to indexing-common/src/main/java/com/metamx/druid/kv/ArrayIndexed.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/ByteBufferSerializer.java b/indexing-common/src/main/java/com/metamx/druid/kv/ByteBufferSerializer.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/ByteBufferSerializer.java rename to indexing-common/src/main/java/com/metamx/druid/kv/ByteBufferSerializer.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/ByteBufferWriter.java b/indexing-common/src/main/java/com/metamx/druid/kv/ByteBufferWriter.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/ByteBufferWriter.java rename to indexing-common/src/main/java/com/metamx/druid/kv/ByteBufferWriter.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/ConciseCompressedIndexedInts.java b/indexing-common/src/main/java/com/metamx/druid/kv/ConciseCompressedIndexedInts.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/ConciseCompressedIndexedInts.java rename to indexing-common/src/main/java/com/metamx/druid/kv/ConciseCompressedIndexedInts.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/EmptyIndexedInts.java b/indexing-common/src/main/java/com/metamx/druid/kv/EmptyIndexedInts.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/EmptyIndexedInts.java rename to indexing-common/src/main/java/com/metamx/druid/kv/EmptyIndexedInts.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/GenericIndexed.java b/indexing-common/src/main/java/com/metamx/druid/kv/GenericIndexed.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/GenericIndexed.java rename to indexing-common/src/main/java/com/metamx/druid/kv/GenericIndexed.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/GenericIndexedWriter.java b/indexing-common/src/main/java/com/metamx/druid/kv/GenericIndexedWriter.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/GenericIndexedWriter.java rename to indexing-common/src/main/java/com/metamx/druid/kv/GenericIndexedWriter.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/IOPeon.java b/indexing-common/src/main/java/com/metamx/druid/kv/IOPeon.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/IOPeon.java rename to indexing-common/src/main/java/com/metamx/druid/kv/IOPeon.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/Indexed.java b/indexing-common/src/main/java/com/metamx/druid/kv/Indexed.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/Indexed.java rename to indexing-common/src/main/java/com/metamx/druid/kv/Indexed.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/IndexedFloats.java b/indexing-common/src/main/java/com/metamx/druid/kv/IndexedFloats.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/IndexedFloats.java rename to indexing-common/src/main/java/com/metamx/druid/kv/IndexedFloats.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/IndexedInts.java b/indexing-common/src/main/java/com/metamx/druid/kv/IndexedInts.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/IndexedInts.java rename to indexing-common/src/main/java/com/metamx/druid/kv/IndexedInts.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/IndexedIntsIterator.java b/indexing-common/src/main/java/com/metamx/druid/kv/IndexedIntsIterator.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/IndexedIntsIterator.java rename to indexing-common/src/main/java/com/metamx/druid/kv/IndexedIntsIterator.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/IndexedIterable.java b/indexing-common/src/main/java/com/metamx/druid/kv/IndexedIterable.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/IndexedIterable.java rename to indexing-common/src/main/java/com/metamx/druid/kv/IndexedIterable.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/IndexedList.java b/indexing-common/src/main/java/com/metamx/druid/kv/IndexedList.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/IndexedList.java rename to indexing-common/src/main/java/com/metamx/druid/kv/IndexedList.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/IndexedLongs.java b/indexing-common/src/main/java/com/metamx/druid/kv/IndexedLongs.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/IndexedLongs.java rename to indexing-common/src/main/java/com/metamx/druid/kv/IndexedLongs.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/IndexedRTree.java b/indexing-common/src/main/java/com/metamx/druid/kv/IndexedRTree.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/IndexedRTree.java rename to indexing-common/src/main/java/com/metamx/druid/kv/IndexedRTree.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/Indexedids.java b/indexing-common/src/main/java/com/metamx/druid/kv/Indexedids.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/Indexedids.java rename to indexing-common/src/main/java/com/metamx/druid/kv/Indexedids.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/IntBufferIndexedInts.java b/indexing-common/src/main/java/com/metamx/druid/kv/IntBufferIndexedInts.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/IntBufferIndexedInts.java rename to indexing-common/src/main/java/com/metamx/druid/kv/IntBufferIndexedInts.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/ListIndexed.java b/indexing-common/src/main/java/com/metamx/druid/kv/ListIndexed.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/ListIndexed.java rename to indexing-common/src/main/java/com/metamx/druid/kv/ListIndexed.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/ObjectStrategy.java b/indexing-common/src/main/java/com/metamx/druid/kv/ObjectStrategy.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/ObjectStrategy.java rename to indexing-common/src/main/java/com/metamx/druid/kv/ObjectStrategy.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/TmpFileIOPeon.java b/indexing-common/src/main/java/com/metamx/druid/kv/TmpFileIOPeon.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/TmpFileIOPeon.java rename to indexing-common/src/main/java/com/metamx/druid/kv/TmpFileIOPeon.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/VSizeIndexed.java b/indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexed.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/VSizeIndexed.java rename to indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexed.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/VSizeIndexedInts.java b/indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexedInts.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/VSizeIndexedInts.java rename to indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexedInts.java diff --git a/index-common/src/main/java/com/metamx/druid/kv/VSizeIndexedWriter.java b/indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexedWriter.java similarity index 100% rename from index-common/src/main/java/com/metamx/druid/kv/VSizeIndexedWriter.java rename to indexing-common/src/main/java/com/metamx/druid/kv/VSizeIndexedWriter.java diff --git a/index-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplierTest.java b/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplierTest.java similarity index 100% rename from index-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplierTest.java rename to indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsIndexedSupplierTest.java diff --git a/index-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsSupplierSerializerTest.java b/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsSupplierSerializerTest.java similarity index 100% rename from index-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsSupplierSerializerTest.java rename to indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedFloatsSupplierSerializerTest.java diff --git a/index-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplierTest.java b/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplierTest.java similarity index 100% rename from index-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplierTest.java rename to indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsIndexedSupplierTest.java diff --git a/index-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsSupplierSerializerTest.java b/indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsSupplierSerializerTest.java similarity index 100% rename from index-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsSupplierSerializerTest.java rename to indexing-common/src/test/java/com/metamx/druid/index/v1/CompressedLongsSupplierSerializerTest.java diff --git a/index-common/src/test/java/com/metamx/druid/index/v1/IOPeonForTesting.java b/indexing-common/src/test/java/com/metamx/druid/index/v1/IOPeonForTesting.java similarity index 100% rename from index-common/src/test/java/com/metamx/druid/index/v1/IOPeonForTesting.java rename to indexing-common/src/test/java/com/metamx/druid/index/v1/IOPeonForTesting.java diff --git a/index-common/src/test/java/com/metamx/druid/index/v1/InMemoryCompressedFloatsTest.java b/indexing-common/src/test/java/com/metamx/druid/index/v1/InMemoryCompressedFloatsTest.java similarity index 100% rename from index-common/src/test/java/com/metamx/druid/index/v1/InMemoryCompressedFloatsTest.java rename to indexing-common/src/test/java/com/metamx/druid/index/v1/InMemoryCompressedFloatsTest.java diff --git a/index-common/src/test/java/com/metamx/druid/index/v1/InMemoryCompressedLongsTest.java b/indexing-common/src/test/java/com/metamx/druid/index/v1/InMemoryCompressedLongsTest.java similarity index 100% rename from index-common/src/test/java/com/metamx/druid/index/v1/InMemoryCompressedLongsTest.java rename to indexing-common/src/test/java/com/metamx/druid/index/v1/InMemoryCompressedLongsTest.java diff --git a/index-common/src/test/java/com/metamx/druid/index/v1/IncrementalIndexTest.java b/indexing-common/src/test/java/com/metamx/druid/index/v1/IncrementalIndexTest.java similarity index 100% rename from index-common/src/test/java/com/metamx/druid/index/v1/IncrementalIndexTest.java rename to indexing-common/src/test/java/com/metamx/druid/index/v1/IncrementalIndexTest.java diff --git a/index-common/src/test/java/com/metamx/druid/kv/GenericIndexedTest.java b/indexing-common/src/test/java/com/metamx/druid/kv/GenericIndexedTest.java similarity index 100% rename from index-common/src/test/java/com/metamx/druid/kv/GenericIndexedTest.java rename to indexing-common/src/test/java/com/metamx/druid/kv/GenericIndexedTest.java diff --git a/index-common/src/test/java/com/metamx/druid/kv/IndexedIntsTest.java b/indexing-common/src/test/java/com/metamx/druid/kv/IndexedIntsTest.java similarity index 100% rename from index-common/src/test/java/com/metamx/druid/kv/IndexedIntsTest.java rename to indexing-common/src/test/java/com/metamx/druid/kv/IndexedIntsTest.java diff --git a/index-common/src/test/java/com/metamx/druid/kv/VSizeIndexedIntsTest.java b/indexing-common/src/test/java/com/metamx/druid/kv/VSizeIndexedIntsTest.java similarity index 100% rename from index-common/src/test/java/com/metamx/druid/kv/VSizeIndexedIntsTest.java rename to indexing-common/src/test/java/com/metamx/druid/kv/VSizeIndexedIntsTest.java diff --git a/index-common/src/test/java/com/metamx/druid/kv/VSizeIndexedTest.java b/indexing-common/src/test/java/com/metamx/druid/kv/VSizeIndexedTest.java similarity index 100% rename from index-common/src/test/java/com/metamx/druid/kv/VSizeIndexedTest.java rename to indexing-common/src/test/java/com/metamx/druid/kv/VSizeIndexedTest.java diff --git a/index-common/src/test/resources/druid.sample.tsv b/indexing-common/src/test/resources/druid.sample.tsv similarity index 100% rename from index-common/src/test/resources/druid.sample.tsv rename to indexing-common/src/test/resources/druid.sample.tsv diff --git a/index-common/src/test/resources/druid.sample.tsv.bottom b/indexing-common/src/test/resources/druid.sample.tsv.bottom similarity index 100% rename from index-common/src/test/resources/druid.sample.tsv.bottom rename to indexing-common/src/test/resources/druid.sample.tsv.bottom diff --git a/index-common/src/test/resources/druid.sample.tsv.top b/indexing-common/src/test/resources/druid.sample.tsv.top similarity index 100% rename from index-common/src/test/resources/druid.sample.tsv.top rename to indexing-common/src/test/resources/druid.sample.tsv.top diff --git a/indexer/README b/indexing-hadoop/README similarity index 100% rename from indexer/README rename to indexing-hadoop/README diff --git a/indexer/pom.xml b/indexing-hadoop/pom.xml similarity index 95% rename from indexer/pom.xml rename to indexing-hadoop/pom.xml index 2ba41ee7a1f..7cd09b4c2ff 100644 --- a/indexer/pom.xml +++ b/indexing-hadoop/pom.xml @@ -21,20 +21,20 @@ 4.0.0 com.metamx.druid - druid-indexer - druid-indexer - Druid Indexer + druid-indexing-hadoop + druid-indexing-hadoop + Druid Indexing Hadoop com.metamx druid - 0.4.33-SNAPSHOT + 0.5.0-SNAPSHOT com.metamx.druid - druid-index-common + druid-indexing-common ${project.parent.version} diff --git a/indexer/src/main/java/com/metamx/druid/indexer/Bucket.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/Bucket.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/Bucket.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/Bucket.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/DbUpdaterJob.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/DeterminePartitionsJob.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexer.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexer.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexer.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexer.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerAzkWrapper.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerAzkWrapper.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerAzkWrapper.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerAzkWrapper.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerConfig.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerJob.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerJob.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerJob.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerJob.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMain.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMain.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMain.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMain.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMapper.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMapper.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMapper.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerMapper.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerNode.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerNode.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerNode.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopDruidIndexerNode.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopIOPeon.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopIOPeon.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/HadoopIOPeon.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopIOPeon.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/HadoopyShardSpec.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopyShardSpec.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/HadoopyShardSpec.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/HadoopyShardSpec.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/IndexGeneratorJob.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/Jobby.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/Jobby.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/Jobby.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/Jobby.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/SortableBytes.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/SortableBytes.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/SortableBytes.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/SortableBytes.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/StringIntervalFunction.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/StringIntervalFunction.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/StringIntervalFunction.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/StringIntervalFunction.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/Utils.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/Utils.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/Utils.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/Utils.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/granularity/ArbitraryGranularitySpec.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/granularity/ArbitraryGranularitySpec.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/granularity/ArbitraryGranularitySpec.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/granularity/ArbitraryGranularitySpec.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/granularity/GranularitySpec.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/granularity/GranularitySpec.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/granularity/GranularitySpec.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/granularity/GranularitySpec.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/granularity/UniformGranularitySpec.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/granularity/UniformGranularitySpec.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/granularity/UniformGranularitySpec.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/granularity/UniformGranularitySpec.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/hadoop/FSSpideringIterator.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/hadoop/FSSpideringIterator.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/hadoop/FSSpideringIterator.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/hadoop/FSSpideringIterator.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/partitions/PartitionsSpec.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/partitions/PartitionsSpec.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/partitions/PartitionsSpec.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/partitions/PartitionsSpec.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/path/GranularUnprocessedPathSpec.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/path/GranularUnprocessedPathSpec.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/path/GranularUnprocessedPathSpec.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/path/GranularUnprocessedPathSpec.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/path/GranularityPathSpec.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/path/GranularityPathSpec.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/path/GranularityPathSpec.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/path/GranularityPathSpec.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/path/PathSpec.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/path/PathSpec.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/path/PathSpec.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/path/PathSpec.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/path/StaticPathSpec.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/path/StaticPathSpec.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/path/StaticPathSpec.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/path/StaticPathSpec.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/rollup/DataRollupSpec.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/rollup/DataRollupSpec.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/rollup/DataRollupSpec.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/rollup/DataRollupSpec.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/updater/DbUpdaterJobSpec.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/updater/DbUpdaterJobSpec.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/updater/DbUpdaterJobSpec.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/updater/DbUpdaterJobSpec.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/updater/UpdaterJobSpec.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/updater/UpdaterJobSpec.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/updater/UpdaterJobSpec.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/updater/UpdaterJobSpec.java diff --git a/indexer/src/main/java/com/metamx/druid/indexer/updater/ZkUpdaterJobSpec.java b/indexing-hadoop/src/main/java/com/metamx/druid/indexer/updater/ZkUpdaterJobSpec.java similarity index 100% rename from indexer/src/main/java/com/metamx/druid/indexer/updater/ZkUpdaterJobSpec.java rename to indexing-hadoop/src/main/java/com/metamx/druid/indexer/updater/ZkUpdaterJobSpec.java diff --git a/indexer/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java similarity index 100% rename from indexer/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java rename to indexing-hadoop/src/test/java/com/metamx/druid/indexer/HadoopDruidIndexerConfigTest.java diff --git a/indexer/src/test/java/com/metamx/druid/indexer/SortableBytesTest.java b/indexing-hadoop/src/test/java/com/metamx/druid/indexer/SortableBytesTest.java similarity index 100% rename from indexer/src/test/java/com/metamx/druid/indexer/SortableBytesTest.java rename to indexing-hadoop/src/test/java/com/metamx/druid/indexer/SortableBytesTest.java diff --git a/indexer/src/test/java/com/metamx/druid/indexer/granularity/ArbitraryGranularityTest.java b/indexing-hadoop/src/test/java/com/metamx/druid/indexer/granularity/ArbitraryGranularityTest.java similarity index 100% rename from indexer/src/test/java/com/metamx/druid/indexer/granularity/ArbitraryGranularityTest.java rename to indexing-hadoop/src/test/java/com/metamx/druid/indexer/granularity/ArbitraryGranularityTest.java diff --git a/indexer/src/test/java/com/metamx/druid/indexer/granularity/UniformGranularityTest.java b/indexing-hadoop/src/test/java/com/metamx/druid/indexer/granularity/UniformGranularityTest.java similarity index 100% rename from indexer/src/test/java/com/metamx/druid/indexer/granularity/UniformGranularityTest.java rename to indexing-hadoop/src/test/java/com/metamx/druid/indexer/granularity/UniformGranularityTest.java diff --git a/indexer/src/test/java/com/metamx/druid/indexer/hadoop/FSSpideringIteratorTest.java b/indexing-hadoop/src/test/java/com/metamx/druid/indexer/hadoop/FSSpideringIteratorTest.java similarity index 100% rename from indexer/src/test/java/com/metamx/druid/indexer/hadoop/FSSpideringIteratorTest.java rename to indexing-hadoop/src/test/java/com/metamx/druid/indexer/hadoop/FSSpideringIteratorTest.java diff --git a/merger/pom.xml b/indexing-service/pom.xml similarity index 95% rename from merger/pom.xml rename to indexing-service/pom.xml index a7bc4e60a48..e3ff3e37803 100644 --- a/merger/pom.xml +++ b/indexing-service/pom.xml @@ -21,14 +21,14 @@ 4.0.0 com.metamx.druid - druid-merger - druid-merger - druid-merger + druid-indexing-service + druid-indexing-service + druid-indexing-service com.metamx druid - 0.4.33-SNAPSHOT + 0.5.0-SNAPSHOT @@ -49,12 +49,12 @@ com.metamx.druid - druid-index-common + druid-indexing-common ${project.parent.version} com.metamx.druid - druid-indexer + druid-indexing-hadoop ${project.parent.version} diff --git a/merger/src/main/java/com/metamx/druid/merger/common/RetryPolicy.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/RetryPolicy.java similarity index 94% rename from merger/src/main/java/com/metamx/druid/merger/common/RetryPolicy.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/RetryPolicy.java index 19d66ee4522..e1089990cd0 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/RetryPolicy.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/RetryPolicy.java @@ -17,9 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common; +package com.metamx.druid.indexing.common; -import com.metamx.druid.merger.common.config.RetryPolicyConfig; +import com.metamx.druid.indexing.common.config.RetryPolicyConfig; import com.metamx.emitter.EmittingLogger; import org.joda.time.Duration; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/RetryPolicyFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/RetryPolicyFactory.java similarity index 90% rename from merger/src/main/java/com/metamx/druid/merger/common/RetryPolicyFactory.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/RetryPolicyFactory.java index ab6a30d5a86..f9dabd54b52 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/RetryPolicyFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/RetryPolicyFactory.java @@ -17,9 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common; +package com.metamx.druid.indexing.common; -import com.metamx.druid.merger.common.config.RetryPolicyConfig; +import com.metamx.druid.indexing.common.config.RetryPolicyConfig; /** */ diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskLock.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskLock.java similarity index 98% rename from merger/src/main/java/com/metamx/druid/merger/common/TaskLock.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskLock.java index 1cb82725482..29941921ebc 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskLock.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskLock.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common; +package com.metamx.druid.indexing.common; import com.google.common.base.Objects; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskStatus.java similarity index 98% rename from merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskStatus.java index ec226429dd0..8a798d2a961 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskStatus.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskStatus.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common; +package com.metamx.druid.indexing.common; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolbox.java similarity index 94% rename from merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolbox.java index 77f8c0830fb..0ee2a6e7632 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolbox.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolbox.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common; +package com.metamx.druid.indexing.common; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Maps; @@ -31,10 +31,10 @@ import com.metamx.druid.loading.S3DataSegmentPuller; import com.metamx.druid.loading.SegmentLoaderConfig; import com.metamx.druid.loading.SegmentLoadingException; import com.metamx.druid.loading.SingleSegmentLoader; -import com.metamx.druid.merger.common.actions.TaskActionClient; -import com.metamx.druid.merger.common.actions.TaskActionClientFactory; -import com.metamx.druid.merger.common.config.TaskConfig; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.actions.TaskActionClient; +import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; +import com.metamx.druid.indexing.common.config.TaskConfig; +import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.emitter.service.ServiceEmitter; import org.jets3t.service.impl.rest.httpclient.RestS3Service; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolboxFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolboxFactory.java similarity index 93% rename from merger/src/main/java/com/metamx/druid/merger/common/TaskToolboxFactory.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolboxFactory.java index 044ba541976..1d8c244ef7d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/TaskToolboxFactory.java @@ -17,16 +17,16 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common; +package com.metamx.druid.indexing.common; import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.druid.client.ServerView; import com.metamx.druid.coordination.DataSegmentAnnouncer; import com.metamx.druid.loading.DataSegmentKiller; import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.merger.common.actions.TaskActionClientFactory; -import com.metamx.druid.merger.common.config.TaskConfig; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; +import com.metamx.druid.indexing.common.config.TaskConfig; +import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.query.QueryRunnerFactoryConglomerate; import com.metamx.emitter.service.ServiceEmitter; import org.jets3t.service.impl.rest.httpclient.RestS3Service; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClient.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/LocalTaskActionClient.java similarity index 87% rename from merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClient.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/LocalTaskActionClient.java index 1083dfcbed5..ac4c41492d2 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClient.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/LocalTaskActionClient.java @@ -1,7 +1,7 @@ -package com.metamx.druid.merger.common.actions; +package com.metamx.druid.indexing.common.actions; -import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.TaskStorage; +import com.metamx.druid.indexing.common.task.Task; +import com.metamx.druid.indexing.coordinator.TaskStorage; import com.metamx.emitter.EmittingLogger; import java.io.IOException; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClientFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/LocalTaskActionClientFactory.java similarity index 88% rename from merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClientFactory.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/LocalTaskActionClientFactory.java index 89e37ceafab..0f7cdeaa748 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/LocalTaskActionClientFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/LocalTaskActionClientFactory.java @@ -17,10 +17,10 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.actions; +package com.metamx.druid.indexing.common.actions; -import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.TaskStorage; +import com.metamx.druid.indexing.common.task.Task; +import com.metamx.druid.indexing.coordinator.TaskStorage; /** */ diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockAcquireAction.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/LockAcquireAction.java similarity index 88% rename from merger/src/main/java/com/metamx/druid/merger/common/actions/LockAcquireAction.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/LockAcquireAction.java index 18441ea54c2..87a0dc8cb11 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockAcquireAction.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/LockAcquireAction.java @@ -1,12 +1,12 @@ -package com.metamx.druid.merger.common.actions; +package com.metamx.druid.indexing.common.actions; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Throwables; -import com.metamx.druid.merger.common.TaskLock; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.TaskLock; +import com.metamx.druid.indexing.common.task.Task; import org.joda.time.Interval; public class LockAcquireAction implements TaskAction diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/LockListAction.java similarity index 79% rename from merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/LockListAction.java index 4ea12ae6d13..08ce106e1b4 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockListAction.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/LockListAction.java @@ -1,8 +1,8 @@ -package com.metamx.druid.merger.common.actions; +package com.metamx.druid.indexing.common.actions; import com.fasterxml.jackson.core.type.TypeReference; -import com.metamx.druid.merger.common.TaskLock; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.TaskLock; +import com.metamx.druid.indexing.common.task.Task; import java.util.List; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockReleaseAction.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/LockReleaseAction.java similarity index 91% rename from merger/src/main/java/com/metamx/druid/merger/common/actions/LockReleaseAction.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/LockReleaseAction.java index 0ee33691104..67efd712c71 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/LockReleaseAction.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/LockReleaseAction.java @@ -1,10 +1,10 @@ -package com.metamx.druid.merger.common.actions; +package com.metamx.druid.indexing.common.actions; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.task.Task; import org.joda.time.Interval; public class LockReleaseAction implements TaskAction diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/RemoteTaskActionClient.java similarity index 94% rename from merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/RemoteTaskActionClient.java index 2b919395571..b9694ad3fba 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClient.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/RemoteTaskActionClient.java @@ -1,13 +1,13 @@ -package com.metamx.druid.merger.common.actions; +package com.metamx.druid.indexing.common.actions; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; -import com.metamx.druid.merger.common.RetryPolicy; -import com.metamx.druid.merger.common.RetryPolicyFactory; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.RetryPolicy; +import com.metamx.druid.indexing.common.RetryPolicyFactory; +import com.metamx.druid.indexing.common.task.Task; import com.metamx.http.client.HttpClient; import com.metamx.http.client.response.ToStringResponseHandler; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClientFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/RemoteTaskActionClientFactory.java similarity index 91% rename from merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClientFactory.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/RemoteTaskActionClientFactory.java index eae61d9ff81..c872a2200a4 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/RemoteTaskActionClientFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/RemoteTaskActionClientFactory.java @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.actions; +package com.metamx.druid.indexing.common.actions; import com.fasterxml.jackson.databind.ObjectMapper; -import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.common.RetryPolicyFactory; +import com.metamx.druid.indexing.common.task.Task; +import com.metamx.druid.indexing.common.RetryPolicyFactory; import com.metamx.http.client.HttpClient; import org.apache.curator.x.discovery.ServiceProvider; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentInsertAction.java similarity index 96% rename from merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentInsertAction.java index 2d2ac9ef02a..4a4dd43f67e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentInsertAction.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentInsertAction.java @@ -1,4 +1,4 @@ -package com.metamx.druid.merger.common.actions; +package com.metamx.druid.indexing.common.actions; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; @@ -7,7 +7,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.collect.ImmutableSet; import com.metamx.common.ISE; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.task.Task; import com.metamx.emitter.service.ServiceMetricEvent; import java.io.IOException; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUnusedAction.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentListUnusedAction.java similarity index 93% rename from merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUnusedAction.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentListUnusedAction.java index 90efbaf1b1c..efff8dc0a65 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUnusedAction.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentListUnusedAction.java @@ -1,11 +1,11 @@ -package com.metamx.druid.merger.common.actions; +package com.metamx.druid.indexing.common.actions; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.task.Task; import org.joda.time.Interval; import java.io.IOException; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUsedAction.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentListUsedAction.java similarity index 93% rename from merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUsedAction.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentListUsedAction.java index ca8fc3a888b..2d86b393f0c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentListUsedAction.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentListUsedAction.java @@ -1,11 +1,11 @@ -package com.metamx.druid.merger.common.actions; +package com.metamx.druid.indexing.common.actions; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.task.Task; import org.joda.time.Interval; import java.io.IOException; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentNukeAction.java similarity index 94% rename from merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentNukeAction.java index 3d4f453a670..d2fdc5be392 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SegmentNukeAction.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SegmentNukeAction.java @@ -1,4 +1,4 @@ -package com.metamx.druid.merger.common.actions; +package com.metamx.druid.indexing.common.actions; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; @@ -7,7 +7,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.collect.ImmutableSet; import com.metamx.common.ISE; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.task.Task; import com.metamx.emitter.service.ServiceMetricEvent; import java.io.IOException; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/SpawnTasksAction.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SpawnTasksAction.java similarity index 91% rename from merger/src/main/java/com/metamx/druid/merger/common/actions/SpawnTasksAction.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SpawnTasksAction.java index 870a15ba25f..1a60e0b6658 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/SpawnTasksAction.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/SpawnTasksAction.java @@ -1,11 +1,11 @@ -package com.metamx.druid.merger.common.actions; +package com.metamx.druid.indexing.common.actions; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.collect.ImmutableList; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.task.Task; import java.util.List; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskAction.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/TaskAction.java similarity index 91% rename from merger/src/main/java/com/metamx/druid/merger/common/actions/TaskAction.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/TaskAction.java index b3ac3b72d5d..c7f4797d1b8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskAction.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/TaskAction.java @@ -1,6 +1,6 @@ -package com.metamx.druid.merger.common.actions; +package com.metamx.druid.indexing.common.actions; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.task.Task; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.core.type.TypeReference; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClient.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/TaskActionClient.java similarity index 75% rename from merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClient.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/TaskActionClient.java index 1f0366c6a56..4c080b25374 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClient.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/TaskActionClient.java @@ -1,4 +1,4 @@ -package com.metamx.druid.merger.common.actions; +package com.metamx.druid.indexing.common.actions; import java.io.IOException; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClientFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/TaskActionClientFactory.java similarity index 90% rename from merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClientFactory.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/TaskActionClientFactory.java index 2784a442f31..a04ecc0aeb1 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionClientFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/TaskActionClientFactory.java @@ -17,9 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.actions; +package com.metamx.druid.indexing.common.actions; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.task.Task; /** */ diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionHolder.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/TaskActionHolder.java similarity index 93% rename from merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionHolder.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/TaskActionHolder.java index a440447a226..80b2c71060c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionHolder.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/TaskActionHolder.java @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.actions; +package com.metamx.druid.indexing.common.actions; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.task.Task; /** */ diff --git a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionToolbox.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/TaskActionToolbox.java similarity index 87% rename from merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionToolbox.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/TaskActionToolbox.java index 0ba45777305..81f8131c77b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/actions/TaskActionToolbox.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/actions/TaskActionToolbox.java @@ -1,13 +1,13 @@ -package com.metamx.druid.merger.common.actions; +package com.metamx.druid.indexing.common.actions; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.merger.common.TaskLock; -import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.MergerDBCoordinator; -import com.metamx.druid.merger.coordinator.TaskLockbox; -import com.metamx.druid.merger.coordinator.TaskQueue; +import com.metamx.druid.indexing.common.TaskLock; +import com.metamx.druid.indexing.common.task.Task; +import com.metamx.druid.indexing.coordinator.MergerDBCoordinator; +import com.metamx.druid.indexing.coordinator.TaskLockbox; +import com.metamx.druid.indexing.coordinator.TaskQueue; import com.metamx.emitter.service.ServiceEmitter; import java.util.List; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/config/IndexerZkConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/IndexerZkConfig.java similarity index 95% rename from merger/src/main/java/com/metamx/druid/merger/common/config/IndexerZkConfig.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/config/IndexerZkConfig.java index d85f53e4350..39210ceae85 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/config/IndexerZkConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/IndexerZkConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.config; +package com.metamx.druid.indexing.common.config; import com.metamx.druid.initialization.ZkPathsConfig; import org.skife.config.Config; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/config/RetryPolicyConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/RetryPolicyConfig.java similarity index 96% rename from merger/src/main/java/com/metamx/druid/merger/common/config/RetryPolicyConfig.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/config/RetryPolicyConfig.java index 1086c5ec2cd..b9ad26ad2d4 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/config/RetryPolicyConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/RetryPolicyConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.config; +package com.metamx.druid.indexing.common.config; import org.joda.time.Duration; import org.skife.config.Config; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskConfig.java similarity index 86% rename from merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskConfig.java index 2cdef82e9fb..9de8bcc9302 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/config/TaskConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.config; +package com.metamx.druid.indexing.common.config; import org.skife.config.Config; import org.skife.config.Default; @@ -26,13 +26,13 @@ import java.io.File; public abstract class TaskConfig { - @Config("druid.merger.taskDir") + @Config("druid.indexer.taskDir") public abstract File getBaseTaskDir(); - @Config("druid.merger.rowFlushBoundary") + @Config("druid.indexer.rowFlushBoundary") @Default("500000") public abstract int getDefaultRowFlushBoundary(); - @Config("druid.merger.hadoopWorkingPath") + @Config("druid.indexer.hadoopWorkingPath") public abstract String getHadoopWorkingPath(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/config/TaskLogConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskLogConfig.java similarity index 64% rename from merger/src/main/java/com/metamx/druid/merger/common/config/TaskLogConfig.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskLogConfig.java index 92a6ef0d30a..a80fdcd76c8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/config/TaskLogConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/config/TaskLogConfig.java @@ -1,15 +1,15 @@ -package com.metamx.druid.merger.common.config; +package com.metamx.druid.indexing.common.config; import org.skife.config.Config; import org.skife.config.DefaultNull; public abstract class TaskLogConfig { - @Config("druid.merger.logs.s3bucket") + @Config("druid.indexer.logs.s3bucket") @DefaultNull public abstract String getLogStorageBucket(); - @Config("druid.merger.logs.s3prefix") + @Config("druid.indexer.logs.s3prefix") @DefaultNull public abstract String getLogStoragePrefix(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/index/ChatHandler.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandler.java similarity index 87% rename from merger/src/main/java/com/metamx/druid/merger/common/index/ChatHandler.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandler.java index 28185e066d6..3458c931de4 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/index/ChatHandler.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandler.java @@ -1,4 +1,4 @@ -package com.metamx.druid.merger.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 diff --git a/merger/src/main/java/com/metamx/druid/merger/common/index/ChatHandlerProvider.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandlerProvider.java similarity index 94% rename from merger/src/main/java/com/metamx/druid/merger/common/index/ChatHandlerProvider.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandlerProvider.java index 53ce944f92f..6d67a7a5fb1 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/index/ChatHandlerProvider.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/ChatHandlerProvider.java @@ -1,11 +1,11 @@ -package com.metamx.druid.merger.common.index; +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.merger.worker.config.ChatHandlerProviderConfig; +import com.metamx.druid.indexing.worker.config.ChatHandlerProviderConfig; import java.util.concurrent.ConcurrentMap; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/index/EventReceiver.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceiver.java similarity index 75% rename from merger/src/main/java/com/metamx/druid/merger/common/index/EventReceiver.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceiver.java index f2339f54c88..66333beaeaa 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/index/EventReceiver.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceiver.java @@ -1,4 +1,4 @@ -package com.metamx.druid.merger.common.index; +package com.metamx.druid.indexing.common.index; import java.util.Collection; import java.util.Map; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/index/EventReceiverFirehoseFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceiverFirehoseFactory.java similarity index 99% rename from merger/src/main/java/com/metamx/druid/merger/common/index/EventReceiverFirehoseFactory.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceiverFirehoseFactory.java index 6372cc467a2..5c437eaa356 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/index/EventReceiverFirehoseFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/EventReceiverFirehoseFactory.java @@ -1,4 +1,4 @@ -package com.metamx.druid.merger.common.index; +package com.metamx.druid.indexing.common.index; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/index/StaticS3FirehoseFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/StaticS3FirehoseFactory.java similarity index 99% rename from merger/src/main/java/com/metamx/druid/merger/common/index/StaticS3FirehoseFactory.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/index/StaticS3FirehoseFactory.java index c369a5e3853..b7289c017b8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/index/StaticS3FirehoseFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/StaticS3FirehoseFactory.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.index; +package com.metamx.druid.indexing.common.index; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/YeOldePlumberSchool.java similarity index 99% rename from merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/index/YeOldePlumberSchool.java index 92d5d06c145..a3c49057c85 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/index/YeOldePlumberSchool.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/index/YeOldePlumberSchool.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.index; +package com.metamx.druid.indexing.common.index; import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/AbstractTask.java similarity index 80% rename from merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/task/AbstractTask.java index 0ab75140303..07d11582982 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AbstractTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/AbstractTask.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.task; +package com.metamx.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; @@ -26,9 +26,9 @@ import com.google.common.base.Objects; import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.metamx.druid.Query; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.actions.SegmentListUsedAction; -import com.metamx.druid.merger.common.actions.TaskActionClient; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.actions.SegmentListUsedAction; +import com.metamx.druid.indexing.common.actions.TaskActionClient; import com.metamx.druid.query.QueryRunner; import org.joda.time.Interval; @@ -42,6 +42,9 @@ public abstract class AbstractTask implements Task @JsonIgnore private final String groupId; + @JsonIgnore + private final String availabilityGroup; + @JsonIgnore private final String dataSource; @@ -50,13 +53,23 @@ public abstract class AbstractTask implements Task protected AbstractTask(String id, String dataSource, Interval interval) { - this(id, id, dataSource, interval); + this(id, id, id, dataSource, interval); } protected AbstractTask(String id, String groupId, String dataSource, Interval interval) { this.id = Preconditions.checkNotNull(id, "id"); this.groupId = Preconditions.checkNotNull(groupId, "groupId"); + this.availabilityGroup = id; + this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); + this.interval = Optional.fromNullable(interval); + } + + protected AbstractTask(String id, String groupId, String availabilityGroup, String dataSource, Interval interval) + { + this.id = Preconditions.checkNotNull(id, "id"); + this.groupId = Preconditions.checkNotNull(groupId, "groupId"); + this.availabilityGroup = Preconditions.checkNotNull(availabilityGroup, "availabilityGroup"); this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); this.interval = Optional.fromNullable(interval); } @@ -75,6 +88,13 @@ public abstract class AbstractTask implements Task return groupId; } + @JsonProperty + @Override + public String getAvailabilityGroup() + { + return availabilityGroup; + } + @Override public String getNodeType() { diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/AppendTask.java similarity index 98% rename from merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/task/AppendTask.java index b00c1c24399..9f38c754809 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/AppendTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/AppendTask.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.task; +package com.metamx.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/DeleteTask.java similarity index 91% rename from merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/task/DeleteTask.java index 594e98b9f63..7c9f8d53801 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/DeleteTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/DeleteTask.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.task; +package com.metamx.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -33,11 +33,11 @@ import com.metamx.druid.index.v1.IncrementalIndex; import com.metamx.druid.index.v1.IncrementalIndexAdapter; import com.metamx.druid.index.v1.IndexMerger; import com.metamx.druid.index.v1.IndexableAdapter; -import com.metamx.druid.merger.common.TaskLock; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.LockListAction; -import com.metamx.druid.merger.common.actions.SegmentInsertAction; +import com.metamx.druid.indexing.common.TaskLock; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.TaskToolbox; +import com.metamx.druid.indexing.common.actions.LockListAction; +import com.metamx.druid.indexing.common.actions.SegmentInsertAction; import com.metamx.druid.shard.NoneShardSpec; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/HadoopIndexTask.java similarity index 94% rename from merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/task/HadoopIndexTask.java index f3ce30c90cb..0ce0b4c382c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/HadoopIndexTask.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.task; +package com.metamx.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; @@ -30,11 +30,11 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.indexer.HadoopDruidIndexerConfig; import com.metamx.druid.indexer.HadoopDruidIndexerJob; import com.metamx.druid.loading.S3DataSegmentPusher; -import com.metamx.druid.merger.common.TaskLock; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.LockListAction; -import com.metamx.druid.merger.common.actions.SegmentInsertAction; +import com.metamx.druid.indexing.common.TaskLock; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.TaskToolbox; +import com.metamx.druid.indexing.common.actions.LockListAction; +import com.metamx.druid.indexing.common.actions.SegmentInsertAction; import com.metamx.druid.utils.JodaUtils; import org.joda.time.DateTime; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexDeterminePartitionsTask.java similarity index 93% rename from merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexDeterminePartitionsTask.java index cbe6127de07..7c7d8707a37 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexDeterminePartitionsTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexDeterminePartitionsTask.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.task; +package com.metamx.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; @@ -32,15 +32,16 @@ import com.google.common.collect.TreeMultiset; import com.google.common.primitives.Ints; import com.metamx.common.logger.Logger; import com.metamx.druid.input.InputRow; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.SpawnTasksAction; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.TaskToolbox; +import com.metamx.druid.indexing.common.actions.SpawnTasksAction; import com.metamx.druid.realtime.firehose.Firehose; import com.metamx.druid.realtime.firehose.FirehoseFactory; import com.metamx.druid.realtime.Schema; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.druid.shard.ShardSpec; import com.metamx.druid.shard.SingleDimensionShardSpec; +import org.joda.time.DateTime; import org.joda.time.Interval; import java.util.List; @@ -49,6 +50,16 @@ import java.util.Set; public class IndexDeterminePartitionsTask extends AbstractTask { + private static String makeTaskId(String groupId, DateTime start, DateTime end) + { + return String.format( + "%s_partitions_%s_%s", + groupId, + start, + end + ); + } + @JsonIgnore private final FirehoseFactory firehoseFactory; @@ -75,13 +86,9 @@ public class IndexDeterminePartitionsTask extends AbstractTask ) { super( - id != null ? id : String.format( - "%s_partitions_%s_%s", - groupId, - interval.getStart(), - interval.getEnd() - ), + id != null ? id : makeTaskId(groupId, interval.getStart(), interval.getEnd()), groupId, + makeTaskId(groupId, interval.getStart(), interval.getEnd()), schema.getDataSource(), Preconditions.checkNotNull(interval, "interval") ); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexGeneratorTask.java similarity index 89% rename from merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexGeneratorTask.java index 5362a21855f..1efa2b5951c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexGeneratorTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexGeneratorTask.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.task; +package com.metamx.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; @@ -30,12 +30,12 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.input.InputRow; import com.metamx.druid.loading.DataSegmentPusher; -import com.metamx.druid.merger.common.TaskLock; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.LockListAction; -import com.metamx.druid.merger.common.actions.SegmentInsertAction; -import com.metamx.druid.merger.common.index.YeOldePlumberSchool; +import com.metamx.druid.indexing.common.TaskLock; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.TaskToolbox; +import com.metamx.druid.indexing.common.actions.LockListAction; +import com.metamx.druid.indexing.common.actions.SegmentInsertAction; +import com.metamx.druid.indexing.common.index.YeOldePlumberSchool; import com.metamx.druid.realtime.FireDepartmentMetrics; import com.metamx.druid.realtime.firehose.Firehose; import com.metamx.druid.realtime.firehose.FirehoseFactory; @@ -75,7 +75,9 @@ public class IndexGeneratorTask extends AbstractTask ) { super( - id != null ? id : String.format( + id != null + ? id + : String.format( "%s_generator_%s_%s_%s", groupId, interval.getStart(), @@ -149,10 +151,10 @@ public class IndexGeneratorTask extends AbstractTask : toolbox.getConfig().getDefaultRowFlushBoundary(); try { - while(firehose.hasMore()) { + while (firehose.hasMore()) { final InputRow inputRow = firehose.nextRow(); - if(shouldIndex(inputRow)) { + if (shouldIndex(inputRow)) { final Sink sink = plumber.getSink(inputRow.getTimestampFromEpoch()); if (sink == null) { throw new NullPointerException( @@ -166,14 +168,15 @@ public class IndexGeneratorTask extends AbstractTask int numRows = sink.add(inputRow); metrics.incrementProcessed(); - if(numRows >= myRowFlushBoundary) { + if (numRows >= myRowFlushBoundary) { plumber.persist(firehose.commit()); } } else { metrics.incrementThrownAway(); } } - } finally { + } + finally { firehose.close(); } @@ -200,18 +203,21 @@ public class IndexGeneratorTask extends AbstractTask /** * Should we index this inputRow? Decision is based on our interval and shardSpec. + * * @param inputRow the row to check + * * @return true or false */ - private boolean shouldIndex(InputRow inputRow) { - if(!getImplicitLockInterval().get().contains(inputRow.getTimestampFromEpoch())) { + private boolean shouldIndex(InputRow inputRow) + { + if (!getImplicitLockInterval().get().contains(inputRow.getTimestampFromEpoch())) { return false; } final Map eventDimensions = Maps.newHashMapWithExpectedSize(inputRow.getDimensions().size()); - for(final String dim : inputRow.getDimensions()) { + for (final String dim : inputRow.getDimensions()) { final List dimValues = inputRow.getDimension(dim); - if(dimValues.size() == 1) { + if (dimValues.size() == 1) { eventDimensions.put(dim, Iterables.getOnlyElement(dimValues)); } } diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexTask.java similarity index 94% rename from merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexTask.java index 49eb9e41d97..8c584040e86 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/IndexTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/IndexTask.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.task; +package com.metamx.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; @@ -29,10 +29,11 @@ import com.metamx.druid.QueryGranularity; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.index.v1.SpatialDimensionSchema; import com.metamx.druid.indexer.granularity.GranularitySpec; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.SpawnTasksAction; -import com.metamx.druid.merger.common.actions.TaskActionClient; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.TaskToolbox; +import com.metamx.druid.indexing.common.actions.SpawnTasksAction; +import com.metamx.druid.indexing.common.actions.TaskActionClient; +import com.metamx.druid.realtime.firehose.FirehoseFactory; import com.metamx.druid.realtime.Schema; import com.metamx.druid.realtime.firehose.FirehoseFactory; import com.metamx.druid.shard.NoneShardSpec; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/KillTask.java similarity index 89% rename from merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/task/KillTask.java index e26a25fd038..2b5784f0081 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/KillTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/KillTask.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.task; +package com.metamx.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -26,12 +26,12 @@ import com.google.common.collect.Iterables; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.merger.common.TaskLock; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.LockListAction; -import com.metamx.druid.merger.common.actions.SegmentListUnusedAction; -import com.metamx.druid.merger.common.actions.SegmentNukeAction; +import com.metamx.druid.indexing.common.TaskLock; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.TaskToolbox; +import com.metamx.druid.indexing.common.actions.LockListAction; +import com.metamx.druid.indexing.common.actions.SegmentListUnusedAction; +import com.metamx.druid.indexing.common.actions.SegmentNukeAction; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/MergeTask.java similarity index 98% rename from merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/task/MergeTask.java index e0f3dfde649..95e79c37f25 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/MergeTask.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.task; +package com.metamx.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTaskBase.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/MergeTaskBase.java similarity index 96% rename from merger/src/main/java/com/metamx/druid/merger/common/task/MergeTaskBase.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/task/MergeTaskBase.java index 33aae0b3dff..68da16fb0ef 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/MergeTaskBase.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/MergeTaskBase.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.task; +package com.metamx.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; @@ -37,12 +37,12 @@ import com.google.common.hash.Hashing; import com.metamx.common.ISE; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IndexIO; -import com.metamx.druid.merger.common.TaskLock; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.LockListAction; -import com.metamx.druid.merger.common.actions.SegmentInsertAction; -import com.metamx.druid.merger.common.actions.TaskActionClient; +import com.metamx.druid.indexing.common.TaskLock; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.TaskToolbox; +import com.metamx.druid.indexing.common.actions.LockListAction; +import com.metamx.druid.indexing.common.actions.SegmentInsertAction; +import com.metamx.druid.indexing.common.actions.TaskActionClient; import com.metamx.druid.shard.NoneShardSpec; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java similarity index 92% rename from merger/src/main/java/com/metamx/druid/merger/common/task/RealtimeIndexTask.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java index ee803e2aef1..d6f775a2611 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/RealtimeIndexTask.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.task; +package com.metamx.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; @@ -31,13 +31,13 @@ import com.metamx.druid.client.DataSegment; import com.metamx.druid.coordination.DataSegmentAnnouncer; import com.metamx.druid.index.v1.IndexGranularity; import com.metamx.druid.input.InputRow; -import com.metamx.druid.merger.common.TaskLock; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.LockAcquireAction; -import com.metamx.druid.merger.common.actions.LockListAction; -import com.metamx.druid.merger.common.actions.LockReleaseAction; -import com.metamx.druid.merger.common.actions.SegmentInsertAction; +import com.metamx.druid.indexing.common.TaskLock; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.TaskToolbox; +import com.metamx.druid.indexing.common.actions.LockAcquireAction; +import com.metamx.druid.indexing.common.actions.LockListAction; +import com.metamx.druid.indexing.common.actions.LockReleaseAction; +import com.metamx.druid.indexing.common.actions.SegmentInsertAction; import com.metamx.druid.query.FinalizeResultsQueryRunner; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryRunnerFactory; @@ -64,6 +64,14 @@ public class RealtimeIndexTask extends AbstractTask { private static final EmittingLogger log = new EmittingLogger(RealtimeIndexTask.class); + private static String makeTaskId(String dataSource, int partitionNum, String version) + { + return String.format( + "index_realtime_%s_%d_%s", + dataSource, partitionNum, version + ); + } + @JsonIgnore private final Schema schema; @@ -97,6 +105,7 @@ public class RealtimeIndexTask extends AbstractTask @JsonCreator public RealtimeIndexTask( @JsonProperty("id") String id, + @JsonProperty("availabilityGroup") String availabilityGroup, @JsonProperty("schema") Schema schema, @JsonProperty("firehose") FirehoseFactory firehoseFactory, @JsonProperty("fireDepartmentConfig") FireDepartmentConfig fireDepartmentConfig, @@ -105,14 +114,16 @@ public class RealtimeIndexTask extends AbstractTask ) { super( - id != null ? id : String.format( - "index_realtime_%s_%d_%s", - schema.getDataSource(), schema.getShardSpec().getPartitionNum(), new DateTime() - ), + id != null + ? id + : makeTaskId(schema.getDataSource(), schema.getShardSpec().getPartitionNum(), new DateTime().toString()), String.format( "index_realtime_%s", schema.getDataSource() ), + availabilityGroup != null + ? availabilityGroup + : makeTaskId(schema.getDataSource(), schema.getShardSpec().getPartitionNum(), new DateTime().toString()), schema.getDataSource(), null ); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java similarity index 89% rename from merger/src/main/java/com/metamx/druid/merger/common/task/Task.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java index 71ae502c1d7..6272669cda5 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/Task.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/Task.java @@ -17,15 +17,15 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.task; +package com.metamx.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.base.Optional; import com.metamx.druid.Query; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.TaskActionClient; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.TaskToolbox; +import com.metamx.druid.indexing.common.actions.TaskActionClient; import com.metamx.druid.query.QueryRunner; import org.joda.time.Interval; @@ -38,8 +38,8 @@ import org.joda.time.Interval; *
  • Tasks are each part of a "task group", which is a set of tasks that can share interval locks. These are * useful for producing sharded segments.
  • *
  • Tasks can optionally have an "implicit lock interval". Tasks with this property are guaranteed to have - * a lock on that interval during their {@link #preflight(com.metamx.druid.merger.common.actions.TaskActionClient)} - * and {@link #run(com.metamx.druid.merger.common.TaskToolbox)} methods.
  • + * a lock on that interval during their {@link #preflight(com.metamx.druid.indexing.common.actions.TaskActionClient)} + * and {@link #run(com.metamx.druid.indexing.common.TaskToolbox)} methods. *
  • Tasks do not need to explicitly release locks; they are released upon task completion. Tasks may choose * to release locks early if they desire.
  • * @@ -71,6 +71,13 @@ public interface Task */ public String getGroupId(); + /** + * Returns availability group ID of this task. Tasks the same availability group cannot be assigned to the same + * worker. If tasks do not have this restriction, a common convention is to set the availability group ID to the + * task ID. + */ + public String getAvailabilityGroup(); + /** * Returns a descriptive label for this task type. Used for metrics emission and logging. */ diff --git a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/VersionConverterTask.java similarity index 91% rename from merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/task/VersionConverterTask.java index 8cf2ede533a..27e46f9ce42 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/task/VersionConverterTask.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/task/VersionConverterTask.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.task; +package com.metamx.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; @@ -31,12 +31,12 @@ import com.metamx.common.logger.Logger; import com.metamx.druid.client.DataSegment; import com.metamx.druid.index.v1.IndexIO; import com.metamx.druid.loading.SegmentLoadingException; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.actions.SegmentInsertAction; -import com.metamx.druid.merger.common.actions.SegmentListUsedAction; -import com.metamx.druid.merger.common.actions.SpawnTasksAction; -import com.metamx.druid.merger.common.actions.TaskActionClient; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.TaskToolbox; +import com.metamx.druid.indexing.common.actions.SegmentInsertAction; +import com.metamx.druid.indexing.common.actions.SegmentListUsedAction; +import com.metamx.druid.indexing.common.actions.SpawnTasksAction; +import com.metamx.druid.indexing.common.actions.TaskActionClient; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -89,8 +89,7 @@ public class VersionConverterTask extends AbstractTask if (id == null) { if (segment == null) { return create(dataSource, interval); - } - else { + } else { return create(segment); } } @@ -105,7 +104,7 @@ public class VersionConverterTask extends AbstractTask DataSegment segment ) { - super(id, groupId, dataSource, interval); + super(id, groupId, id, dataSource, interval); this.segment = segment; } @@ -206,6 +205,13 @@ public class VersionConverterTask extends AbstractTask segment.getShardSpec().getPartitionNum() ), groupId, + joinId( + groupId, + "sub", + segment.getInterval().getStart(), + segment.getInterval().getEnd(), + segment.getShardSpec().getPartitionNum() + ), segment.getDataSource(), segment.getInterval() ); diff --git a/merger/src/main/java/com/metamx/druid/merger/common/tasklogs/NoopTaskLogs.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/NoopTaskLogs.java similarity index 92% rename from merger/src/main/java/com/metamx/druid/merger/common/tasklogs/NoopTaskLogs.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/NoopTaskLogs.java index d4cd77c0bef..feb3f9f4a4f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/tasklogs/NoopTaskLogs.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/NoopTaskLogs.java @@ -1,4 +1,4 @@ -package com.metamx.druid.merger.common.tasklogs; +package com.metamx.druid.indexing.common.tasklogs; import com.google.common.base.Optional; import com.google.common.io.InputSupplier; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/tasklogs/S3TaskLogs.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/S3TaskLogs.java similarity index 98% rename from merger/src/main/java/com/metamx/druid/merger/common/tasklogs/S3TaskLogs.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/S3TaskLogs.java index fb450ac9ac3..a09f2ef7e00 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/tasklogs/S3TaskLogs.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/S3TaskLogs.java @@ -1,4 +1,4 @@ -package com.metamx.druid.merger.common.tasklogs; +package com.metamx.druid.indexing.common.tasklogs; import com.google.common.base.Optional; import com.google.common.base.Preconditions; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/tasklogs/SwitchingTaskLogProvider.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/SwitchingTaskLogProvider.java similarity index 94% rename from merger/src/main/java/com/metamx/druid/merger/common/tasklogs/SwitchingTaskLogProvider.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/SwitchingTaskLogProvider.java index a114066c7e8..4acad86386b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/tasklogs/SwitchingTaskLogProvider.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/SwitchingTaskLogProvider.java @@ -1,4 +1,4 @@ -package com.metamx.druid.merger.common.tasklogs; +package com.metamx.druid.indexing.common.tasklogs; import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/tasklogs/TaskLogProvider.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskLogProvider.java similarity index 93% rename from merger/src/main/java/com/metamx/druid/merger/common/tasklogs/TaskLogProvider.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskLogProvider.java index 0ec3865d07b..46b1bfc2b9c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/tasklogs/TaskLogProvider.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskLogProvider.java @@ -1,4 +1,4 @@ -package com.metamx.druid.merger.common.tasklogs; +package com.metamx.druid.indexing.common.tasklogs; import com.google.common.base.Optional; import com.google.common.io.InputSupplier; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/tasklogs/TaskLogPusher.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskLogPusher.java similarity index 83% rename from merger/src/main/java/com/metamx/druid/merger/common/tasklogs/TaskLogPusher.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskLogPusher.java index f78a6ee0c99..a6354c6fd06 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/tasklogs/TaskLogPusher.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskLogPusher.java @@ -1,4 +1,4 @@ -package com.metamx.druid.merger.common.tasklogs; +package com.metamx.druid.indexing.common.tasklogs; import java.io.File; import java.io.IOException; diff --git a/merger/src/main/java/com/metamx/druid/merger/common/tasklogs/TaskLogs.java b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskLogs.java similarity index 57% rename from merger/src/main/java/com/metamx/druid/merger/common/tasklogs/TaskLogs.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskLogs.java index 84b4459dc97..0c1994d6073 100644 --- a/merger/src/main/java/com/metamx/druid/merger/common/tasklogs/TaskLogs.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/common/tasklogs/TaskLogs.java @@ -1,4 +1,4 @@ -package com.metamx.druid.merger.common.tasklogs; +package com.metamx.druid.indexing.common.tasklogs; public interface TaskLogs extends TaskLogProvider, TaskLogPusher { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/DbTaskStorage.java similarity index 97% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/DbTaskStorage.java index 4e13adfdd55..a647770b33b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/DbTaskStorage.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/DbTaskStorage.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator; +package com.metamx.druid.indexing.coordinator; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; @@ -28,11 +28,11 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.actions.TaskAction; -import com.metamx.druid.merger.common.TaskLock; -import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.actions.TaskAction; +import com.metamx.druid.indexing.common.TaskLock; +import com.metamx.druid.indexing.common.task.Task; +import com.metamx.druid.indexing.coordinator.config.IndexerDbConnectorConfig; import com.metamx.emitter.EmittingLogger; import org.joda.time.DateTime; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/ForkingTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java similarity index 94% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/ForkingTaskRunner.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java index 7071c7e50f0..1b9153432ee 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/ForkingTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ForkingTaskRunner.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator; +package com.metamx.druid.indexing.coordinator; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.CharMatcher; @@ -39,12 +39,12 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.metamx.common.ISE; import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.common.tasklogs.TaskLogProvider; -import com.metamx.druid.merger.common.tasklogs.TaskLogPusher; -import com.metamx.druid.merger.coordinator.config.ForkingTaskRunnerConfig; -import com.metamx.druid.merger.worker.executor.ExecutorMain; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.task.Task; +import com.metamx.druid.indexing.common.tasklogs.TaskLogProvider; +import com.metamx.druid.indexing.common.tasklogs.TaskLogPusher; +import com.metamx.druid.indexing.coordinator.config.ForkingTaskRunnerConfig; +import com.metamx.druid.indexing.worker.executor.ExecutorMain; import com.metamx.emitter.EmittingLogger; import org.apache.commons.io.FileUtils; @@ -154,6 +154,17 @@ public class ForkingTaskRunner implements TaskRunner, TaskLogProvider .split(config.getJavaOptions()) ); + for (String propName : props.stringPropertyNames()) { + command.add( + String.format( + "-D%s=%s", + propName, + props.getProperty(propName) + ) + ); + } + + // Override child JVM specific properties for (String propName : props.stringPropertyNames()) { if (propName.startsWith(CHILD_PROPERTY_PREFIX)) { command.add( diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/HeapMemoryTaskStorage.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/HeapMemoryTaskStorage.java similarity index 95% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/HeapMemoryTaskStorage.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/HeapMemoryTaskStorage.java index 895804bc7fd..75fa7919d72 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/HeapMemoryTaskStorage.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/HeapMemoryTaskStorage.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator; +package com.metamx.druid.indexing.coordinator; import com.google.common.base.Optional; import com.google.common.base.Preconditions; @@ -27,10 +27,10 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; import com.google.common.collect.Multimap; import com.metamx.common.logger.Logger; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.actions.TaskAction; -import com.metamx.druid.merger.common.TaskLock; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.actions.TaskAction; +import com.metamx.druid.indexing.common.TaskLock; +import com.metamx.druid.indexing.common.task.Task; import java.util.List; import java.util.Map; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/MergerDBCoordinator.java similarity index 99% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/MergerDBCoordinator.java index d2a63bad26d..1b327f40a13 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/MergerDBCoordinator.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/MergerDBCoordinator.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator; +package com.metamx.druid.indexing.coordinator; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java similarity index 90% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java index 2e094bf8964..08157d2ab43 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/RemoteTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunner.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator; +package com.metamx.druid.indexing.coordinator; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Charsets; @@ -38,14 +38,14 @@ import com.metamx.common.ISE; import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.druid.merger.common.RetryPolicy; -import com.metamx.druid.merger.common.RetryPolicyFactory; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.tasklogs.TaskLogProvider; -import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; -import com.metamx.druid.merger.worker.Worker; +import com.metamx.druid.indexing.common.RetryPolicy; +import com.metamx.druid.indexing.common.RetryPolicyFactory; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.task.Task; +import com.metamx.druid.indexing.common.tasklogs.TaskLogProvider; +import com.metamx.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; +import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; +import com.metamx.druid.indexing.worker.Worker; import com.metamx.emitter.EmittingLogger; import com.metamx.http.client.HttpClient; import com.metamx.http.client.response.InputStreamResponseHandler; @@ -70,6 +70,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -85,7 +86,7 @@ import java.util.concurrent.atomic.AtomicReference; *

    * The RemoteTaskRunner will assign tasks to a node until the node hits capacity. At that point, task assignment will * fail. The RemoteTaskRunner depends on another component to create additional worker resources. - * For example, {@link com.metamx.druid.merger.coordinator.scaling.ResourceManagementScheduler} can take care of these duties. + * For example, {@link com.metamx.druid.indexing.coordinator.scaling.ResourceManagementScheduler} can take care of these duties. *

    * If a worker node becomes inexplicably disconnected from Zk, the RemoteTaskRunner will automatically retry any tasks * that were associated with the node. @@ -113,6 +114,8 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider private final TaskRunnerWorkQueue runningTasks = new TaskRunnerWorkQueue(); // tasks that have not yet run private final TaskRunnerWorkQueue pendingTasks = new TaskRunnerWorkQueue(); + // idempotent task retry + private final Set tasksToRetry = new ConcurrentSkipListSet(); private final ExecutorService runPendingTasksExec = Executors.newSingleThreadExecutor(); @@ -161,14 +164,14 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider event.getData().getData(), Worker.class ); - log.info("New worker[%s] found!", worker.getHost()); + log.info("Worker[%s] reportin' for duty!", worker.getHost()); addWorker(worker); } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED)) { final Worker worker = jsonMapper.readValue( event.getData().getData(), Worker.class ); - log.info("Worker[%s] removed!", worker.getHost()); + log.info("Kaboom! Worker[%s] removed!", worker.getHost()); removeWorker(worker); } } @@ -349,7 +352,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider Preconditions.checkArgument(path.startsWith("/"), "path must start with '/': %s", path); try { - return new URL(String.format("http://%s/mmx/worker/v1%s", worker.getHost(), path)); + return new URL(String.format("http://%s/druid/worker/v1%s", worker.getHost(), path)); } catch (MalformedURLException e) { throw Throwables.propagate(e); @@ -402,14 +405,19 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider /** * Retries a task by inserting it back into the pending queue after a given delay. - * This method will also clean up any status paths that were associated with the task. * * @param taskRunnerWorkItem - the task to retry - * @param workerId - the worker that was previously running this task */ - private void retryTask(final TaskRunnerWorkItem taskRunnerWorkItem, final String workerId) + 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( @@ -419,6 +427,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider public void run() { runningTasks.remove(taskId); + tasksToRetry.remove(taskId); addPendingTask(taskRunnerWorkItem); } }, @@ -469,7 +478,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider log.info("Task %s switched from pending to running", taskId); } else { // Nothing running this task, announce it in ZK for a worker to run it - zkWorker = findWorkerForTask(); + zkWorker = findWorkerForTask(taskRunnerWorkItem.getTask()); if (zkWorker != null) { announceTask(zkWorker.getWorker(), taskRunnerWorkItem); } @@ -518,14 +527,14 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider synchronized (statusLock) { while (!isWorkerRunningTask(theWorker.getHost(), task.getId())) { statusLock.wait(config.getTaskAssignmentTimeoutDuration().getMillis()); - if (timeoutStopwatch.elapsedMillis() >= config.getTaskAssignmentTimeoutDuration().getMillis()) { + if (timeoutStopwatch.elapsed(TimeUnit.MILLISECONDS) >= config.getTaskAssignmentTimeoutDuration().getMillis()) { log.error( "Something went wrong! %s never ran task %s after %s!", theWorker.getHost(), task.getId(), config.getTaskAssignmentTimeoutDuration() ); - retryTask(runningTasks.get(task.getId()), theWorker.getHost()); + retryTask(runningTasks.get(task.getId())); break; } } @@ -568,7 +577,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider // This can fail if a worker writes a bogus status. Retry if so. if (!taskStatus.getId().equals(taskId)) { - retryTask(runningTasks.get(taskId), worker.getHost()); + retryTask(runningTasks.get(taskId)); return; } @@ -579,6 +588,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider taskId ); + // Synchronizing state with ZK statusLock.notify(); @@ -606,9 +616,10 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider } } else if (event.getType().equals(PathChildrenCacheEvent.Type.CHILD_REMOVED)) { final String taskId = ZKPaths.getNodeFromPath(event.getData().getPath()); - if (runningTasks.containsKey(taskId)) { + TaskRunnerWorkItem taskRunnerWorkItem = runningTasks.get(taskId); + if (taskRunnerWorkItem != null) { log.info("Task %s just disappeared!", taskId); - retryTask(runningTasks.get(taskId), worker.getHost()); + retryTask(taskRunnerWorkItem); } } } @@ -644,9 +655,15 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider ZkWorker zkWorker = zkWorkers.get(worker.getHost()); if (zkWorker != null) { try { - List tasksToRetry = cf.getChildren() - .forPath(JOINER.join(config.getIndexerTaskPath(), worker.getHost())); - log.info("%s has %d pending tasks to retry", worker.getHost(), tasksToRetry.size()); + Set tasksToRetry = Sets.newHashSet( + cf.getChildren() + .forPath(JOINER.join(config.getIndexerTaskPath(), worker.getHost())) + ); + tasksToRetry.addAll( + cf.getChildren() + .forPath(JOINER.join(config.getIndexerStatusPath(), worker.getHost())) + ); + log.info("%s has %d tasks to retry", worker.getHost(), tasksToRetry.size()); for (String taskId : tasksToRetry) { TaskRunnerWorkItem taskRunnerWorkItem = runningTasks.get(taskId); @@ -655,13 +672,13 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider if (cf.checkExists().forPath(taskPath) != null) { cf.delete().guaranteed().forPath(taskPath); } - retryTask(taskRunnerWorkItem, worker.getHost()); + retryTask(taskRunnerWorkItem); } else { log.warn("RemoteTaskRunner has no knowledge of task %s", taskId); } } - zkWorker.getStatusCache().close(); + zkWorker.close(); } catch (Exception e) { throw Throwables.propagate(e); @@ -672,7 +689,7 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider } } - private ZkWorker findWorkerForTask() + private ZkWorker findWorkerForTask(final Task task) { try { final MinMaxPriorityQueue workerQueue = MinMaxPriorityQueue.orderedBy( @@ -691,6 +708,13 @@ public class RemoteTaskRunner implements TaskRunner, TaskLogProvider @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() diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskExistsException.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskExistsException.java similarity index 88% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/TaskExistsException.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskExistsException.java index ddc146c3167..c4fe2dffd8e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskExistsException.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskExistsException.java @@ -1,4 +1,4 @@ -package com.metamx.druid.merger.coordinator; +package com.metamx.druid.indexing.coordinator; public class TaskExistsException extends RuntimeException { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskLockbox.java similarity index 99% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskLockbox.java index 811429b0a05..b47e66aa573 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskLockbox.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskLockbox.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator; +package com.metamx.druid.indexing.coordinator; import com.google.common.base.Function; import com.google.common.base.Objects; @@ -31,8 +31,8 @@ import com.google.common.collect.Sets; import com.metamx.common.IAE; import com.metamx.common.guava.Comparators; import com.metamx.common.guava.FunctionalIterable; -import com.metamx.druid.merger.common.TaskLock; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.TaskLock; +import com.metamx.druid.indexing.common.task.Task; import com.metamx.emitter.EmittingLogger; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java similarity index 90% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java index c00df6bf252..d83ecf2f747 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskMasterLifecycle.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskMasterLifecycle.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator; +package com.metamx.druid.indexing.coordinator; import com.google.common.base.Optional; import com.google.common.base.Throwables; @@ -27,13 +27,13 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.curator.discovery.ServiceAnnouncer; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServiceDiscoveryConfig; -import com.metamx.druid.merger.common.actions.TaskActionClient; -import com.metamx.druid.merger.common.actions.TaskActionClientFactory; -import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; -import com.metamx.druid.merger.coordinator.exec.TaskConsumer; -import com.metamx.druid.merger.coordinator.scaling.ResourceManagementScheduler; -import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerFactory; +import com.metamx.druid.indexing.common.actions.TaskActionClient; +import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; +import com.metamx.druid.indexing.common.task.Task; +import com.metamx.druid.indexing.coordinator.config.IndexerCoordinatorConfig; +import com.metamx.druid.indexing.coordinator.exec.TaskConsumer; +import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementScheduler; +import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementSchedulerFactory; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import org.apache.curator.framework.CuratorFramework; @@ -105,7 +105,10 @@ public class TaskMasterLifecycle leaderLifecycle.addManagedInstance(taskRunner); Initialization.announceDefaultService(serviceDiscoveryConfig, serviceAnnouncer, leaderLifecycle); leaderLifecycle.addManagedInstance(taskConsumer); - leaderLifecycle.addManagedInstance(resourceManagementScheduler); + + if (indexerCoordinatorConfig.isAutoScalingEnabled()) { + leaderLifecycle.addManagedInstance(resourceManagementScheduler); + } try { leaderLifecycle.start(); diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskQueue.java similarity index 97% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskQueue.java index fdb053c764f..630934c6d17 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskQueue.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskQueue.java @@ -17,9 +17,8 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator; +package com.metamx.druid.indexing.coordinator; -import com.fasterxml.jackson.core.JsonProcessingException; import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; @@ -29,9 +28,9 @@ import com.google.common.collect.Multimap; import com.google.common.collect.Ordering; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.druid.merger.common.TaskLock; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.TaskLock; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.task.Task; import com.metamx.emitter.EmittingLogger; import java.util.List; @@ -46,7 +45,7 @@ import java.util.concurrent.locks.ReentrantLock; * {@link #take} or {@link #poll}. Ordering is mostly-FIFO, with deviations when the natural next task would conflict * with a currently-running task. In that case, tasks are skipped until a runnable one is found. *

    - * To manage locking, the queue keeps track of currently-running tasks as {@link com.metamx.druid.merger.common.TaskLock} objects. The idea is that + * To manage locking, the queue keeps track of currently-running tasks as {@link com.metamx.druid.indexing.common.TaskLock} objects. The idea is that * only one TaskLock can be running on a particular dataSource + interval, and that TaskLock has a single version * string that all tasks in the group must use to publish segments. Tasks in the same TaskLock may run concurrently. *

    diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java similarity index 88% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java index ca6c7365a55..6e8fcfd6a0f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunner.java @@ -17,16 +17,16 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator; +package com.metamx.druid.indexing.coordinator; import com.google.common.util.concurrent.ListenableFuture; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.task.Task; import java.util.Collection; /** - * Interface for handing off tasks. Used by a {@link com.metamx.druid.merger.coordinator.exec.TaskConsumer} to + * Interface for handing off tasks. Used by a {@link com.metamx.druid.indexing.coordinator.exec.TaskConsumer} to * run tasks that have been locked. */ public interface TaskRunner diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerFactory.java similarity index 94% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerFactory.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerFactory.java index a1b5746c036..9bac2678235 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerFactory.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator; +package com.metamx.druid.indexing.coordinator; public interface TaskRunnerFactory { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerWorkItem.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkItem.java similarity index 90% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerWorkItem.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkItem.java index d1b3ccbd403..74321f5b1b6 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerWorkItem.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkItem.java @@ -17,16 +17,14 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator; +package com.metamx.druid.indexing.coordinator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ComparisonChain; -import com.google.common.collect.Ordering; import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.SettableFuture; -import com.metamx.druid.merger.common.RetryPolicy; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.RetryPolicy; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.task.Task; import org.joda.time.DateTime; import org.joda.time.DateTimeComparator; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerWorkQueue.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkQueue.java similarity index 96% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerWorkQueue.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkQueue.java index f0dc161ad1a..0562040c35b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskRunnerWorkQueue.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskRunnerWorkQueue.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator; +package com.metamx.druid.indexing.coordinator; import org.joda.time.DateTime; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskStorage.java similarity index 91% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskStorage.java index 9058050ce98..00a9ad8103b 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorage.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskStorage.java @@ -17,13 +17,13 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator; +package com.metamx.druid.indexing.coordinator; import com.google.common.base.Optional; -import com.metamx.druid.merger.common.TaskLock; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.actions.TaskAction; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.TaskLock; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.actions.TaskAction; +import com.metamx.druid.indexing.common.task.Task; import java.util.List; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorageQueryAdapter.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskStorageQueryAdapter.java similarity index 94% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorageQueryAdapter.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskStorageQueryAdapter.java index bd7bca2af12..09000104521 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/TaskStorageQueryAdapter.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/TaskStorageQueryAdapter.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator; +package com.metamx.druid.indexing.coordinator; import com.google.common.base.Function; import com.google.common.base.Optional; @@ -27,11 +27,11 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.common.guava.FunctionalIterable; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.actions.SegmentInsertAction; -import com.metamx.druid.merger.common.actions.SpawnTasksAction; -import com.metamx.druid.merger.common.actions.TaskAction; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.actions.SegmentInsertAction; +import com.metamx.druid.indexing.common.actions.SpawnTasksAction; +import com.metamx.druid.indexing.common.actions.TaskAction; +import com.metamx.druid.indexing.common.task.Task; import java.util.List; import java.util.Map; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/ThreadPoolTaskRunner.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java similarity index 96% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/ThreadPoolTaskRunner.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java index 2906c0196a8..6ed4d3abf82 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/ThreadPoolTaskRunner.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ThreadPoolTaskRunner.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator; +package com.metamx.druid.indexing.coordinator; import com.google.common.base.Function; import com.google.common.base.Throwables; @@ -30,10 +30,10 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.Query; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.TaskToolboxFactory; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.TaskToolbox; +import com.metamx.druid.indexing.common.TaskToolboxFactory; +import com.metamx.druid.indexing.common.task.Task; import com.metamx.druid.query.NoopQueryRunner; import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.segment.QuerySegmentWalker; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/ZkWorker.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java similarity index 93% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/ZkWorker.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java index a001fa7afe3..16b0f57f5c5 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/ZkWorker.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/ZkWorker.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator; +package com.metamx.druid.indexing.coordinator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; @@ -25,8 +25,8 @@ import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.worker.Worker; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.worker.Worker; import org.apache.curator.framework.recipes.cache.ChildData; import org.apache.curator.framework.recipes.cache.PathChildrenCache; @@ -84,11 +84,6 @@ public class ZkWorker implements Closeable ); } - public PathChildrenCache getStatusCache() - { - return statusCache; - } - @JsonProperty public DateTime getLastCompletedTaskTime() { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/EC2AutoScalingStrategyConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/EC2AutoScalingStrategyConfig.java similarity index 95% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/config/EC2AutoScalingStrategyConfig.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/EC2AutoScalingStrategyConfig.java index a8cfcf8df22..77575b37cfd 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/EC2AutoScalingStrategyConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/EC2AutoScalingStrategyConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.config; +package com.metamx.druid.indexing.coordinator.config; import org.skife.config.Config; import org.skife.config.Default; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/ForkingTaskRunnerConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java similarity index 83% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/config/ForkingTaskRunnerConfig.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java index fc8cf85e60b..b5a873d8aa3 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/ForkingTaskRunnerConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/ForkingTaskRunnerConfig.java @@ -1,6 +1,6 @@ -package com.metamx.druid.merger.coordinator.config; +package com.metamx.druid.indexing.coordinator.config; -import com.metamx.druid.merger.worker.executor.ExecutorMain; +import com.metamx.druid.indexing.worker.executor.ExecutorMain; import org.skife.config.Config; import org.skife.config.Default; @@ -8,7 +8,7 @@ import java.io.File; public abstract class ForkingTaskRunnerConfig { - @Config("druid.merger.taskDir") + @Config("druid.indexer.taskDir") public abstract File getBaseTaskDir(); @Config("druid.indexer.fork.java") diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerCoordinatorConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/IndexerCoordinatorConfig.java similarity index 77% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerCoordinatorConfig.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/IndexerCoordinatorConfig.java index 88d00b42500..9c3baf06c24 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerCoordinatorConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/IndexerCoordinatorConfig.java @@ -17,14 +17,13 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.config; +package com.metamx.druid.indexing.coordinator.config; import com.google.common.base.Splitter; import com.google.common.collect.ImmutableSet; import com.metamx.druid.initialization.ZkPathsConfig; import org.skife.config.Config; import org.skife.config.Default; -import org.skife.config.DefaultNull; import java.util.Set; @@ -37,31 +36,31 @@ public abstract class IndexerCoordinatorConfig extends ZkPathsConfig @Config("druid.host") public abstract String getServerName(); - @Config("druid.merger.threads") + @Config("druid.indexer.threads") @Default("1") public abstract int getNumLocalThreads(); - @Config("druid.merger.runner") + @Config("druid.indexer.runner") @Default("remote") public abstract String getRunnerImpl(); - @Config("druid.merger.storage") + @Config("druid.indexer.storage") @Default("local") public abstract String getStorageImpl(); - @Config("druid.merger.whitelist.enabled") + @Config("druid.indexer.whitelist.enabled") @Default("false") public abstract boolean isWhitelistEnabled(); - @Config("druid.merger.whitelist.datasources") + @Config("druid.indexer.whitelist.datasources") @Default("") public abstract String getWhitelistDatasourcesString(); public Set getWhitelistDatasources() { - if(whitelistDatasources == null) { + if (whitelistDatasources == null) { synchronized (this) { - if(whitelistDatasources == null) { + if (whitelistDatasources == null) { whitelistDatasources = ImmutableSet.copyOf(Splitter.on(",").split(getWhitelistDatasourcesString())); } } @@ -70,7 +69,13 @@ public abstract class IndexerCoordinatorConfig extends ZkPathsConfig return whitelistDatasources; } - @Config("druid.indexer.strategy") + @Config("druid.indexer.autoscaling.enabled") + public boolean isAutoScalingEnabled() + { + return false; + } + + @Config("druid.indexer.autoscaling.strategy") @Default("noop") - public abstract String getStrategyImpl(); + public abstract String getAutoScalingImpl(); } diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/IndexerDbConnectorConfig.java similarity index 96% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/IndexerDbConnectorConfig.java index 4e2265789cf..63fb4c4245a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/IndexerDbConnectorConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/IndexerDbConnectorConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.config; +package com.metamx.druid.indexing.coordinator.config; import com.fasterxml.jackson.annotation.JsonProperty; import com.metamx.druid.db.DbConnectorConfig; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java similarity index 90% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java index 14e1656e8b8..15970fc37b3 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/config/RemoteTaskRunnerConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/config/RemoteTaskRunnerConfig.java @@ -17,9 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.config; +package com.metamx.druid.indexing.coordinator.config; -import com.metamx.druid.merger.common.config.IndexerZkConfig; +import com.metamx.druid.indexing.common.config.IndexerZkConfig; import org.joda.time.Duration; import org.skife.config.Config; import org.skife.config.Default; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/exec/TaskConsumer.java similarity index 94% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/exec/TaskConsumer.java index 4d086368bc8..df0902ce838 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/exec/TaskConsumer.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/exec/TaskConsumer.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.exec; +package com.metamx.druid.indexing.coordinator.exec; import com.google.common.base.Throwables; import com.google.common.util.concurrent.FutureCallback; @@ -25,11 +25,11 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.actions.TaskActionClientFactory; -import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.TaskQueue; -import com.metamx.druid.merger.coordinator.TaskRunner; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; +import com.metamx.druid.indexing.common.task.Task; +import com.metamx.druid.indexing.coordinator.TaskQueue; +import com.metamx.druid.indexing.coordinator.TaskRunner; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorMain.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorMain.java similarity index 96% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorMain.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorMain.java index bfcf4573e46..3714986a8a0 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorMain.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorMain.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.http; +package com.metamx.druid.indexing.coordinator.http; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java similarity index 88% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java index 8a2923431ec..69a9d4dac88 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorNode.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.http; +package com.metamx.druid.indexing.coordinator.http; import com.amazonaws.auth.BasicAWSCredentials; import com.amazonaws.services.ec2.AmazonEC2Client; @@ -59,46 +59,46 @@ import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.merger.common.RetryPolicyFactory; -import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory; -import com.metamx.druid.merger.common.actions.TaskActionClientFactory; -import com.metamx.druid.merger.common.actions.TaskActionToolbox; -import com.metamx.druid.merger.common.config.IndexerZkConfig; -import com.metamx.druid.merger.common.config.RetryPolicyConfig; -import com.metamx.druid.merger.common.config.TaskLogConfig; -import com.metamx.druid.merger.common.index.EventReceiverFirehoseFactory; -import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; -import com.metamx.druid.merger.common.tasklogs.NoopTaskLogs; -import com.metamx.druid.merger.common.tasklogs.S3TaskLogs; -import com.metamx.druid.merger.common.tasklogs.SwitchingTaskLogProvider; -import com.metamx.druid.merger.common.tasklogs.TaskLogProvider; -import com.metamx.druid.merger.common.tasklogs.TaskLogs; -import com.metamx.druid.merger.coordinator.DbTaskStorage; -import com.metamx.druid.merger.coordinator.ForkingTaskRunner; -import com.metamx.druid.merger.coordinator.HeapMemoryTaskStorage; -import com.metamx.druid.merger.coordinator.MergerDBCoordinator; -import com.metamx.druid.merger.coordinator.RemoteTaskRunner; -import com.metamx.druid.merger.coordinator.TaskLockbox; -import com.metamx.druid.merger.coordinator.TaskMasterLifecycle; -import com.metamx.druid.merger.coordinator.TaskQueue; -import com.metamx.druid.merger.coordinator.TaskRunner; -import com.metamx.druid.merger.coordinator.TaskRunnerFactory; -import com.metamx.druid.merger.coordinator.TaskStorage; -import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter; -import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; -import com.metamx.druid.merger.coordinator.config.ForkingTaskRunnerConfig; -import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; -import com.metamx.druid.merger.coordinator.config.IndexerDbConnectorConfig; -import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; -import com.metamx.druid.merger.coordinator.scaling.AutoScalingStrategy; -import com.metamx.druid.merger.coordinator.scaling.EC2AutoScalingStrategy; -import com.metamx.druid.merger.coordinator.scaling.NoopAutoScalingStrategy; -import com.metamx.druid.merger.coordinator.scaling.ResourceManagementScheduler; -import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerConfig; -import com.metamx.druid.merger.coordinator.scaling.ResourceManagementSchedulerFactory; -import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagementStrategy; -import com.metamx.druid.merger.coordinator.scaling.SimpleResourceManagmentConfig; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; +import com.metamx.druid.indexing.common.RetryPolicyFactory; +import com.metamx.druid.indexing.common.actions.LocalTaskActionClientFactory; +import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; +import com.metamx.druid.indexing.common.actions.TaskActionToolbox; +import com.metamx.druid.indexing.common.config.IndexerZkConfig; +import com.metamx.druid.indexing.common.config.RetryPolicyConfig; +import com.metamx.druid.indexing.common.config.TaskLogConfig; +import com.metamx.druid.indexing.common.index.EventReceiverFirehoseFactory; +import com.metamx.druid.indexing.common.index.StaticS3FirehoseFactory; +import com.metamx.druid.indexing.common.tasklogs.NoopTaskLogs; +import com.metamx.druid.indexing.common.tasklogs.S3TaskLogs; +import com.metamx.druid.indexing.common.tasklogs.SwitchingTaskLogProvider; +import com.metamx.druid.indexing.common.tasklogs.TaskLogProvider; +import com.metamx.druid.indexing.common.tasklogs.TaskLogs; +import com.metamx.druid.indexing.coordinator.DbTaskStorage; +import com.metamx.druid.indexing.coordinator.ForkingTaskRunner; +import com.metamx.druid.indexing.coordinator.HeapMemoryTaskStorage; +import com.metamx.druid.indexing.coordinator.MergerDBCoordinator; +import com.metamx.druid.indexing.coordinator.RemoteTaskRunner; +import com.metamx.druid.indexing.coordinator.TaskLockbox; +import com.metamx.druid.indexing.coordinator.TaskMasterLifecycle; +import com.metamx.druid.indexing.coordinator.TaskQueue; +import com.metamx.druid.indexing.coordinator.TaskRunner; +import com.metamx.druid.indexing.coordinator.TaskRunnerFactory; +import com.metamx.druid.indexing.coordinator.TaskStorage; +import com.metamx.druid.indexing.coordinator.TaskStorageQueryAdapter; +import com.metamx.druid.indexing.coordinator.config.EC2AutoScalingStrategyConfig; +import com.metamx.druid.indexing.coordinator.config.ForkingTaskRunnerConfig; +import com.metamx.druid.indexing.coordinator.config.IndexerCoordinatorConfig; +import com.metamx.druid.indexing.coordinator.config.IndexerDbConnectorConfig; +import com.metamx.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; +import com.metamx.druid.indexing.coordinator.scaling.AutoScalingStrategy; +import com.metamx.druid.indexing.coordinator.scaling.EC2AutoScalingStrategy; +import com.metamx.druid.indexing.coordinator.scaling.NoopAutoScalingStrategy; +import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementScheduler; +import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementSchedulerConfig; +import com.metamx.druid.indexing.coordinator.scaling.ResourceManagementSchedulerFactory; +import com.metamx.druid.indexing.coordinator.scaling.SimpleResourceManagementStrategy; +import com.metamx.druid.indexing.coordinator.scaling.SimpleResourceManagmentConfig; +import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitters; @@ -311,7 +311,8 @@ public class IndexerCoordinatorNode extends QueryableNode> simplifyTaskFn = + new Function>() + { + @Override + public Map apply(TaskRunnerWorkItem input) + { + return new ImmutableMap.Builder() + .put("id", input.getTask().getId()) + .put("dataSource", input.getTask().getDataSource()) + .put("interval", + input.getTask().getImplicitLockInterval().get() == null + ? "" + : input.getTask().getImplicitLockInterval().get() + ) + .put("nodeType", input.getTask().getNodeType() == null ? "" : input.getTask().getNodeType()) + .put("createdTime", input.getCreatedTime()) + .put("queueInsertionTime", input.getQueueInsertionTime()) + .build(); + } + }; + private final IndexerCoordinatorConfig config; private final ServiceEmitter emitter; private final TaskMasterLifecycle taskMasterLifecycle; @@ -256,8 +279,24 @@ public class IndexerCoordinatorResource @GET @Path("/pendingTasks") @Produces("application/json") - public Response getPendingTasks() + public Response getPendingTasks( + @QueryParam("full") String full + ) { + if (full != null) { + return asLeaderWith( + taskMasterLifecycle.getTaskRunner(), + new Function() + { + @Override + public Response apply(TaskRunner taskRunner) + { + return Response.ok(taskRunner.getPendingTasks()).build(); + } + } + ); + } + return asLeaderWith( taskMasterLifecycle.getTaskRunner(), new Function() @@ -265,7 +304,12 @@ public class IndexerCoordinatorResource @Override public Response apply(TaskRunner taskRunner) { - return Response.ok(taskRunner.getPendingTasks()).build(); + return Response.ok( + Collections2.transform( + taskRunner.getPendingTasks(), + simplifyTaskFn + ) + ).build(); } } ); @@ -274,8 +318,24 @@ public class IndexerCoordinatorResource @GET @Path("/runningTasks") @Produces("application/json") - public Response getRunningTasks() + public Response getRunningTasks( + @QueryParam("full") String full + ) { + if (full != null) { + return asLeaderWith( + taskMasterLifecycle.getTaskRunner(), + new Function() + { + @Override + public Response apply(TaskRunner taskRunner) + { + return Response.ok(taskRunner.getRunningTasks()).build(); + } + } + ); + } + return asLeaderWith( taskMasterLifecycle.getTaskRunner(), new Function() @@ -283,7 +343,12 @@ public class IndexerCoordinatorResource @Override public Response apply(TaskRunner taskRunner) { - return Response.ok(taskRunner.getRunningTasks()).build(); + return Response.ok( + Collections2.transform( + taskRunner.getRunningTasks(), + simplifyTaskFn + ) + ).build(); } } ); @@ -340,13 +405,15 @@ public class IndexerCoordinatorResource } else { return Response.status(Response.Status.NOT_FOUND).build(); } - } catch (Exception e) { + } + catch (Exception e) { log.warn(e, "Failed to stream log for task %s", taskid); return Response.status(Response.Status.INTERNAL_SERVER_ERROR).build(); } } - public Response optionalTaskResponse(String taskid, String objectType, Optional x) { + public Response optionalTaskResponse(String taskid, String objectType, Optional x) + { final Map results = Maps.newHashMap(); results.put("task", taskid); if (x.isPresent()) { diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorServletModule.java similarity index 89% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorServletModule.java index 8618a622997..64383a667a1 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/http/IndexerCoordinatorServletModule.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/IndexerCoordinatorServletModule.java @@ -17,16 +17,16 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.http; +package com.metamx.druid.indexing.coordinator.http; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; import com.google.inject.Provides; import com.metamx.druid.config.JacksonConfigManager; -import com.metamx.druid.merger.common.tasklogs.TaskLogProvider; -import com.metamx.druid.merger.coordinator.TaskMasterLifecycle; -import com.metamx.druid.merger.coordinator.TaskStorageQueryAdapter; -import com.metamx.druid.merger.coordinator.config.IndexerCoordinatorConfig; +import com.metamx.druid.indexing.common.tasklogs.TaskLogProvider; +import com.metamx.druid.indexing.coordinator.TaskMasterLifecycle; +import com.metamx.druid.indexing.coordinator.TaskStorageQueryAdapter; +import com.metamx.druid.indexing.coordinator.config.IndexerCoordinatorConfig; import com.metamx.emitter.service.ServiceEmitter; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; @@ -68,6 +68,7 @@ public class IndexerCoordinatorServletModule extends JerseyServletModule protected void configureServlets() { bind(IndexerCoordinatorResource.class); + bind(OldIndexerCoordinatorResource.class); bind(ObjectMapper.class).toInstance(jsonMapper); bind(IndexerCoordinatorConfig.class).toInstance(indexerCoordinatorConfig); bind(ServiceEmitter.class).toInstance(emitter); diff --git a/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/OldIndexerCoordinatorResource.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/OldIndexerCoordinatorResource.java new file mode 100644 index 00000000000..36e7e948cd8 --- /dev/null +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/http/OldIndexerCoordinatorResource.java @@ -0,0 +1,33 @@ +package com.metamx.druid.indexing.coordinator.http; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import com.metamx.druid.config.JacksonConfigManager; +import com.metamx.druid.indexing.common.tasklogs.TaskLogProvider; +import com.metamx.druid.indexing.coordinator.TaskMasterLifecycle; +import com.metamx.druid.indexing.coordinator.TaskStorageQueryAdapter; +import com.metamx.druid.indexing.coordinator.config.IndexerCoordinatorConfig; +import com.metamx.emitter.service.ServiceEmitter; + +import javax.ws.rs.Path; + +/** + */ +@Deprecated +@Path("/mmx/merger/v1") +public class OldIndexerCoordinatorResource extends IndexerCoordinatorResource +{ + @Inject + public OldIndexerCoordinatorResource( + IndexerCoordinatorConfig config, + ServiceEmitter emitter, + TaskMasterLifecycle taskMasterLifecycle, + TaskStorageQueryAdapter taskStorageQueryAdapter, + TaskLogProvider taskLogProvider, + JacksonConfigManager configManager, + ObjectMapper jsonMapper + ) throws Exception + { + super(config, emitter, taskMasterLifecycle, taskStorageQueryAdapter, taskLogProvider, configManager, jsonMapper); + } +} diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/AutoScalingData.java similarity index 96% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/AutoScalingData.java index 9eb3cb5a093..8a7bc1840d5 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingData.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/AutoScalingData.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.scaling; +package com.metamx.druid.indexing.coordinator.scaling; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingStrategy.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/AutoScalingStrategy.java similarity index 96% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingStrategy.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/AutoScalingStrategy.java index 831bc3aa313..68a3170c54a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/AutoScalingStrategy.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/AutoScalingStrategy.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.scaling; +package com.metamx.druid.indexing.coordinator.scaling; import java.util.List; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategy.java similarity index 96% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategy.java index fb3e00a2dc5..f4f44d76e8c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategy.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategy.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.scaling; +package com.metamx.druid.indexing.coordinator.scaling; import com.amazonaws.services.ec2.AmazonEC2Client; import com.amazonaws.services.ec2.model.DescribeInstancesRequest; @@ -31,9 +31,9 @@ import com.amazonaws.services.ec2.model.TerminateInstancesRequest; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.collect.Lists; -import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; -import com.metamx.druid.merger.coordinator.setup.EC2NodeData; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; +import com.metamx.druid.indexing.coordinator.config.EC2AutoScalingStrategyConfig; +import com.metamx.druid.indexing.coordinator.setup.EC2NodeData; +import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; import com.metamx.emitter.EmittingLogger; import org.apache.commons.codec.binary.Base64; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopAutoScalingStrategy.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/NoopAutoScalingStrategy.java similarity index 97% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopAutoScalingStrategy.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/NoopAutoScalingStrategy.java index 0772f167231..84881deb49a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/NoopAutoScalingStrategy.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/NoopAutoScalingStrategy.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.scaling; +package com.metamx.druid.indexing.coordinator.scaling; import com.metamx.emitter.EmittingLogger; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementScheduler.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementScheduler.java similarity index 97% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementScheduler.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementScheduler.java index b3ea474cc0c..5ac9d4384bc 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementScheduler.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementScheduler.java @@ -17,14 +17,14 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.scaling; +package com.metamx.druid.indexing.coordinator.scaling; import com.metamx.common.concurrent.ScheduledExecutors; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.druid.PeriodGranularity; -import com.metamx.druid.merger.coordinator.TaskRunner; +import com.metamx.druid.indexing.coordinator.TaskRunner; import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Period; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementSchedulerConfig.java similarity index 96% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerConfig.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementSchedulerConfig.java index 924fe897490..4850ecaaf02 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementSchedulerConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.scaling; +package com.metamx.druid.indexing.coordinator.scaling; import org.joda.time.DateTime; import org.joda.time.Duration; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementSchedulerFactory.java similarity index 89% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerFactory.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementSchedulerFactory.java index 8d33c8be220..51c7833db9a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementSchedulerFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementSchedulerFactory.java @@ -17,9 +17,9 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.scaling; +package com.metamx.druid.indexing.coordinator.scaling; -import com.metamx.druid.merger.coordinator.TaskRunner; +import com.metamx.druid.indexing.coordinator.TaskRunner; /** */ diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementStrategy.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementStrategy.java similarity index 88% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementStrategy.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementStrategy.java index 7e2d666c3bd..b658b812ac8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ResourceManagementStrategy.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ResourceManagementStrategy.java @@ -17,10 +17,10 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.scaling; +package com.metamx.druid.indexing.coordinator.scaling; -import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem; -import com.metamx.druid.merger.coordinator.ZkWorker; +import com.metamx.druid.indexing.coordinator.TaskRunnerWorkItem; +import com.metamx.druid.indexing.coordinator.ZkWorker; import java.util.Collection; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStats.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ScalingStats.java similarity index 98% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStats.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ScalingStats.java index 70c15bcada0..6661b5d135d 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/ScalingStats.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/ScalingStats.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.scaling; +package com.metamx.druid.indexing.coordinator.scaling; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonValue; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java similarity index 97% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java index 6da62c14578..023091935f8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategy.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategy.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.scaling; +package com.metamx.druid.indexing.coordinator.scaling; import com.google.common.base.Function; import com.google.common.base.Predicate; @@ -25,9 +25,9 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.common.guava.FunctionalIterable; -import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem; -import com.metamx.druid.merger.coordinator.ZkWorker; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; +import com.metamx.druid.indexing.coordinator.TaskRunnerWorkItem; +import com.metamx.druid.indexing.coordinator.ZkWorker; +import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; import com.metamx.emitter.EmittingLogger; import org.joda.time.DateTime; import org.joda.time.Duration; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagmentConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagmentConfig.java similarity index 96% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagmentConfig.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagmentConfig.java index 19fc2fdcedb..1584e4b5dfc 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagmentConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagmentConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.scaling; +package com.metamx.druid.indexing.coordinator.scaling; import org.joda.time.Duration; import org.skife.config.Config; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/setup/EC2NodeData.java similarity index 98% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/setup/EC2NodeData.java index c966705d7a6..28f877a1f17 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/EC2NodeData.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/setup/EC2NodeData.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.setup; +package com.metamx.druid.indexing.coordinator.setup; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/setup/GalaxyUserData.java similarity index 97% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/setup/GalaxyUserData.java index 4b6012748f6..60241f472ff 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/GalaxyUserData.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/setup/GalaxyUserData.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.setup; +package com.metamx.druid.indexing.coordinator.setup; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/setup/WorkerSetupData.java similarity index 97% rename from merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/setup/WorkerSetupData.java index d4384b0cae9..dcbf0f270ad 100644 --- a/merger/src/main/java/com/metamx/druid/merger/coordinator/setup/WorkerSetupData.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/coordinator/setup/WorkerSetupData.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.setup; +package com.metamx.druid.indexing.coordinator.setup; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/Worker.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/Worker.java similarity index 95% rename from merger/src/main/java/com/metamx/druid/merger/worker/Worker.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/worker/Worker.java index ad389caf2f4..7749ff8565f 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/Worker.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/Worker.java @@ -17,11 +17,11 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.worker; +package com.metamx.druid.indexing.worker; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.metamx.druid.merger.worker.config.WorkerConfig; +import com.metamx.druid.indexing.worker.config.WorkerConfig; /** * A container for worker metadata. diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerCuratorCoordinator.java similarity index 82% rename from merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerCuratorCoordinator.java index 55921b06410..44b308165b8 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/WorkerCuratorCoordinator.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerCuratorCoordinator.java @@ -17,18 +17,20 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.worker; +package com.metamx.druid.indexing.worker; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Joiner; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.MoreExecutors; import com.metamx.common.ISE; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.config.IndexerZkConfig; +import com.metamx.druid.curator.announcement.Announcer; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.config.IndexerZkConfig; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.state.ConnectionState; import org.apache.curator.framework.state.ConnectionStateListener; @@ -52,6 +54,7 @@ public class WorkerCuratorCoordinator private final CuratorFramework curatorFramework; private final Worker worker; private final IndexerZkConfig config; + private final Announcer announcer; private final String baseAnnouncementsPath; private final String baseTaskPath; @@ -71,6 +74,8 @@ public class WorkerCuratorCoordinator this.worker = worker; this.config = config; + this.announcer = new Announcer(curatorFramework, MoreExecutors.sameThreadExecutor()); + this.baseAnnouncementsPath = getPath(Arrays.asList(config.getIndexerAnnouncementPath(), worker.getHost())); this.baseTaskPath = getPath(Arrays.asList(config.getIndexerTaskPath(), worker.getHost())); this.baseStatusPath = getPath(Arrays.asList(config.getIndexerStatusPath(), worker.getHost())); @@ -79,7 +84,7 @@ public class WorkerCuratorCoordinator @LifecycleStart public void start() throws Exception { - log.info("Starting WorkerCuratorCoordinator for server[%s]", worker.getHost()); + log.info("WorkerCuratorCoordinator good to go sir. Server[%s]", worker.getHost()); synchronized (lock) { if (started) { return; @@ -95,33 +100,8 @@ public class WorkerCuratorCoordinator CreateMode.PERSISTENT, ImmutableMap.of("created", new DateTime().toString()) ); - makePathIfNotExisting( - getAnnouncementsPathForWorker(), - CreateMode.EPHEMERAL, - worker - ); - - curatorFramework.getConnectionStateListenable().addListener( - new ConnectionStateListener() - { - @Override - public void stateChanged(CuratorFramework client, ConnectionState newState) - { - try { - if (newState.equals(ConnectionState.RECONNECTED)) { - makePathIfNotExisting( - getAnnouncementsPathForWorker(), - CreateMode.EPHEMERAL, - worker - ); - } - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - } - ); + announcer.start(); + announcer.announce(getAnnouncementsPathForWorker(), jsonMapper.writeValueAsBytes(worker)); started = true; } @@ -136,9 +116,8 @@ public class WorkerCuratorCoordinator return; } - curatorFramework.delete() - .guaranteed() - .forPath(getAnnouncementsPathForWorker()); + announcer.unannounce(getAnnouncementsPathForWorker()); + announcer.stop(); started = false; } @@ -193,16 +172,6 @@ public class WorkerCuratorCoordinator return getPath(Arrays.asList(baseStatusPath, statusId)); } - public boolean statusExists(String id) - { - try { - return (curatorFramework.checkExists().forPath(getStatusPathForId(id)) != null); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } - public void unannounceTask(String taskId) { try { diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/WorkerTaskMonitor.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerTaskMonitor.java similarity index 87% rename from merger/src/main/java/com/metamx/druid/merger/worker/WorkerTaskMonitor.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerTaskMonitor.java index c3dfac1cb53..584ec0a2ef6 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/WorkerTaskMonitor.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/WorkerTaskMonitor.java @@ -17,26 +17,21 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.worker; +package com.metamx.druid.indexing.worker; import com.fasterxml.jackson.databind.ObjectMapper; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.TaskToolboxFactory; -import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.TaskRunner; -import com.metamx.druid.merger.coordinator.TaskRunnerFactory; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.task.Task; +import com.metamx.druid.indexing.coordinator.TaskRunner; import com.metamx.druid.query.segment.QuerySegmentWalker; import com.metamx.emitter.EmittingLogger; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; -import org.apache.commons.io.FileUtils; -import java.io.File; import java.util.List; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; @@ -44,7 +39,7 @@ import java.util.concurrent.ExecutorService; /** * The monitor watches ZK at a specified path for new tasks to appear. Upon starting the monitor, a listener will be * created that waits for new tasks. Tasks are executed as soon as they are seen. - * + *

    * The monitor implements {@link QuerySegmentWalker} so tasks can offer up queryable data. This is useful for * realtime index tasks. */ @@ -101,7 +96,10 @@ public class WorkerTaskMonitor ); if (isTaskRunning(task)) { - log.warn("Got task %s that I am already running...", task.getId()); + log.warn( + "I can't build it. There's something in the way. Got task %s that I am already running...", + task.getId() + ); workerCuratorCoordinator.unannounceTask(task.getId()); return; } @@ -114,7 +112,7 @@ public class WorkerTaskMonitor { final long startTime = System.currentTimeMillis(); - log.info("Running task [%s]", task.getId()); + log.info("Affirmative. Running task [%s]", task.getId()); running.add(task); TaskStatus taskStatus; @@ -124,11 +122,12 @@ public class WorkerTaskMonitor taskStatus = taskRunner.run(task).get(); } catch (Exception e) { - log.makeAlert(e, "Failed to run task") + log.makeAlert(e, "I can't build there. Failed to run task") .addData("task", task.getId()) .emit(); taskStatus = TaskStatus.failure(task.getId()); - } finally { + } + finally { running.remove(task); } @@ -136,7 +135,11 @@ public class WorkerTaskMonitor try { workerCuratorCoordinator.updateStatus(taskStatus); - log.info("Completed task [%s] with status [%s]", task.getId(), taskStatus.getStatusCode()); + log.info( + "Job's finished. Completed [%s] with status [%s]", + task.getId(), + taskStatus.getStatusCode() + ); } catch (Exception e) { log.makeAlert(e, "Failed to update task status") diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/config/ChatHandlerProviderConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/config/ChatHandlerProviderConfig.java similarity index 87% rename from merger/src/main/java/com/metamx/druid/merger/worker/config/ChatHandlerProviderConfig.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/worker/config/ChatHandlerProviderConfig.java index 498f0008d58..da0ce3b67b3 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/config/ChatHandlerProviderConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/config/ChatHandlerProviderConfig.java @@ -1,4 +1,4 @@ -package com.metamx.druid.merger.worker.config; +package com.metamx.druid.indexing.worker.config; import org.skife.config.Config; import org.skife.config.DefaultNull; diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/config/WorkerConfig.java similarity index 94% rename from merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/worker/config/WorkerConfig.java index afb217c5e77..c6b3bdcdc74 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/config/WorkerConfig.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/config/WorkerConfig.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.worker.config; +package com.metamx.druid.indexing.worker.config; import org.skife.config.Config; import org.skife.config.Default; @@ -26,13 +26,13 @@ import org.skife.config.Default; */ public abstract class WorkerConfig { - @Config("druid.merger.threads") - @Default("1") - public abstract int getNumThreads(); - @Config("druid.host") public abstract String getHost(); + @Config("druid.worker.threads") + @Default("1") + public abstract int getNumThreads(); + @Config("druid.worker.ip") public abstract String getIp(); diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/executor/ChatHandlerResource.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ChatHandlerResource.java similarity index 80% rename from merger/src/main/java/com/metamx/druid/merger/worker/executor/ChatHandlerResource.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ChatHandlerResource.java index ab33dc4d99c..05d459d1d3a 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/executor/ChatHandlerResource.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ChatHandlerResource.java @@ -1,16 +1,16 @@ -package com.metamx.druid.merger.worker.executor; +package com.metamx.druid.indexing.worker.executor; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.inject.Inject; -import com.metamx.druid.merger.common.index.ChatHandler; -import com.metamx.druid.merger.common.index.ChatHandlerProvider; +import com.metamx.druid.indexing.common.index.ChatHandler; +import com.metamx.druid.indexing.common.index.ChatHandlerProvider; import javax.ws.rs.Path; import javax.ws.rs.PathParam; import javax.ws.rs.core.Response; -@Path("/mmx/worker/v1") +@Path("/druid/worker/v1") public class ChatHandlerResource { private final ObjectMapper jsonMapper; diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/executor/ExecutorLifecycle.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycle.java similarity index 95% rename from merger/src/main/java/com/metamx/druid/merger/worker/executor/ExecutorLifecycle.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycle.java index 778021e7b42..2872da7ed58 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/executor/ExecutorLifecycle.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycle.java @@ -1,4 +1,4 @@ -package com.metamx.druid.merger.worker.executor; +package com.metamx.druid.indexing.worker.executor; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; @@ -10,9 +10,9 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.metamx.common.ISE; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.TaskRunner; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.task.Task; +import com.metamx.druid.indexing.coordinator.TaskRunner; import com.metamx.emitter.EmittingLogger; import java.io.BufferedReader; diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/executor/ExecutorLifecycleFactory.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycleFactory.java similarity index 85% rename from merger/src/main/java/com/metamx/druid/merger/worker/executor/ExecutorLifecycleFactory.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycleFactory.java index 7573289eb60..eec2fff2929 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/executor/ExecutorLifecycleFactory.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorLifecycleFactory.java @@ -1,7 +1,7 @@ -package com.metamx.druid.merger.worker.executor; +package com.metamx.druid.indexing.worker.executor; import com.fasterxml.jackson.databind.ObjectMapper; -import com.metamx.druid.merger.coordinator.TaskRunner; +import com.metamx.druid.indexing.coordinator.TaskRunner; import java.io.File; import java.io.InputStream; diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/executor/ExecutorMain.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorMain.java similarity index 97% rename from merger/src/main/java/com/metamx/druid/merger/worker/executor/ExecutorMain.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorMain.java index f495aad6469..f4471b61ab4 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/executor/ExecutorMain.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorMain.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.worker.executor; +package com.metamx.druid.indexing.worker.executor; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/executor/ExecutorNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java similarity index 94% rename from merger/src/main/java/com/metamx/druid/merger/worker/executor/ExecutorNode.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java index 93d7b770372..06c160106d5 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/executor/ExecutorNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorNode.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.worker.executor; +package com.metamx.druid.indexing.worker.executor; import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; @@ -51,17 +51,17 @@ import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentKiller; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.S3DataSegmentKiller; -import com.metamx.druid.merger.common.RetryPolicyFactory; -import com.metamx.druid.merger.common.TaskToolboxFactory; -import com.metamx.druid.merger.common.actions.RemoteTaskActionClientFactory; -import com.metamx.druid.merger.common.config.RetryPolicyConfig; -import com.metamx.druid.merger.common.config.TaskConfig; -import com.metamx.druid.merger.common.index.EventReceiverFirehoseFactory; -import com.metamx.druid.merger.common.index.ChatHandlerProvider; -import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; -import com.metamx.druid.merger.coordinator.ThreadPoolTaskRunner; -import com.metamx.druid.merger.worker.config.ChatHandlerProviderConfig; -import com.metamx.druid.merger.worker.config.WorkerConfig; +import com.metamx.druid.indexing.common.RetryPolicyFactory; +import com.metamx.druid.indexing.common.TaskToolboxFactory; +import com.metamx.druid.indexing.common.actions.RemoteTaskActionClientFactory; +import com.metamx.druid.indexing.common.config.RetryPolicyConfig; +import com.metamx.druid.indexing.common.config.TaskConfig; +import com.metamx.druid.indexing.common.index.EventReceiverFirehoseFactory; +import com.metamx.druid.indexing.common.index.ChatHandlerProvider; +import com.metamx.druid.indexing.common.index.StaticS3FirehoseFactory; +import com.metamx.druid.indexing.coordinator.ThreadPoolTaskRunner; +import com.metamx.druid.indexing.worker.config.ChatHandlerProviderConfig; +import com.metamx.druid.indexing.worker.config.WorkerConfig; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitters; @@ -220,7 +220,7 @@ public class ExecutorNode extends BaseServerNode root.addServlet(new ServletHolder(new StatusServlet()), "/status"); root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); root.addEventListener(new GuiceServletConfig(injector)); - root.addFilter(GuiceFilter.class, "/mmx/worker/v1/*", 0); + root.addFilter(GuiceFilter.class, "/druid/worker/v1/*", 0); root.addServlet( new ServletHolder( new QueryServlet(getJsonMapper(), getSmileMapper(), taskRunner, emitter, getRequestLogger()) diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/executor/ExecutorServletModule.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorServletModule.java similarity index 90% rename from merger/src/main/java/com/metamx/druid/merger/worker/executor/ExecutorServletModule.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorServletModule.java index fa6ea39f2ba..1350094420c 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/executor/ExecutorServletModule.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/executor/ExecutorServletModule.java @@ -1,9 +1,9 @@ -package com.metamx.druid.merger.worker.executor; +package com.metamx.druid.indexing.worker.executor; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; import com.google.inject.Provides; -import com.metamx.druid.merger.common.index.ChatHandlerProvider; +import com.metamx.druid.indexing.common.index.ChatHandlerProvider; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerMain.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerMain.java similarity index 96% rename from merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerMain.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerMain.java index d7c22587a5b..9fa2f131979 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerMain.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerMain.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.worker.http; +package com.metamx.druid.indexing.worker.http; import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.logger.Logger; diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java similarity index 93% rename from merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java index 9ef0a900de6..1ccf32e0795 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerNode.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerNode.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.worker.http; +package com.metamx.druid.indexing.worker.http; import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; @@ -34,28 +34,26 @@ import com.metamx.common.lifecycle.Lifecycle; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.druid.QueryableNode; -import com.metamx.druid.curator.discovery.CuratorServiceAnnouncer; import com.metamx.druid.curator.discovery.ServiceAnnouncer; -import com.metamx.druid.curator.discovery.ServiceInstanceFactory; import com.metamx.druid.http.GuiceServletConfig; import com.metamx.druid.http.StatusServlet; import com.metamx.druid.initialization.Initialization; import com.metamx.druid.initialization.ServerConfig; import com.metamx.druid.initialization.ServiceDiscoveryConfig; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.merger.common.config.IndexerZkConfig; -import com.metamx.druid.merger.common.config.TaskLogConfig; -import com.metamx.druid.merger.common.index.EventReceiverFirehoseFactory; -import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; -import com.metamx.druid.merger.common.tasklogs.NoopTaskLogs; -import com.metamx.druid.merger.common.tasklogs.S3TaskLogs; -import com.metamx.druid.merger.common.tasklogs.TaskLogs; -import com.metamx.druid.merger.coordinator.ForkingTaskRunner; -import com.metamx.druid.merger.coordinator.config.ForkingTaskRunnerConfig; -import com.metamx.druid.merger.worker.Worker; -import com.metamx.druid.merger.worker.WorkerCuratorCoordinator; -import com.metamx.druid.merger.worker.WorkerTaskMonitor; -import com.metamx.druid.merger.worker.config.WorkerConfig; +import com.metamx.druid.indexing.common.config.IndexerZkConfig; +import com.metamx.druid.indexing.common.config.TaskLogConfig; +import com.metamx.druid.indexing.common.index.EventReceiverFirehoseFactory; +import com.metamx.druid.indexing.common.index.StaticS3FirehoseFactory; +import com.metamx.druid.indexing.common.tasklogs.NoopTaskLogs; +import com.metamx.druid.indexing.common.tasklogs.S3TaskLogs; +import com.metamx.druid.indexing.common.tasklogs.TaskLogs; +import com.metamx.druid.indexing.coordinator.ForkingTaskRunner; +import com.metamx.druid.indexing.coordinator.config.ForkingTaskRunnerConfig; +import com.metamx.druid.indexing.worker.Worker; +import com.metamx.druid.indexing.worker.WorkerCuratorCoordinator; +import com.metamx.druid.indexing.worker.WorkerTaskMonitor; +import com.metamx.druid.indexing.worker.config.WorkerConfig; import com.metamx.druid.utils.PropUtils; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.Emitters; @@ -211,7 +209,7 @@ public class WorkerNode extends QueryableNode root.addServlet(new ServletHolder(new StatusServlet()), "/status"); root.addServlet(new ServletHolder(new DefaultServlet()), "/*"); root.addEventListener(new GuiceServletConfig(injector)); - root.addFilter(GuiceFilter.class, "/mmx/worker/v1/*", 0); + root.addFilter(GuiceFilter.class, "/druid/worker/v1/*", 0); } @LifecycleStart diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerResource.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerResource.java similarity index 93% rename from merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerResource.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerResource.java index cb1671ed8e7..5dc5301160e 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerResource.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerResource.java @@ -17,16 +17,15 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.worker.http; +package com.metamx.druid.indexing.worker.http; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; -import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.io.InputSupplier; import com.google.inject.Inject; import com.metamx.common.logger.Logger; -import com.metamx.druid.merger.coordinator.ForkingTaskRunner; +import com.metamx.druid.indexing.coordinator.ForkingTaskRunner; import javax.ws.rs.DefaultValue; import javax.ws.rs.GET; @@ -36,12 +35,11 @@ import javax.ws.rs.PathParam; import javax.ws.rs.Produces; import javax.ws.rs.QueryParam; import javax.ws.rs.core.Response; -import java.io.IOException; import java.io.InputStream; /** */ -@Path("/mmx/worker/v1") +@Path("/druid/worker/v1") public class WorkerResource { private static final Logger log = new Logger(WorkerResource.class); diff --git a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerServletModule.java b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerServletModule.java similarity index 95% rename from merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerServletModule.java rename to indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerServletModule.java index b32c2e41da3..b135af7dcb6 100644 --- a/merger/src/main/java/com/metamx/druid/merger/worker/http/WorkerServletModule.java +++ b/indexing-service/src/main/java/com/metamx/druid/indexing/worker/http/WorkerServletModule.java @@ -17,12 +17,12 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.worker.http; +package com.metamx.druid.indexing.worker.http; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; import com.google.inject.Provides; -import com.metamx.druid.merger.coordinator.ForkingTaskRunner; +import com.metamx.druid.indexing.coordinator.ForkingTaskRunner; import com.metamx.emitter.service.ServiceEmitter; import com.sun.jersey.guice.JerseyServletModule; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; diff --git a/merger/src/main/resources/indexer_static/console.html b/indexing-service/src/main/resources/indexer_static/console.html similarity index 100% rename from merger/src/main/resources/indexer_static/console.html rename to indexing-service/src/main/resources/indexer_static/console.html diff --git a/merger/src/main/resources/indexer_static/js/console-0.0.1.js b/indexing-service/src/main/resources/indexer_static/js/console-0.0.1.js similarity index 100% rename from merger/src/main/resources/indexer_static/js/console-0.0.1.js rename to indexing-service/src/main/resources/indexer_static/js/console-0.0.1.js diff --git a/merger/src/test/java/com/metamx/druid/merger/TestTask.java b/indexing-service/src/test/java/com/metamx/druid/indexing/TestTask.java similarity index 90% rename from merger/src/test/java/com/metamx/druid/merger/TestTask.java rename to indexing-service/src/test/java/com/metamx/druid/indexing/TestTask.java index 2aa41dc031f..588e8fc2c6d 100644 --- a/merger/src/test/java/com/metamx/druid/merger/TestTask.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/TestTask.java @@ -17,16 +17,16 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger; +package com.metamx.druid.indexing; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.task.MergeTask; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.TaskToolbox; +import com.metamx.druid.indexing.common.task.MergeTask; import java.util.List; diff --git a/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskBaseTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/common/task/MergeTaskBaseTest.java similarity index 98% rename from merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskBaseTest.java rename to indexing-service/src/test/java/com/metamx/druid/indexing/common/task/MergeTaskBaseTest.java index e8c6622369a..b09b86e9756 100644 --- a/merger/src/test/java/com/metamx/druid/merger/common/task/MergeTaskBaseTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/common/task/MergeTaskBaseTest.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.task; +package com.metamx.druid.indexing.common.task; import com.google.common.base.Charsets; import com.google.common.collect.ImmutableList; diff --git a/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/common/task/TaskSerdeTest.java similarity index 99% rename from merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java rename to indexing-service/src/test/java/com/metamx/druid/indexing/common/task/TaskSerdeTest.java index adbd188be14..dd819056ac9 100644 --- a/merger/src/test/java/com/metamx/druid/merger/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/common/task/TaskSerdeTest.java @@ -1,4 +1,4 @@ -package com.metamx.druid.merger.common.task; +package com.metamx.druid.indexing.common.task; import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; @@ -15,7 +15,6 @@ import com.metamx.druid.indexer.granularity.UniformGranularitySpec; import com.metamx.druid.indexer.path.StaticPathSpec; import com.metamx.druid.indexer.rollup.DataRollupSpec; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.merger.common.index.StaticS3FirehoseFactory; import com.metamx.druid.realtime.Schema; import com.metamx.druid.shard.NoneShardSpec; import junit.framework.Assert; @@ -202,6 +201,7 @@ public class TaskSerdeTest public void testRealtimeIndexTaskSerde() throws Exception { final Task task = new RealtimeIndexTask( + null, null, new Schema("foo", null, new AggregatorFactory[0], QueryGranularity.NONE, new NoneShardSpec()), null, diff --git a/merger/src/test/java/com/metamx/druid/merger/common/task/VersionConverterTaskTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/common/task/VersionConverterTaskTest.java similarity index 97% rename from merger/src/test/java/com/metamx/druid/merger/common/task/VersionConverterTaskTest.java rename to indexing-service/src/test/java/com/metamx/druid/indexing/common/task/VersionConverterTaskTest.java index 8beeae6d411..70f13b6fbbf 100644 --- a/merger/src/test/java/com/metamx/druid/merger/common/task/VersionConverterTaskTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/common/task/VersionConverterTaskTest.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.common.task; +package com.metamx.druid.indexing.common.task; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RealtimeishTask.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RealtimeishTask.java new file mode 100644 index 00000000000..a43ec4fb2fc --- /dev/null +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RealtimeishTask.java @@ -0,0 +1,127 @@ +/* + * 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.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.metamx.druid.client.DataSegment; +import com.metamx.druid.indexing.common.TaskLock; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.TaskToolbox; +import com.metamx.druid.indexing.common.actions.LockAcquireAction; +import com.metamx.druid.indexing.common.actions.LockListAction; +import com.metamx.druid.indexing.common.actions.LockReleaseAction; +import com.metamx.druid.indexing.common.actions.SegmentInsertAction; +import com.metamx.druid.indexing.common.task.AbstractTask; +import org.joda.time.Interval; +import org.junit.Assert; + +import java.util.List; + +/** + */ +public class RealtimeishTask extends AbstractTask +{ + public RealtimeishTask() + { + super("rt1", "rt", "rt1", "foo", null); + } + + public RealtimeishTask(String id, String groupId, String availGroup, String dataSource, Interval interval) + { + super(id, groupId, availGroup, dataSource, interval); + } + + @Override + public String getType() + { + return "realtime_test"; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + final Interval interval1 = new Interval("2010-01-01T00/PT1H"); + final Interval interval2 = new Interval("2010-01-01T01/PT1H"); + + // Sort of similar to what realtime tasks do: + + // Acquire lock for first interval + final TaskLock lock1 = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval1)); + final List locks1 = toolbox.getTaskActionClient().submit(new LockListAction()); + + // (Confirm lock sanity) + Assert.assertEquals("lock1 interval", interval1, lock1.getInterval()); + Assert.assertEquals("locks1", ImmutableList.of(lock1), locks1); + + // Acquire lock for second interval + final TaskLock lock2 = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval2)); + final List locks2 = toolbox.getTaskActionClient().submit(new LockListAction()); + + // (Confirm lock sanity) + Assert.assertEquals("lock2 interval", interval2, lock2.getInterval()); + Assert.assertEquals("locks2", ImmutableList.of(lock1, lock2), locks2); + + // Push first segment + toolbox.getTaskActionClient() + .submit( + new SegmentInsertAction( + ImmutableSet.of( + DataSegment.builder() + .dataSource("foo") + .interval(interval1) + .version(lock1.getVersion()) + .build() + ) + ) + ); + + // Release first lock + toolbox.getTaskActionClient().submit(new LockReleaseAction(interval1)); + final List locks3 = toolbox.getTaskActionClient().submit(new LockListAction()); + + // (Confirm lock sanity) + Assert.assertEquals("locks3", ImmutableList.of(lock2), locks3); + + // Push second segment + toolbox.getTaskActionClient() + .submit( + new SegmentInsertAction( + ImmutableSet.of( + DataSegment.builder() + .dataSource("foo") + .interval(interval2) + .version(lock2.getVersion()) + .build() + ) + ) + ); + + // Release second lock + toolbox.getTaskActionClient().submit(new LockReleaseAction(interval2)); + final List locks4 = toolbox.getTaskActionClient().submit(new LockListAction()); + + // (Confirm lock sanity) + Assert.assertEquals("locks4", ImmutableList.of(), locks4); + + // Exit + return TaskStatus.success(getId()); + } +} diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java similarity index 84% rename from merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java rename to indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java index 573e4e1cac8..0a11fcb49fb 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RemoteTaskRunnerTest.java @@ -1,7 +1,8 @@ -package com.metamx.druid.merger.coordinator; +package com.metamx.druid.indexing.coordinator; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.google.common.base.Stopwatch; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.google.common.util.concurrent.FutureCallback; @@ -9,27 +10,28 @@ import com.google.common.util.concurrent.Futures; import com.metamx.common.ISE; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; +import com.metamx.druid.curator.PotentiallyGzippedCompressionProvider; +import com.metamx.druid.indexing.TestTask; +import com.metamx.druid.indexing.common.RetryPolicyFactory; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.TaskToolboxFactory; +import com.metamx.druid.indexing.common.config.IndexerZkConfig; +import com.metamx.druid.indexing.common.config.RetryPolicyConfig; +import com.metamx.druid.indexing.common.config.TaskConfig; +import com.metamx.druid.indexing.coordinator.config.RemoteTaskRunnerConfig; +import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; +import com.metamx.druid.indexing.worker.Worker; +import com.metamx.druid.indexing.worker.WorkerCuratorCoordinator; +import com.metamx.druid.indexing.worker.WorkerTaskMonitor; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.merger.TestTask; -import com.metamx.druid.merger.common.RetryPolicyFactory; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolboxFactory; -import com.metamx.druid.merger.common.config.IndexerZkConfig; -import com.metamx.druid.merger.common.config.RetryPolicyConfig; -import com.metamx.druid.merger.common.config.TaskConfig; -import com.metamx.druid.merger.coordinator.config.RemoteTaskRunnerConfig; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; -import com.metamx.druid.merger.worker.Worker; -import com.metamx.druid.merger.worker.WorkerCuratorCoordinator; -import com.metamx.druid.merger.worker.WorkerTaskMonitor; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.service.ServiceEmitter; +import org.apache.commons.lang.mutable.MutableBoolean; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.curator.test.TestingCluster; -import org.apache.commons.lang.mutable.MutableBoolean; import org.apache.zookeeper.CreateMode; import org.easymock.EasyMock; import org.joda.time.DateTime; @@ -43,6 +45,7 @@ import org.junit.Test; import java.io.File; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import static junit.framework.Assert.fail; @@ -78,6 +81,7 @@ public class RemoteTaskRunnerTest cf = CuratorFrameworkFactory.builder() .connectString(testingCluster.getConnectString()) .retryPolicy(new ExponentialBackoffRetry(1, 10)) + .compressionProvider(new PotentiallyGzippedCompressionProvider(false)) .build(); cf.start(); @@ -245,6 +249,34 @@ public class RemoteTaskRunnerTest Assert.assertTrue("TaskCallback was not called!", callbackCalled.booleanValue()); } + + @Test + public void testRunSameAvailabilityGroup() throws Exception + { + TestRealtimeTask theTask = new TestRealtimeTask("rt1", "rt1", "foo", TaskStatus.running("rt1")); + remoteTaskRunner.run(theTask); + remoteTaskRunner.run( + new TestRealtimeTask("rt2", "rt1", "foo", TaskStatus.running("rt2")) + ); + remoteTaskRunner.run( + new TestRealtimeTask("rt3", "rt2", "foo", TaskStatus.running("rt3")) + ); + + Stopwatch stopwatch = new Stopwatch(); + stopwatch.start(); + while (remoteTaskRunner.getRunningTasks().isEmpty()) { + Thread.sleep(100); + if (stopwatch.elapsed(TimeUnit.MILLISECONDS) > 1000) { + throw new ISE("Cannot find running task"); + } + } + + Assert.assertTrue(remoteTaskRunner.getRunningTasks().size() == 2); + Assert.assertTrue(remoteTaskRunner.getPendingTasks().size() == 1); + Assert.assertTrue(remoteTaskRunner.getPendingTasks().iterator().next().getTask().getId().equals("rt2")); + } + + private void makeTaskMonitor() throws Exception { WorkerCuratorCoordinator workerCuratorCoordinator = new WorkerCuratorCoordinator( @@ -323,6 +355,7 @@ public class RemoteTaskRunnerTest Executors.newSingleThreadExecutor() ); jsonMapper.registerSubtypes(new NamedType(TestTask.class, "test")); + jsonMapper.registerSubtypes(new NamedType(TestRealtimeTask.class, "test_realtime")); workerTaskMonitor.start(); } diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/RetryPolicyTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RetryPolicyTest.java similarity index 88% rename from merger/src/test/java/com/metamx/druid/merger/coordinator/RetryPolicyTest.java rename to indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RetryPolicyTest.java index 41b356d5f8d..cfa08d44401 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/RetryPolicyTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/RetryPolicyTest.java @@ -1,7 +1,7 @@ -package com.metamx.druid.merger.coordinator; +package com.metamx.druid.indexing.coordinator; -import com.metamx.druid.merger.common.RetryPolicy; -import com.metamx.druid.merger.common.config.RetryPolicyConfig; +import com.metamx.druid.indexing.common.RetryPolicy; +import com.metamx.druid.indexing.common.config.RetryPolicyConfig; import junit.framework.Assert; import org.joda.time.Duration; import org.junit.Test; diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java similarity index 79% rename from merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java rename to indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java index 92ffc0596c7..ef3c6412c6f 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskLifecycleTest.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator; +package com.metamx.druid.indexing.coordinator; import com.google.common.base.Optional; import com.google.common.base.Throwables; @@ -42,23 +42,23 @@ import com.metamx.druid.jackson.DefaultObjectMapper; import com.metamx.druid.loading.DataSegmentPusher; import com.metamx.druid.loading.DataSegmentKiller; import com.metamx.druid.loading.SegmentLoadingException; -import com.metamx.druid.merger.common.TaskLock; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.TaskToolboxFactory; -import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory; -import com.metamx.druid.merger.common.actions.LockAcquireAction; -import com.metamx.druid.merger.common.actions.LockListAction; -import com.metamx.druid.merger.common.actions.LockReleaseAction; -import com.metamx.druid.merger.common.actions.SegmentInsertAction; -import com.metamx.druid.merger.common.actions.TaskActionClientFactory; -import com.metamx.druid.merger.common.actions.TaskActionToolbox; -import com.metamx.druid.merger.common.config.TaskConfig; -import com.metamx.druid.merger.common.task.AbstractTask; -import com.metamx.druid.merger.common.task.IndexTask; -import com.metamx.druid.merger.common.task.KillTask; -import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.exec.TaskConsumer; +import com.metamx.druid.indexing.common.TaskLock; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.TaskToolbox; +import com.metamx.druid.indexing.common.TaskToolboxFactory; +import com.metamx.druid.indexing.common.actions.LocalTaskActionClientFactory; +import com.metamx.druid.indexing.common.actions.LockAcquireAction; +import com.metamx.druid.indexing.common.actions.LockListAction; +import com.metamx.druid.indexing.common.actions.LockReleaseAction; +import com.metamx.druid.indexing.common.actions.SegmentInsertAction; +import com.metamx.druid.indexing.common.actions.TaskActionClientFactory; +import com.metamx.druid.indexing.common.actions.TaskActionToolbox; +import com.metamx.druid.indexing.common.config.TaskConfig; +import com.metamx.druid.indexing.common.task.AbstractTask; +import com.metamx.druid.indexing.common.task.IndexTask; +import com.metamx.druid.indexing.common.task.KillTask; +import com.metamx.druid.indexing.common.task.Task; +import com.metamx.druid.indexing.coordinator.exec.TaskConsumer; import com.metamx.druid.realtime.firehose.Firehose; import com.metamx.druid.realtime.firehose.FirehoseFactory; import com.metamx.emitter.EmittingLogger; @@ -273,90 +273,6 @@ public class TaskLifecycleTest @Test public void testRealtimeishTask() throws Exception { - class RealtimeishTask extends AbstractTask - { - RealtimeishTask() - { - super("rt1", "rt", "foo", null); - } - - @Override - public String getType() - { - return "realtime_test"; - } - - @Override - public TaskStatus run(TaskToolbox toolbox) throws Exception - { - final Interval interval1 = new Interval("2010-01-01T00/PT1H"); - final Interval interval2 = new Interval("2010-01-01T01/PT1H"); - - // Sort of similar to what realtime tasks do: - - // Acquire lock for first interval - final TaskLock lock1 = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval1)); - final List locks1 = toolbox.getTaskActionClient().submit(new LockListAction()); - - // (Confirm lock sanity) - Assert.assertEquals("lock1 interval", interval1, lock1.getInterval()); - Assert.assertEquals("locks1", ImmutableList.of(lock1), locks1); - - // Acquire lock for second interval - final TaskLock lock2 = toolbox.getTaskActionClient().submit(new LockAcquireAction(interval2)); - final List locks2 = toolbox.getTaskActionClient().submit(new LockListAction()); - - // (Confirm lock sanity) - Assert.assertEquals("lock2 interval", interval2, lock2.getInterval()); - Assert.assertEquals("locks2", ImmutableList.of(lock1, lock2), locks2); - - // Push first segment - toolbox.getTaskActionClient() - .submit( - new SegmentInsertAction( - ImmutableSet.of( - DataSegment.builder() - .dataSource("foo") - .interval(interval1) - .version(lock1.getVersion()) - .build() - ) - ) - ); - - // Release first lock - toolbox.getTaskActionClient().submit(new LockReleaseAction(interval1)); - final List locks3 = toolbox.getTaskActionClient().submit(new LockListAction()); - - // (Confirm lock sanity) - Assert.assertEquals("locks3", ImmutableList.of(lock2), locks3); - - // Push second segment - toolbox.getTaskActionClient() - .submit( - new SegmentInsertAction( - ImmutableSet.of( - DataSegment.builder() - .dataSource("foo") - .interval(interval2) - .version(lock2.getVersion()) - .build() - ) - ) - ); - - // Release second lock - toolbox.getTaskActionClient().submit(new LockReleaseAction(interval2)); - final List locks4 = toolbox.getTaskActionClient().submit(new LockListAction()); - - // (Confirm lock sanity) - Assert.assertEquals("locks4", ImmutableList.of(), locks4); - - // Exit - return TaskStatus.success(getId()); - } - } - final Task rtishTask = new RealtimeishTask(); final TaskStatus status = runTask(rtishTask); @@ -368,7 +284,7 @@ public class TaskLifecycleTest @Test public void testSimple() throws Exception { - final Task task = new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D")) + final Task task = new AbstractTask("id1", "id1", "id1", "ds", new Interval("2012-01-01/P1D")) { @Override public String getType() @@ -405,7 +321,7 @@ public class TaskLifecycleTest @Test public void testBadInterval() throws Exception { - final Task task = new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D")) + final Task task = new AbstractTask("id1", "id1", "id1", "ds", new Interval("2012-01-01/P1D")) { @Override public String getType() @@ -439,7 +355,7 @@ public class TaskLifecycleTest @Test public void testBadVersion() throws Exception { - final Task task = new AbstractTask("id1", "id1", "ds", new Interval("2012-01-01/P1D")) + final Task task = new AbstractTask("id1", "id1", "id1", "ds", new Interval("2012-01-01/P1D")) { @Override public String getType() diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskQueueTest.java similarity index 94% rename from merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java rename to indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskQueueTest.java index 0a1968546c9..4e023b736dd 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/TaskQueueTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TaskQueueTest.java @@ -17,21 +17,21 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator; +package com.metamx.druid.indexing.coordinator; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import com.metamx.druid.merger.common.TaskLock; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.TaskToolbox; -import com.metamx.druid.merger.common.TaskToolboxFactory; -import com.metamx.druid.merger.common.actions.LocalTaskActionClientFactory; -import com.metamx.druid.merger.common.actions.SpawnTasksAction; -import com.metamx.druid.merger.common.actions.TaskActionToolbox; -import com.metamx.druid.merger.common.task.AbstractTask; -import com.metamx.druid.merger.common.task.Task; +import com.metamx.druid.indexing.common.TaskLock; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.TaskToolbox; +import com.metamx.druid.indexing.common.TaskToolboxFactory; +import com.metamx.druid.indexing.common.actions.LocalTaskActionClientFactory; +import com.metamx.druid.indexing.common.actions.SpawnTasksAction; +import com.metamx.druid.indexing.common.actions.TaskActionToolbox; +import com.metamx.druid.indexing.common.task.AbstractTask; +import com.metamx.druid.indexing.common.task.Task; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; @@ -346,7 +346,7 @@ public class TaskQueueTest private static Task newTask(final String id, final String groupId, final String dataSource, final Interval interval) { - return new AbstractTask(id, groupId, dataSource, interval) + return new AbstractTask(id, groupId, id, dataSource, interval) { @Override public TaskStatus run(TaskToolbox toolbox) throws Exception @@ -370,7 +370,7 @@ public class TaskQueueTest final List nextTasks ) { - return new AbstractTask(id, groupId, dataSource, interval) + return new AbstractTask(id, groupId, id, dataSource, interval) { @Override public String getType() diff --git a/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TestRealtimeTask.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TestRealtimeTask.java new file mode 100644 index 00000000000..515b75e3e1f --- /dev/null +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/TestRealtimeTask.java @@ -0,0 +1,78 @@ +/* + * 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.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.metamx.druid.QueryGranularity; +import com.metamx.druid.aggregation.AggregatorFactory; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.TaskToolbox; +import com.metamx.druid.indexing.common.task.RealtimeIndexTask; +import com.metamx.druid.realtime.Schema; +import com.metamx.druid.shard.NoneShardSpec; + +/** + */ +@JsonTypeName("test_realtime") +public class TestRealtimeTask extends RealtimeIndexTask +{ + private final TaskStatus status; + + @JsonCreator + public TestRealtimeTask( + @JsonProperty("id") String id, + @JsonProperty("availabilityGroup") String availGroup, + @JsonProperty("dataSource") String dataSource, + @JsonProperty("taskStatus") TaskStatus status + ) + { + super( + id, + availGroup, + new Schema(dataSource, null, new AggregatorFactory[]{}, QueryGranularity.NONE, new NoneShardSpec()), + null, + null, + null, + null + ); + this.status = status; + } + + @Override + @JsonProperty + public String getType() + { + return "test_realtime"; + } + + @JsonProperty + public TaskStatus getStatus() + { + return status; + } + + @Override + public TaskStatus run(TaskToolbox toolbox) throws Exception + { + return status; + } +} diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategyTest.java similarity index 93% rename from merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java rename to indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategyTest.java index e6ab815e262..eba026ea620 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/EC2AutoScalingStrategyTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/EC2AutoScalingStrategyTest.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.scaling; +package com.metamx.druid.indexing.coordinator.scaling; import com.amazonaws.services.ec2.AmazonEC2Client; import com.amazonaws.services.ec2.model.DescribeInstancesRequest; @@ -29,10 +29,10 @@ import com.amazonaws.services.ec2.model.RunInstancesResult; import com.amazonaws.services.ec2.model.TerminateInstancesRequest; import com.google.common.collect.Lists; import com.metamx.druid.jackson.DefaultObjectMapper; -import com.metamx.druid.merger.coordinator.config.EC2AutoScalingStrategyConfig; -import com.metamx.druid.merger.coordinator.setup.EC2NodeData; -import com.metamx.druid.merger.coordinator.setup.GalaxyUserData; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; +import com.metamx.druid.indexing.coordinator.config.EC2AutoScalingStrategyConfig; +import com.metamx.druid.indexing.coordinator.setup.EC2NodeData; +import com.metamx.druid.indexing.coordinator.setup.GalaxyUserData; +import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; import org.easymock.EasyMock; import org.junit.After; import org.junit.Assert; diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java similarity index 96% rename from merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java rename to indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java index c4ceed76c37..c44d555f798 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/SimpleResourceManagementStrategyTest.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/SimpleResourceManagementStrategyTest.java @@ -17,21 +17,20 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.scaling; +package com.metamx.druid.indexing.coordinator.scaling; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.metamx.druid.aggregation.AggregatorFactory; import com.metamx.druid.client.DataSegment; -import com.metamx.druid.merger.TestTask; -import com.metamx.druid.merger.common.TaskStatus; -import com.metamx.druid.merger.common.task.Task; -import com.metamx.druid.merger.coordinator.TaskRunnerWorkItem; -import com.metamx.druid.merger.coordinator.ZkWorker; -import com.metamx.druid.merger.coordinator.setup.WorkerSetupData; -import com.metamx.druid.merger.worker.Worker; +import com.metamx.druid.indexing.TestTask; +import com.metamx.druid.indexing.common.TaskStatus; +import com.metamx.druid.indexing.common.task.Task; +import com.metamx.druid.indexing.coordinator.TaskRunnerWorkItem; +import com.metamx.druid.indexing.coordinator.ZkWorker; +import com.metamx.druid.indexing.coordinator.setup.WorkerSetupData; +import com.metamx.druid.indexing.worker.Worker; import com.metamx.emitter.EmittingLogger; -import com.metamx.emitter.core.Event; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEventBuilder; import junit.framework.Assert; diff --git a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/TestAutoScalingStrategy.java b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/TestAutoScalingStrategy.java similarity index 95% rename from merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/TestAutoScalingStrategy.java rename to indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/TestAutoScalingStrategy.java index 9618dc7af48..e86180c79ee 100644 --- a/merger/src/test/java/com/metamx/druid/merger/coordinator/scaling/TestAutoScalingStrategy.java +++ b/indexing-service/src/test/java/com/metamx/druid/indexing/coordinator/scaling/TestAutoScalingStrategy.java @@ -17,7 +17,7 @@ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. */ -package com.metamx.druid.merger.coordinator.scaling; +package com.metamx.druid.indexing.coordinator.scaling; import java.util.List; diff --git a/pom.xml b/pom.xml index 15060a4c39b..b9483992da0 100644 --- a/pom.xml +++ b/pom.xml @@ -18,12 +18,13 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx druid pom - 0.4.33-SNAPSHOT + 0.5.0-SNAPSHOT druid druid @@ -39,18 +40,18 @@ UTF-8 0.22.3 - 2.0.2-21-22 + 2.0.2-21-22 - common - index-common client - indexer - server - merger - realtime + common examples + indexing-common + indexing-hadoop + indexing-service + realtime + server services @@ -75,7 +76,7 @@ com.metamx bytebuffer-collections - 0.0.1 + 0.0.2 com.metamx @@ -126,22 +127,22 @@ org.apache.curator curator-client - ${netflix.curator.version} + ${apache.curator.version} org.apache.curator curator-framework - ${netflix.curator.version} + ${apache.curator.version} org.apache.curator curator-recipes - ${netflix.curator.version} + ${apache.curator.version} org.apache.curator curator-x-discovery - ${netflix.curator.version} + ${apache.curator.version} org.apache.hadoop @@ -180,24 +181,24 @@ 4.8.1 - com.fasterxml.jackson.core - jackson-annotations - 2.1.4 + com.fasterxml.jackson.core + jackson-annotations + 2.1.4 - com.fasterxml.jackson.core - jackson-core - 2.1.4 + com.fasterxml.jackson.core + jackson-core + 2.1.4 - com.fasterxml.jackson.core - jackson-databind - 2.1.4 + com.fasterxml.jackson.core + jackson-databind + 2.1.4 - com.fasterxml.jackson.datatype - jackson-datatype-guava - 2.1.2 + com.fasterxml.jackson.datatype + jackson-datatype-guava + 2.1.2 com.fasterxml.jackson.datatype @@ -253,12 +254,12 @@ org.mortbay.jetty jetty 6.1.26 - - - org.mortbay.jetty - servlet-api - - + + + org.mortbay.jetty + servlet-api + + org.mortbay.jetty @@ -310,8 +311,16 @@ commons-cli 1.2 - - + + org.codehaus.jackson + jackson-core-asl + 1.9.11 + + + org.codehaus.jackson + jackson-mapper-asl + 1.9.11 + @@ -342,7 +351,7 @@ org.apache.curator curator-test - ${netflix.curator.version} + ${apache.curator.version} test diff --git a/realtime/pom.xml b/realtime/pom.xml index 98d76e67f72..daf2dca271e 100644 --- a/realtime/pom.xml +++ b/realtime/pom.xml @@ -28,7 +28,7 @@ com.metamx druid - 0.4.33-SNAPSHOT + 0.5.0-SNAPSHOT diff --git a/server/pom.xml b/server/pom.xml index 7dc3376eb12..be9fc9a2e7d 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -18,17 +18,18 @@ ~ Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA. --> - + 4.0.0 com.metamx.druid druid-server druid-server - druid-server + Druid Server com.metamx druid - 0.4.33-SNAPSHOT + 0.5.0-SNAPSHOT @@ -39,7 +40,7 @@ com.metamx.druid - druid-index-common + druid-indexing-common ${project.parent.version} @@ -195,7 +196,7 @@ true - com.netflix.astyanax + com.netflix.astyanax astyanax 1.0.1 @@ -214,7 +215,7 @@ com.metamx.druid - druid-index-common + druid-indexing-common ${project.parent.version} test-jar test @@ -233,7 +234,6 @@ - diff --git a/server/src/main/java/com/metamx/druid/http/InfoResource.java b/server/src/main/java/com/metamx/druid/http/InfoResource.java index 931eb4a6f79..d858c72467a 100644 --- a/server/src/main/java/com/metamx/druid/http/InfoResource.java +++ b/server/src/main/java/com/metamx/druid/http/InfoResource.java @@ -60,6 +60,42 @@ import java.util.TreeSet; @Path("/info") public class InfoResource { + private static final Function> simplifyClusterFn = + new Function>() + { + @Override + public Map apply(DruidServer server) + { + return new ImmutableMap.Builder() + .put("host", server.getHost()) + .put("type", server.getType()) + .put("tier", server.getTier()) + .put("currSize", server.getCurrSize()) + .put("maxSize", server.getMaxSize()) + .put( + "segments", + Collections2.transform( + server.getSegments().values(), + new Function>() + { + @Override + public Map apply(DataSegment segment) + { + return new ImmutableMap.Builder() + .put("id", segment.getIdentifier()) + .put("dataSource", segment.getDimensions()) + .put("interval", segment.getInterval()) + .put("version", segment.getVersion()) + .put("size", segment.getSize()) + .build(); + } + } + ) + ) + .build(); + } + }; + private final DruidMaster master; private final InventoryView serverInventoryView; private final DatabaseSegmentManager databaseSegmentManager; @@ -96,11 +132,20 @@ public class InfoResource @GET @Path("/cluster") @Produces("application/json") - public Response getClusterInfo() + public Response getClusterInfo( + @QueryParam("full") String full + ) { - return Response.status(Response.Status.OK) - .entity(serverInventoryView.getInventory()) - .build(); + if (full != null) { + return Response.ok(serverInventoryView.getInventory()) + .build(); + } + return Response.ok( + Iterables.transform( + serverInventoryView.getInventory(), + simplifyClusterFn + ) + ).build(); } @GET @@ -301,11 +346,16 @@ public class InfoResource @Path("/rules/{dataSourceName}") @Produces("application/json") public Response getDatasourceRules( - @PathParam("dataSourceName") final String dataSourceName + @PathParam("dataSourceName") final String dataSourceName, + @QueryParam("full") final String full + ) { - return Response.status(Response.Status.OK) - .entity(databaseRuleManager.getRules(dataSourceName)) + if (full != null) { + return Response.ok(databaseRuleManager.getRulesWithDefault(dataSourceName)) + .build(); + } + return Response.ok(databaseRuleManager.getRules(dataSourceName)) .build(); } @@ -350,23 +400,6 @@ public class InfoResource ).build(); } - @GET - @Path("/rules/{dataSourceName}") - @Produces("application/json") - public Response getDatasourceRules( - @PathParam("dataSourceName") final String dataSourceName, - @QueryParam("full") final String full - - ) - { - if (full != null) { - return Response.ok(databaseRuleManager.getRulesWithDefault(dataSourceName)) - .build(); - } - return Response.ok(databaseRuleManager.getRules(dataSourceName)) - .build(); - } - @DELETE @Path("/datasources/{dataSourceName}") public Response deleteDataSource( diff --git a/server/src/main/java/com/metamx/druid/index/brita/SpatialFilter.java b/server/src/main/java/com/metamx/druid/index/brita/SpatialFilter.java index 2abb55035fe..4d7e4e36f6a 100644 --- a/server/src/main/java/com/metamx/druid/index/brita/SpatialFilter.java +++ b/server/src/main/java/com/metamx/druid/index/brita/SpatialFilter.java @@ -43,48 +43,7 @@ public class SpatialFilter implements Filter @Override public ImmutableConciseSet goConcise(final BitmapIndexSelector selector) { - final Iterator dimValueIndexesIter = selector.getSpatialIndex(dimension).search(bound) - .iterator(); - ImmutableConciseSet retVal = ImmutableConciseSet.union( - new Iterable() - { - @Override - public Iterator iterator() - { - return new Iterator() - { - private IntSet.IntIterator iter; - - @Override - public boolean hasNext() - { - return dimValueIndexesIter.hasNext() || (iter != null && iter.hasNext()); - } - - @Override - public ImmutableConciseSet next() - { - if (iter != null && !iter.hasNext()) { - iter = null; - } - if (iter == null) { - ImmutableConciseSet immutableConciseSet = dimValueIndexesIter.next(); - iter = immutableConciseSet.iterator(); - } - return selector.getConciseInvertedIndex(dimension, iter.next()); - } - - @Override - public void remove() - { - throw new UnsupportedOperationException(); - } - }; - } - } - ); - - return retVal; + return ImmutableConciseSet.union(selector.getSpatialIndex(dimension).search(bound)); } @Override diff --git a/server/src/main/resources/static/cluster.html b/server/src/main/resources/static/cluster.html index b6b0c5c2c5f..5bc9b857999 100644 --- a/server/src/main/resources/static/cluster.html +++ b/server/src/main/resources/static/cluster.html @@ -48,7 +48,7 @@

    Loading segment data... this may take a few minutes
    -
    User Console
    + - + + --> \ No newline at end of file diff --git a/server/src/main/resources/static/css/demo_table.css b/server/src/main/resources/static/css/demo_table.css index d14a80b7b46..679a1066e58 100644 --- a/server/src/main/resources/static/css/demo_table.css +++ b/server/src/main/resources/static/css/demo_table.css @@ -109,7 +109,7 @@ table { font-size:12px; clear:both; padding-right:20px; - cursor:pointer; + /*cursor:pointer;*/ } td, tr { diff --git a/server/src/main/resources/static/js/init-0.0.2.js b/server/src/main/resources/static/js/init-0.0.2.js index 18377266a99..b18a51751ac 100644 --- a/server/src/main/resources/static/js/init-0.0.2.js +++ b/server/src/main/resources/static/js/init-0.0.2.js @@ -26,7 +26,6 @@ $(document).ready(function() { // build segment table for (var segment in segments) { segmentTable.setCell(segmentRow, 'Server host', data[server]['host']); - segmentTable.setCell(segmentRow, 'Segment name', segment); for (field in segments[segment]) { if (!(segments[segment][field] instanceof Object)) { @@ -119,6 +118,7 @@ $(document).ready(function() { }); } +/* $.ajax({ url:"/master/move", type: "POST", @@ -134,11 +134,13 @@ $(document).ready(function() { } } }); - +*/ return false; }); - /*$('#drop_segment').submit(function() { + +/*$ + ('#drop_segment').submit(function() { var data = []; if ($.isEmptyObject(CONSOLE.selected_segments)) { @@ -163,7 +165,8 @@ $(document).ready(function() { }); return false; - });*/ + }); +*/ $.get('/info/cluster', function(data) { $('.loading').hide(); @@ -175,7 +178,7 @@ $(document).ready(function() { initDataTable($('#segments'), oTable); // init select segments - $("#segments tbody").click(function(event) { + /*$("#segments tbody").click(function(event) { var el = $(event.target.parentNode); var key = el.children('.segment_name').text(); if (el.is("tr")) { @@ -193,6 +196,6 @@ $(document).ready(function() { } $('#selected_segments').html(html); } - }); + });*/ }); }); \ No newline at end of file diff --git a/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java b/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java index 5e74a37990d..8af66f30eb4 100644 --- a/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java +++ b/server/src/test/java/com/metamx/druid/index/v1/IndexMergerTest.java @@ -42,6 +42,7 @@ public class IndexMergerTest public void testPersistCaseInsensitive() throws Exception { final long timestamp = System.currentTimeMillis(); + IncrementalIndex toPersist = IncrementalIndexTest.createCaseInsensitiveIndex(timestamp); final File tempDir = Files.createTempDir(); diff --git a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java index ef1567f0e81..b60b8ce825e 100644 --- a/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java +++ b/server/src/test/java/com/metamx/druid/query/group/GroupByQueryRunnerTest.java @@ -43,6 +43,7 @@ import com.metamx.druid.query.QueryRunner; import com.metamx.druid.query.QueryRunnerTestHelper; import com.metamx.druid.query.dimension.DefaultDimensionSpec; import com.metamx.druid.query.dimension.DimensionSpec; +import com.metamx.druid.query.filter.RegexDimFilter; import com.metamx.druid.query.group.having.EqualToHavingSpec; import com.metamx.druid.query.group.having.GreaterThanHavingSpec; import com.metamx.druid.query.group.having.OrHavingSpec; @@ -633,6 +634,32 @@ public class GroupByQueryRunnerTest TestHelper.assertExpectedObjects(expectedResults, mergedRunner.run(fullQuery), "merged"); } + @Test + public void testGroupByWithRegEx() throws Exception + { + GroupByQuery.Builder builder = GroupByQuery + .builder() + .setDataSource(QueryRunnerTestHelper.dataSource) + .setInterval("2011-04-02/2011-04-04") + .setDimFilter(new RegexDimFilter("quality", "auto.*")) + .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("quality", "quality"))) + .setAggregatorSpecs( + Arrays.asList( + QueryRunnerTestHelper.rowsCount + ) + ) + .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)); + + final GroupByQuery query = builder.build(); + + List expectedResults = Arrays.asList( + createExpectedRow("2011-04-01", "quality", "automotive", "rows", 2L) + ); + + QueryRunner mergeRunner = new GroupByQueryQueryToolChest().mergeResults(runner); + TestHelper.assertExpectedObjects(expectedResults, mergeRunner.run(query), "no-limit"); + } + private Row createExpectedRow(final String timestamp, Object... vals) { return createExpectedRow(new DateTime(timestamp), vals); diff --git a/services/pom.xml b/services/pom.xml index 046d9b54f84..e8dff6c3aa3 100644 --- a/services/pom.xml +++ b/services/pom.xml @@ -24,11 +24,11 @@ druid-services druid-services druid-services - 0.4.33-SNAPSHOT + 0.5.0-SNAPSHOT com.metamx druid - 0.4.33-SNAPSHOT + 0.5.0-SNAPSHOT